diff --git a/CHANGELOG.md b/CHANGELOG.md index 58c38dc120a..607f650deeb 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -9,6 +9,9 @@ * `RENAME` queries now work with all storages. [#5953](https://github.com/yandex/ClickHouse/pull/5953) ([Ivan](https://github.com/abyss7)) * Now client receive logs from server with any desired level by setting `send_logs_level` regardless to the log level specified in server settings. [#5964](https://github.com/yandex/ClickHouse/pull/5964) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +### Backward Incompatible Change +* The setting `input_format_defaults_for_omitted_fields` is enabled by default. Inserts in Distibuted tables need this setting to be the same on cluster (you need to set it before rolling update). It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behaviour but may lead to negligible performance difference. [#6043](https://github.com/yandex/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/yandex/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) + ### Experimental features * New query processing pipeline. Use `experimental_use_processors=1` option to enable it. Use for your own trouble. [#4914](https://github.com/yandex/ClickHouse/pull/4914) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -18,7 +21,6 @@ * Fixed overestimation of `max_rows_to_read` if the setting `merge_tree_uniform_read_distribution` is set to 0. [#6019](https://github.com/yandex/ClickHouse/pull/6019) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### Improvement -* The setting `input_format_defaults_for_omitted_fields` is enabled by default. It enables calculation of complex default expressions for omitted fields in `JSONEachRow` and `CSV*` formats. It should be the expected behaviour but may lead to negligible performance difference or subtle incompatibilities. [#6043](https://github.com/yandex/ClickHouse/pull/6043) ([Artem Zuikov](https://github.com/4ertus2)), [#5625](https://github.com/yandex/ClickHouse/pull/5625) ([akuzm](https://github.com/akuzm)) * Throws an exception if `config.d` file doesn't have the corresponding root element as the config file [#6123](https://github.com/yandex/ClickHouse/pull/6123) ([dimarub2000](https://github.com/dimarub2000)) ### Performance Improvement diff --git a/cmake/find_parquet.cmake b/cmake/find_parquet.cmake index 39af93955f4..63f589a9ea5 100644 --- a/cmake/find_parquet.cmake +++ b/cmake/find_parquet.cmake @@ -2,7 +2,7 @@ option (ENABLE_PARQUET "Enable parquet" ON) if (ENABLE_PARQUET) -if (NOT OS_FREEBSD) # Freebsd: ../contrib/arrow/cpp/src/arrow/util/bit-util.h:27:10: fatal error: endian.h: No such file or directory +if (NOT OS_FREEBSD AND NOT APPLE) # Freebsd: ../contrib/arrow/cpp/src/arrow/util/bit-util.h:27:10: fatal error: endian.h: No such file or directory option(USE_INTERNAL_PARQUET_LIBRARY "Set to FALSE to use system parquet library instead of bundled" ${NOT_UNBUNDLED}) endif() diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index 435d06da854..7ac03dfdd3b 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -1932,15 +1932,13 @@ protected: TaskTable & task_table = task_shard.task_table; - String query; - { - WriteBufferFromOwnString wb; - wb << "SELECT 1" - << " FROM "<< getQuotedTable(task_shard.table_read_shard) - << " WHERE " << queryToString(task_table.engine_push_partition_key_ast) << " = " << partition_quoted_name - << " LIMIT 1"; - query = wb.str(); - } + std::string query = "SELECT 1 FROM " + getQuotedTable(task_shard.table_read_shard) + + " WHERE (" + queryToString(task_table.engine_push_partition_key_ast) + " = (" + partition_quoted_name + " AS partition_key))"; + + if (!task_table.where_condition_str.empty()) + query += " AND (" + task_table.where_condition_str + ")"; + + query += " LIMIT 1"; LOG_DEBUG(log, "Checking shard " << task_shard.getDescription() << " for partition " << partition_quoted_name << " existence, executing query: " << query); diff --git a/dbms/src/Common/Dwarf.cpp b/dbms/src/Common/Dwarf.cpp index 9d4a2f05632..b7dc6a6eaa9 100644 --- a/dbms/src/Common/Dwarf.cpp +++ b/dbms/src/Common/Dwarf.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + /* * Copyright 2012-present Facebook, Inc. * @@ -1031,3 +1033,5 @@ bool Dwarf::LineNumberVM::findAddress(uintptr_t target, Path & file, uint64_t & } } + +#endif diff --git a/dbms/src/Common/Dwarf.h b/dbms/src/Common/Dwarf.h index 5bc358df863..216b9bca1a9 100644 --- a/dbms/src/Common/Dwarf.h +++ b/dbms/src/Common/Dwarf.h @@ -1,5 +1,7 @@ #pragma once +#ifdef __ELF__ + /* * Copyright 2012-present Facebook, Inc. * @@ -285,3 +287,5 @@ private: }; } + +#endif diff --git a/dbms/src/Common/Elf.cpp b/dbms/src/Common/Elf.cpp index 035477d0243..594264c0099 100644 --- a/dbms/src/Common/Elf.cpp +++ b/dbms/src/Common/Elf.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include @@ -128,3 +130,5 @@ size_t Elf::Section::size() const } } + +#endif diff --git a/dbms/src/Common/Elf.h b/dbms/src/Common/Elf.h index 869b869b530..f0576ab7d58 100644 --- a/dbms/src/Common/Elf.h +++ b/dbms/src/Common/Elf.h @@ -1,5 +1,7 @@ #pragma once +#ifdef __ELF__ + #include #include @@ -61,3 +63,5 @@ private: }; } + +#endif diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 4128ddb8edc..7c6ef556aed 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -443,6 +443,7 @@ namespace ErrorCodes extern const int INSECURE_PATH = 466; extern const int CANNOT_PARSE_BOOL = 467; extern const int CANNOT_PTHREAD_ATTR = 468; + extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 469; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index 9792f2fb1f1..a642ec2b73a 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -250,6 +250,7 @@ static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offs if (size == 0) return callback(""); +#ifdef __ELF__ const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance(); std::unordered_map dwarfs; @@ -290,6 +291,18 @@ static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offs callback(out.str()); out.str({}); } +#else + std::stringstream out; + + for (size_t i = offset; i < size; ++i) + { + const void * addr = frames[i]; + out << i << ". " << addr; + + callback(out.str()); + out.str({}); + } +#endif } static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size) diff --git a/dbms/src/Common/SymbolIndex.cpp b/dbms/src/Common/SymbolIndex.cpp index d4cf41342a0..05268c4b6f1 100644 --- a/dbms/src/Common/SymbolIndex.cpp +++ b/dbms/src/Common/SymbolIndex.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include @@ -316,3 +318,5 @@ const SymbolIndex::Object * SymbolIndex::findObject(const void * address) const } } + +#endif diff --git a/dbms/src/Common/SymbolIndex.h b/dbms/src/Common/SymbolIndex.h index 41a773f5f4a..01afe9bf7fa 100644 --- a/dbms/src/Common/SymbolIndex.h +++ b/dbms/src/Common/SymbolIndex.h @@ -1,5 +1,7 @@ #pragma once +#ifdef __ELF__ + #include #include #include @@ -53,3 +55,5 @@ private: }; } + +#endif diff --git a/dbms/src/Common/UInt128.h b/dbms/src/Common/UInt128.h index b895c514c3e..9dc15bba758 100644 --- a/dbms/src/Common/UInt128.h +++ b/dbms/src/Common/UInt128.h @@ -1,7 +1,8 @@ #pragma once #include - +#include +#include #include #include @@ -33,6 +34,13 @@ struct UInt128 auto tuple() const { return std::tie(high, low); } + String toHexString() const + { + std::ostringstream os; + os << std::setw(16) << std::setfill('0') << std::hex << high << low; + return String(os.str()); + } + bool inline operator== (const UInt128 rhs) const { return tuple() == rhs.tuple(); } bool inline operator!= (const UInt128 rhs) const { return tuple() != rhs.tuple(); } bool inline operator< (const UInt128 rhs) const { return tuple() < rhs.tuple(); } diff --git a/dbms/src/Common/tests/symbol_index.cpp b/dbms/src/Common/tests/symbol_index.cpp index 6c0d303fe35..dde5ce185ae 100644 --- a/dbms/src/Common/tests/symbol_index.cpp +++ b/dbms/src/Common/tests/symbol_index.cpp @@ -16,6 +16,7 @@ using namespace DB; int main(int argc, char ** argv) { +#ifdef __ELF__ if (argc < 2) { std::cerr << "Usage: ./symbol_index address\n"; @@ -53,6 +54,12 @@ int main(int argc, char ** argv) std::cerr << "\n"; std::cerr << StackTrace().toString() << "\n"; +#else + (void)argc; + (void)argv; + + std::cerr << "This test does not make sense for non-ELF objects.\n"; +#endif return 0; } diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index d3ce0c67b79..4a93e5ed803 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -144,7 +144,8 @@ private: using Blocks = std::vector; using BlocksList = std::list; - +using BlocksPtr = std::shared_ptr; +using BlocksPtrs = std::shared_ptr>; /// Compare number of columns, data types, column types, column names, and values of constant columns. bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs); diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index be957340bab..0c72c926006 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -32,7 +32,11 @@ */ #define DEFAULT_MERGE_BLOCK_SIZE 8192 +#define DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC 5 +#define DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC 15 +#define DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS 10000 #define SHOW_CHARS_ON_SYNTAX_ERROR ptrdiff_t(160) +#define DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC 15 #define DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE 1024 #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3 /// each period reduces the error counter by 2 times diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 877f99856c9..22b0b5c8d03 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -345,6 +345,12 @@ struct Settings : public SettingsCollection /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ \ M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \ + \ + M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \ + M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \ + M(SettingSeconds, temporary_live_channel_timeout, DEFAULT_TEMPORARY_LIVE_CHANNEL_TIMEOUT_SEC, "Timeout after which temporary live channel is deleted.") \ + M(SettingMilliseconds, alter_channel_wait_ms, DEFAULT_ALTER_LIVE_CHANNEL_WAIT_MS, "The wait time for alter channel request.") \ + M(SettingUInt64, max_live_view_insert_blocks_before_refresh, 64, "Limit maximum number of inserted blocks after which mergeable blocks are dropped and query is re-executed.") \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/DataStreams/BlocksBlockInputStream.h b/dbms/src/DataStreams/BlocksBlockInputStream.h new file mode 100644 index 00000000000..27407acbfae --- /dev/null +++ b/dbms/src/DataStreams/BlocksBlockInputStream.h @@ -0,0 +1,51 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + + +namespace DB +{ + +/** A stream of blocks from a shared vector of blocks + */ +class BlocksBlockInputStream : public IBlockInputStream +{ +public: + /// Acquires shared ownership of the blocks vector + BlocksBlockInputStream(const std::shared_ptr & blocks_ptr_, Block header_) + : blocks(*blocks_ptr_), it((*blocks_ptr_)->begin()), end((*blocks_ptr_)->end()), header(std::move(header_)) {} + + String getName() const override { return "Blocks"; } + + Block getHeader() const override { return header; } + +protected: + Block readImpl() override + { + if (it == end) + return Block(); + + Block res = *it; + ++it; + return res; + } + +private: + BlocksPtr blocks; + Blocks::iterator it; + const Blocks::iterator end; + Block header; +}; + +} diff --git a/dbms/src/DataStreams/LiveViewBlockInputStream.h b/dbms/src/DataStreams/LiveViewBlockInputStream.h new file mode 100644 index 00000000000..b3756c9ff6d --- /dev/null +++ b/dbms/src/DataStreams/LiveViewBlockInputStream.h @@ -0,0 +1,222 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Implements LIVE VIEW table WATCH input stream. + * Keeps stream alive by outputing blocks with no rows + * based on period specified by the heartbeat interval. + */ +class LiveViewBlockInputStream : public IBlockInputStream +{ + +using NonBlockingResult = std::pair; + +public: + ~LiveViewBlockInputStream() override + { + /// Start storage no users thread + /// if we are the last active user + if (!storage->is_dropped && blocks_ptr.use_count() < 3) + storage->startNoUsersThread(temporary_live_view_timeout_sec); + } + + LiveViewBlockInputStream(std::shared_ptr storage_, + std::shared_ptr blocks_ptr_, + std::shared_ptr blocks_metadata_ptr_, + std::shared_ptr active_ptr_, + const bool has_limit_, const UInt64 limit_, + const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_live_view_timeout_sec_) + : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + { + /// grab active pointer + active = active_ptr.lock(); + } + + String getName() const override { return "LiveViewBlockInputStream"; } + + void cancel(bool kill) override + { + if (isCancelled() || storage->is_dropped) + return; + IBlockInputStream::cancel(kill); + Poco::FastMutex::ScopedLock lock(storage->mutex); + storage->condition.broadcast(); + } + + Block getHeader() const override { return storage->getHeader(); } + + void refresh() + { + if (active && blocks && it == end) + it = blocks->begin(); + } + + void suspend() + { + active.reset(); + } + + void resume() + { + active = active_ptr.lock(); + { + if (!blocks || blocks.get() != (*blocks_ptr).get()) + blocks = (*blocks_ptr); + } + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + NonBlockingResult tryRead() + { + return tryRead_(false); + } + +protected: + Block readImpl() override + { + /// try reading + return tryRead_(true).first; + } + + /** tryRead method attempts to read a block in either blocking + * or non-blocking mode. If blocking is set to false + * then method return empty block with flag set to false + * to indicate that method would block to get the next block. + */ + NonBlockingResult tryRead_(bool blocking) + { + Block res; + + if (has_limit && num_updates == static_cast(limit)) + { + return { Block(), true }; + } + /// If blocks were never assigned get blocks + if (!blocks) + { + Poco::FastMutex::ScopedLock lock(storage->mutex); + if (!active) + return { Block(), false }; + blocks = (*blocks_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + if (isCancelled() || storage->is_dropped) + { + return { Block(), true }; + } + + if (it == end) + { + { + Poco::FastMutex::ScopedLock lock(storage->mutex); + if (!active) + return { Block(), false }; + /// If we are done iterating over our blocks + /// and there are new blocks availble then get them + if (blocks.get() != (*blocks_ptr).get()) + { + blocks = (*blocks_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + /// No new blocks available wait for new ones + else + { + if (!blocking) + { + return { Block(), false }; + } + if (!end_of_blocks) + { + end_of_blocks = true; + return { getHeader(), true }; + } + while (true) + { + UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); + bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000); + + if (isCancelled() || storage->is_dropped) + { + return { Block(), true }; + } + if (signaled) + { + break; + } + else + { + // heartbeat + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); + return { getHeader(), true }; + } + } + } + } + return tryRead_(blocking); + } + + res = *it; + + ++it; + + if (it == end) + { + end_of_blocks = false; + num_updates += 1; + } + + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); + return { res, true }; + } + +private: + std::shared_ptr storage; + std::shared_ptr blocks_ptr; + std::shared_ptr blocks_metadata_ptr; + std::weak_ptr active_ptr; + std::shared_ptr active; + BlocksPtr blocks; + BlocksMetadataPtr blocks_metadata; + Blocks::iterator it; + Blocks::iterator end; + Blocks::iterator begin; + const bool has_limit; + const UInt64 limit; + Int64 num_updates = -1; + bool end_of_blocks = false; + UInt64 heartbeat_interval_usec; + UInt64 temporary_live_view_timeout_sec; + UInt64 last_event_timestamp_usec = 0; + Poco::Timestamp timestamp; +}; + +} diff --git a/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h new file mode 100644 index 00000000000..93fb6a76372 --- /dev/null +++ b/dbms/src/DataStreams/LiveViewEventsBlockInputStream.h @@ -0,0 +1,243 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Implements LIVE VIEW table WATCH EVENTS input stream. + * Keeps stream alive by outputing blocks with no rows + * based on period specified by the heartbeat interval. + */ +class LiveViewEventsBlockInputStream : public IBlockInputStream +{ + +using NonBlockingResult = std::pair; + +public: + ~LiveViewEventsBlockInputStream() override + { + /// Start storage no users thread + /// if we are the last active user + if (!storage->is_dropped && blocks_ptr.use_count() < 3) + storage->startNoUsersThread(temporary_live_view_timeout_sec); + } + /// length default -2 because we want LIMIT to specify number of updates so that LIMIT 1 waits for 1 update + /// and LIMIT 0 just returns data without waiting for any updates + LiveViewEventsBlockInputStream(std::shared_ptr storage_, + std::shared_ptr blocks_ptr_, + std::shared_ptr blocks_metadata_ptr_, + std::shared_ptr active_ptr_, + const bool has_limit_, const UInt64 limit_, + const UInt64 heartbeat_interval_sec_, + const UInt64 temporary_live_view_timeout_sec_) + : storage(std::move(storage_)), blocks_ptr(std::move(blocks_ptr_)), blocks_metadata_ptr(std::move(blocks_metadata_ptr_)), active_ptr(std::move(active_ptr_)), has_limit(has_limit_), limit(limit_), heartbeat_interval_usec(heartbeat_interval_sec_ * 1000000), temporary_live_view_timeout_sec(temporary_live_view_timeout_sec_) + { + /// grab active pointer + active = active_ptr.lock(); + } + + String getName() const override { return "LiveViewEventsBlockInputStream"; } + + void cancel(bool kill) override + { + if (isCancelled() || storage->is_dropped) + return; + IBlockInputStream::cancel(kill); + Poco::FastMutex::ScopedLock lock(storage->mutex); + storage->condition.broadcast(); + } + + Block getHeader() const override + { + return {ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "version")}; + } + + void refresh() + { + if (active && blocks && it == end) + it = blocks->begin(); + } + + void suspend() + { + active.reset(); + } + + void resume() + { + active = active_ptr.lock(); + { + if (!blocks || blocks.get() != (*blocks_ptr).get()) + { + blocks = (*blocks_ptr); + blocks_metadata = (*blocks_metadata_ptr); + } + } + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + NonBlockingResult tryRead() + { + return tryRead_(false); + } + + Block getEventBlock() + { + Block res{ + ColumnWithTypeAndName( + DataTypeUInt64().createColumnConst(1, blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "version") + }; + return res; + } +protected: + Block readImpl() override + { + /// try reading + return tryRead_(true).first; + } + + /** tryRead method attempts to read a block in either blocking + * or non-blocking mode. If blocking is set to false + * then method return empty block with flag set to false + * to indicate that method would block to get the next block. + */ + NonBlockingResult tryRead_(bool blocking) + { + if (has_limit && num_updates == static_cast(limit)) + { + return { Block(), true }; + } + /// If blocks were never assigned get blocks + if (!blocks) + { + Poco::FastMutex::ScopedLock lock(storage->mutex); + if (!active) + return { Block(), false }; + blocks = (*blocks_ptr); + blocks_metadata = (*blocks_metadata_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + + if (isCancelled() || storage->is_dropped) + { + return { Block(), true }; + } + + if (it == end) + { + { + Poco::FastMutex::ScopedLock lock(storage->mutex); + if (!active) + return { Block(), false }; + /// If we are done iterating over our blocks + /// and there are new blocks availble then get them + if (blocks.get() != (*blocks_ptr).get()) + { + blocks = (*blocks_ptr); + blocks_metadata = (*blocks_metadata_ptr); + it = blocks->begin(); + begin = blocks->begin(); + end = blocks->end(); + } + /// No new blocks available wait for new ones + else + { + if (!blocking) + { + return { Block(), false }; + } + if (!end_of_blocks) + { + end_of_blocks = true; + return { getHeader(), true }; + } + while (true) + { + UInt64 timestamp_usec = static_cast(timestamp.epochMicroseconds()); + bool signaled = storage->condition.tryWait(storage->mutex, std::max(static_cast(0), heartbeat_interval_usec - (timestamp_usec - last_event_timestamp_usec)) / 1000); + + if (isCancelled() || storage->is_dropped) + { + return { Block(), true }; + } + if (signaled) + { + break; + } + else + { + // repeat the event block as a heartbeat + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); + return { getHeader(), true }; + } + } + } + } + return tryRead_(blocking); + } + + // move right to the end + it = end; + + if (it == end) + { + end_of_blocks = false; + num_updates += 1; + } + + last_event_timestamp_usec = static_cast(timestamp.epochMicroseconds()); + + return { getEventBlock(), true }; + } + +private: + std::shared_ptr storage; + std::shared_ptr blocks_ptr; + std::shared_ptr blocks_metadata_ptr; + std::weak_ptr active_ptr; + std::shared_ptr active; + BlocksPtr blocks; + BlocksMetadataPtr blocks_metadata; + Blocks::iterator it; + Blocks::iterator end; + Blocks::iterator begin; + const bool has_limit; + const UInt64 limit; + Int64 num_updates = -1; + bool end_of_blocks = false; + UInt64 heartbeat_interval_usec; + UInt64 temporary_live_view_timeout_sec; + UInt64 last_event_timestamp_usec = 0; + Poco::Timestamp timestamp; +}; + +} diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 0879642c8a9..840d3479ab9 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB { @@ -47,17 +48,30 @@ PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( for (const auto & database_table : dependencies) { auto dependent_table = context.getTable(database_table.first, database_table.second); - auto & materialized_view = dynamic_cast(*dependent_table); - StoragePtr inner_table = materialized_view.getTargetTable(); - auto query = materialized_view.getInnerQuery(); - std::unique_ptr insert = std::make_unique(); - insert->database = inner_table->getDatabaseName(); - insert->table = inner_table->getTableName(); - ASTPtr insert_query_ptr(insert.release()); - InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); - BlockIO io = interpreter.execute(); - views.emplace_back(ViewInfo{query, database_table.first, database_table.second, io.out}); + ASTPtr query; + BlockOutputStreamPtr out; + + if (auto * materialized_view = dynamic_cast(dependent_table.get())) + { + StoragePtr inner_table = materialized_view->getTargetTable(); + query = materialized_view->getInnerQuery(); + std::unique_ptr insert = std::make_unique(); + insert->database = inner_table->getDatabaseName(); + insert->table = inner_table->getTableName(); + ASTPtr insert_query_ptr(insert.release()); + InterpreterInsertQuery interpreter(insert_query_ptr, *views_context); + BlockIO io = interpreter.execute(); + out = io.out; + } + else if (dynamic_cast(dependent_table.get())) + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr(), true); + else + out = std::make_shared( + database_table.first, database_table.second, dependent_table, *views_context, ASTPtr()); + + views.emplace_back(ViewInfo{std::move(query), database_table.first, database_table.second, std::move(out)}); } } @@ -90,10 +104,18 @@ void PushingToViewsBlockOutputStream::write(const Block & block) */ Nested::validateArraySizes(block); - if (output) - /// TODO: to support virtual and alias columns inside MVs, we should return here the inserted block extended - /// with additional columns directly from storage and pass it to MVs instead of raw block. - output->write(block); + if (auto * live_view = dynamic_cast(storage.get())) + { + BlockOutputStreamPtr output_ = std::make_shared(*live_view); + StorageLiveView::writeIntoLiveView(*live_view, block, context, output_); + } + else + { + if (output) + /// TODO: to support virtual and alias columns inside MVs, we should return here the inserted block extended + /// with additional columns directly from storage and pass it to MVs instead of raw block. + output->write(block); + } /// Don't process materialized views if this block is duplicate if (replicated_output && replicated_output->lastBlockIsDuplicate()) @@ -180,20 +202,29 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n try { - /// We create a table with the same name as original table and the same alias columns, - /// but it will contain single block (that is INSERT-ed into main table). - /// InterpreterSelectQuery will do processing of alias columns. - Context local_context = *views_context; - local_context.addViewSource(StorageValues::create(storage->getDatabaseName(), storage->getTableName(), storage->getColumns(), block)); - InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); + BlockInputStreamPtr in; - BlockInputStreamPtr in = std::make_shared(select.execute().in); - /// Squashing is needed here because the materialized view query can generate a lot of blocks - /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY - /// and two-level aggregation is triggered). - in = std::make_shared( - in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); - in = std::make_shared(context, in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); + if (view.query) + { + /// We create a table with the same name as original table and the same alias columns, + /// but it will contain single block (that is INSERT-ed into main table). + /// InterpreterSelectQuery will do processing of alias columns. + Context local_context = *views_context; + local_context.addViewSource( + StorageValues::create(storage->getDatabaseName(), storage->getTableName(), storage->getColumns(), + block)); + InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); + in = std::make_shared(select.execute().in); + + /// Squashing is needed here because the materialized view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + in = std::make_shared( + in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); + in = std::make_shared(context, in, view.out->getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name); + } + else + in = std::make_shared(block); in->readPrefix(); diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index 34b8cb43042..e3f96241b1d 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -5,7 +5,7 @@ #include #include #include - +#include namespace DB { diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index eb576075f80..9e9d47f9516 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -164,7 +164,18 @@ static Block adaptBlockStructure(const Block & block, const Block & header, cons res.info = block.info; for (const auto & elem : header) - res.insert({ castColumn(block.getByName(elem.name), elem.type, context), elem.type, elem.name }); + { + ColumnPtr column; + + if (elem.column && isColumnConst(*elem.column)) + /// TODO: check that column from block contains the same value. + /// TODO: serialize const columns. + column = elem.column->cloneResized(block.rows()); + else + column = castColumn(block.getByName(elem.name), elem.type, context); + + res.insert({column, elem.type, elem.name}); + } return res; } diff --git a/dbms/src/DataStreams/copyData.cpp b/dbms/src/DataStreams/copyData.cpp index 9d17596fc8d..5000c87be7c 100644 --- a/dbms/src/DataStreams/copyData.cpp +++ b/dbms/src/DataStreams/copyData.cpp @@ -28,6 +28,8 @@ void copyDataImpl(IBlockInputStream & from, IBlockOutputStream & to, TCancelCall break; to.write(block); + if (!block.rows()) + to.flush(); progress(block); } diff --git a/dbms/src/Databases/DatabasesCommon.cpp b/dbms/src/Databases/DatabasesCommon.cpp index 3b5654083e0..af43b25fbdb 100644 --- a/dbms/src/Databases/DatabasesCommon.cpp +++ b/dbms/src/Databases/DatabasesCommon.cpp @@ -41,7 +41,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query) create.replace_view = false; /// For views it is necessary to save the SELECT query itself, for the rest - on the contrary - if (!create.is_view && !create.is_materialized_view) + if (!create.is_view && !create.is_materialized_view && !create.is_live_view) create.select = nullptr; create.format = nullptr; diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index f587b2fd5ae..75f30e74761 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -252,6 +252,7 @@ void registerOutputFormatProcessorPrettySpace(FormatFactory & factory); void registerOutputFormatProcessorVertical(FormatFactory & factory); void registerOutputFormatProcessorJSON(FormatFactory & factory); void registerOutputFormatProcessorJSONCompact(FormatFactory & factory); +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory); void registerOutputFormatProcessorXML(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver(FormatFactory & factory); void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory); @@ -268,6 +269,8 @@ FormatFactory::FormatFactory() registerInputFormatTabSeparated(*this); registerInputFormatCSV(*this); + registerOutputFormatProcessorJSONEachRowWithProgress(*this); + registerInputFormatProcessorNative(*this); registerOutputFormatProcessorNative(*this); registerInputFormatProcessorRowBinary(*this); diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index 287e7a84170..ce15891f30a 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -159,6 +159,13 @@ public: */ virtual bool isSuitableForConstantFolding() const { return true; } + /** Some functions like ignore(...) or toTypeName(...) always return constant result which doesn't depend on arguments. + * In this case we can calculate result and assume that it's constant in stream header. + * There is no need to implement function if it has zero arguments. + * Must return ColumnConst with single row or nullptr. + */ + virtual ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & /*block*/, const ColumnNumbers & /*arguments*/) const { return nullptr; } + /** Function is called "injective" if it returns different result for different values of arguments. * Example: hex, negate, tuple... * @@ -456,6 +463,10 @@ public: } bool isSuitableForConstantFolding() const override { return function->isSuitableForConstantFolding(); } + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments_) const override + { + return function->getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments_); + } bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); } diff --git a/dbms/src/Functions/addressToLine.cpp b/dbms/src/Functions/addressToLine.cpp index 7f7bd609dee..498ab6e7a12 100644 --- a/dbms/src/Functions/addressToLine.cpp +++ b/dbms/src/Functions/addressToLine.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include #include @@ -149,3 +151,5 @@ void registerFunctionAddressToLine(FunctionFactory & factory) } } + +#endif diff --git a/dbms/src/Functions/addressToSymbol.cpp b/dbms/src/Functions/addressToSymbol.cpp index ceb641e457c..80b197c4429 100644 --- a/dbms/src/Functions/addressToSymbol.cpp +++ b/dbms/src/Functions/addressToSymbol.cpp @@ -1,3 +1,5 @@ +#ifdef __ELF__ + #include #include #include @@ -92,3 +94,5 @@ void registerFunctionAddressToSymbol(FunctionFactory & factory) } } + +#endif diff --git a/dbms/src/Functions/array/arrayWithConstant.cpp b/dbms/src/Functions/array/arrayWithConstant.cpp index efdcc7d6b72..b23d76b694d 100644 --- a/dbms/src/Functions/array/arrayWithConstant.cpp +++ b/dbms/src/Functions/array/arrayWithConstant.cpp @@ -72,7 +72,7 @@ public: offsets.push_back(offset); } - block.getByPosition(result).column = ColumnArray::create(col_value->replicate(offsets), std::move(offsets_col)); + block.getByPosition(result).column = ColumnArray::create(col_value->replicate(offsets)->convertToFullColumnIfConst(), std::move(offsets_col)); } }; diff --git a/dbms/src/Functions/defaultValueOfArgumentType.cpp b/dbms/src/Functions/defaultValueOfArgumentType.cpp index 452f25b51d8..db1a1533163 100644 --- a/dbms/src/Functions/defaultValueOfArgumentType.cpp +++ b/dbms/src/Functions/defaultValueOfArgumentType.cpp @@ -37,6 +37,12 @@ public: const IDataType & type = *block.getByPosition(arguments[0]).type; block.getByPosition(result).column = type.createColumnConst(input_rows_count, type.getDefault()); } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override + { + const IDataType & type = *block.getByPosition(arguments[0]).type; + return type.createColumnConst(1, type.getDefault()); + } }; diff --git a/dbms/src/Functions/getSizeOfEnumType.cpp b/dbms/src/Functions/getSizeOfEnumType.cpp index 11a22ecddfb..9b598ccca3c 100644 --- a/dbms/src/Functions/getSizeOfEnumType.cpp +++ b/dbms/src/Functions/getSizeOfEnumType.cpp @@ -49,11 +49,16 @@ public: } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = getResultIfAlwaysReturnsConstantAndHasArguments(block, arguments)->cloneResized(input_rows_count); + } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override { if (auto type8 = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, type8->getValues().size()); + return DataTypeUInt8().createColumnConst(1, type8->getValues().size()); else if (auto type16 = checkAndGetDataType(block.getByPosition(arguments[0]).type.get())) - block.getByPosition(result).column = DataTypeUInt16().createColumnConst(input_rows_count, type16->getValues().size()); + return DataTypeUInt16().createColumnConst(1, type16->getValues().size()); else throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } diff --git a/dbms/src/Functions/ignore.cpp b/dbms/src/Functions/ignore.cpp index 73aaea9f3ca..592de576659 100644 --- a/dbms/src/Functions/ignore.cpp +++ b/dbms/src/Functions/ignore.cpp @@ -42,6 +42,11 @@ public: { block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, 0u); } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block &, const ColumnNumbers &) const override + { + return DataTypeUInt8().createColumnConst(1, 0u); + } }; diff --git a/dbms/src/Functions/ignoreExceptNull.cpp b/dbms/src/Functions/ignoreExceptNull.cpp index 3ebdbea8c58..d977a87342f 100644 --- a/dbms/src/Functions/ignoreExceptNull.cpp +++ b/dbms/src/Functions/ignoreExceptNull.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -38,7 +39,11 @@ namespace DB void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override { - block.getByPosition(result).column = DataTypeUInt8().createColumnConst(input_rows_count, UInt64(0)); + /// This function is mainly used in query analysis instead of "in" functions + /// in the case when only header is needed and set for in is not calculated. + /// Because of that function must return the same column type as "in" function, which is ColumnUInt8. + auto res = ColumnUInt8::create(input_rows_count, 0); + block.getByPosition(result).column = std::move(res); } }; diff --git a/dbms/src/Functions/in.cpp b/dbms/src/Functions/in.cpp index 2911f743538..fa3a7652e9e 100644 --- a/dbms/src/Functions/in.cpp +++ b/dbms/src/Functions/in.cpp @@ -75,6 +75,8 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override { + /// NOTE: after updating this code, check that FunctionIgnoreExceptNull returns the same type of column. + /// Second argument must be ColumnSet. ColumnPtr column_set_ptr = block.getByPosition(arguments[1]).column; const ColumnSet * column_set = typeid_cast(&*column_set_ptr); diff --git a/dbms/src/Functions/nullIf.cpp b/dbms/src/Functions/nullIf.cpp index a68537c9c7e..91fb311f444 100644 --- a/dbms/src/Functions/nullIf.cpp +++ b/dbms/src/Functions/nullIf.cpp @@ -43,18 +43,17 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - /// nullIf(col1, col2) == if(col1 != col2, col1, NULL) + /// nullIf(col1, col2) == if(col1 = col2, NULL, 1) Block temp_block = block; - size_t res_pos = temp_block.columns(); - temp_block.insert({nullptr, std::make_shared(), ""}); + auto equals_func = FunctionFactory::instance().get("equals", context)->build( + {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); - { - auto equals_func = FunctionFactory::instance().get("notEquals", context)->build( - {temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])}); - equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count); - } + size_t equals_res_pos = temp_block.columns(); + temp_block.insert({nullptr, equals_func->getReturnType(), ""}); + + equals_func->execute(temp_block, {arguments[0], arguments[1]}, equals_res_pos, input_rows_count); /// Argument corresponding to the NULL value. size_t null_pos = temp_block.columns(); @@ -68,15 +67,14 @@ public: temp_block.insert(null_elem); auto func_if = FunctionFactory::instance().get("if", context)->build( - {temp_block.getByPosition(res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); - func_if->execute(temp_block, {res_pos, arguments[0], null_pos}, result, input_rows_count); + {temp_block.getByPosition(equals_res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])}); + func_if->execute(temp_block, {equals_res_pos, null_pos, arguments[0]}, result, input_rows_count); - block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column); + block.getByPosition(result).column = makeNullable(std::move(temp_block.getByPosition(result).column)); } }; - void registerFunctionNullIf(FunctionFactory & factory) { factory.registerFunction(FunctionFactory::CaseInsensitive); diff --git a/dbms/src/Functions/toColumnTypeName.cpp b/dbms/src/Functions/toColumnTypeName.cpp index 35d332d082f..023a9350547 100644 --- a/dbms/src/Functions/toColumnTypeName.cpp +++ b/dbms/src/Functions/toColumnTypeName.cpp @@ -38,6 +38,11 @@ public: block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, block.getByPosition(arguments[0]).column->getName()); } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block & block, const ColumnNumbers & arguments) const override + { + return DataTypeString().createColumnConst(1, block.getByPosition(arguments[0]).type->createColumn()->getName()); + } }; diff --git a/dbms/src/Functions/toTypeName.cpp b/dbms/src/Functions/toTypeName.cpp index a631c7f4044..84ab32c3643 100644 --- a/dbms/src/Functions/toTypeName.cpp +++ b/dbms/src/Functions/toTypeName.cpp @@ -9,33 +9,16 @@ namespace DB /** toTypeName(x) - get the type name * Returns name of IDataType instance (name of data type). */ -class FunctionToTypeName : public IFunction +class PreparedFunctionToTypeName : public PreparedFunctionImpl { public: static constexpr auto name = "toTypeName"; - static FunctionPtr create(const Context &) - { - return std::make_shared(); - } - - String getName() const override - { - return name; - } + String getName() const override { return name; } +protected: bool useDefaultImplementationForNulls() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - size_t getNumberOfArguments() const override - { - return 1; - } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - /// Execute the function on the block. void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { @@ -45,9 +28,64 @@ public: }; +class BaseFunctionToTypeName : public IFunctionBase +{ +public: + BaseFunctionToTypeName(DataTypes argument_types_, DataTypePtr return_type_) + : argument_types(std::move(argument_types_)), return_type(std::move(return_type_)) {} + + static constexpr auto name = "toTypeName"; + String getName() const override { return name; } + + const DataTypes & getArgumentTypes() const override { return argument_types; } + const DataTypePtr & getReturnType() const override { return return_type; } + + PreparedFunctionPtr prepare(const Block &, const ColumnNumbers &, size_t) const override + { + return std::make_shared(); + } + + ColumnPtr getResultIfAlwaysReturnsConstantAndHasArguments(const Block &, const ColumnNumbers &) const override + { + return DataTypeString().createColumnConst(1, argument_types.at(0)->getName()); + } + +private: + DataTypes argument_types; + DataTypePtr return_type; +}; + + +class FunctionToTypeNameBuilder : public FunctionBuilderImpl +{ +public: + static constexpr auto name = "toTypeName"; + String getName() const override { return name; } + static FunctionBuilderPtr create(const Context &) { return std::make_shared(); } + + size_t getNumberOfArguments() const override { return 1; } + +protected: + DataTypePtr getReturnTypeImpl(const DataTypes &) const override { return std::make_shared(); } + + FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override + { + DataTypes types; + types.reserve(arguments.size()); + for (auto & elem : arguments) + types.emplace_back(elem.type); + + return std::make_shared(types, return_type); + } + + bool useDefaultImplementationForNulls() const override { return false; } + bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } +}; + + void registerFunctionToTypeName(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/src/IO/ReadBufferFromFile.cpp b/dbms/src/IO/ReadBufferFromFile.cpp index b9cd7caf155..37225c0a36e 100644 --- a/dbms/src/IO/ReadBufferFromFile.cpp +++ b/dbms/src/IO/ReadBufferFromFile.cpp @@ -47,7 +47,7 @@ ReadBufferFromFile::ReadBufferFromFile( if (o_direct) { if (fcntl(fd, F_NOCACHE, 1) == -1) - throwFromErrno("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); + throwFromErrnoWithPath("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); } #endif } diff --git a/dbms/src/IO/WriteBufferFromFile.cpp b/dbms/src/IO/WriteBufferFromFile.cpp index 5e6fd7d6fe1..70aff23a420 100644 --- a/dbms/src/IO/WriteBufferFromFile.cpp +++ b/dbms/src/IO/WriteBufferFromFile.cpp @@ -51,7 +51,7 @@ WriteBufferFromFile::WriteBufferFromFile( if (o_direct) { if (fcntl(fd, F_NOCACHE, 1) == -1) - throwFromErrno("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); + throwFromErrnoWithPath("Cannot set F_NOCACHE on file " + file_name, file_name, ErrorCodes::CANNOT_OPEN_FILE); } #endif } diff --git a/dbms/src/IO/WriteBufferValidUTF8.cpp b/dbms/src/IO/WriteBufferValidUTF8.cpp index edff9e5bcf4..220b6cd44fc 100644 --- a/dbms/src/IO/WriteBufferValidUTF8.cpp +++ b/dbms/src/IO/WriteBufferValidUTF8.cpp @@ -117,6 +117,9 @@ void WriteBufferValidUTF8::nextImpl() memory[i] = p[i]; working_buffer = Buffer(&memory[cnt], memory.data() + memory.size()); + + /// Propagate next() to the output buffer + output_buffer.next(); } diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 9e49d302100..905827205b4 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -68,7 +68,13 @@ BlockInputStreamPtr createLocalStream(const ASTPtr & query_ast, const Context & * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. */ - return std::make_shared(stream); + + /* Now we don't need to materialize constants, because RemoteBlockInputStream will ignore constant and take it from header. + * So, streams from different threads will always have the same header. + */ + /// return std::make_shared(stream); + + return stream; } } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 0bb59713868..c7b510abcf0 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace ProfileEvents @@ -159,20 +160,24 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column } ExpressionAction ExpressionAction::ordinaryJoin( + const ASTTableJoin & join_params, std::shared_ptr join_, const Names & join_key_names_left, + const Names & join_key_names_right, const NamesAndTypesList & columns_added_by_join_) { ExpressionAction a; a.type = JOIN; a.join = std::move(join_); + a.join_kind = join_params.kind; a.join_key_names_left = join_key_names_left; + a.join_key_names_right = join_key_names_right; a.columns_added_by_join = columns_added_by_join_; return a; } -void ExpressionAction::prepare(Block & sample_block, const Settings & settings) +void ExpressionAction::prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding) { // std::cerr << "preparing: " << toString() << std::endl; @@ -187,6 +192,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); bool all_const = true; + bool all_suitable_for_constant_folding = true; ColumnNumbers arguments(argument_names.size()); for (size_t i = 0; i < argument_names.size(); ++i) @@ -195,6 +201,9 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) ColumnPtr col = sample_block.safeGetByPosition(arguments[i]).column; if (!col || !isColumnConst(*col)) all_const = false; + + if (names_not_for_constant_folding.count(argument_names[i])) + all_suitable_for_constant_folding = false; } size_t result_position = sample_block.columns(); @@ -229,6 +238,22 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) if (col.column->empty()) col.column = col.column->cloneResized(1); + + if (!all_suitable_for_constant_folding) + names_not_for_constant_folding.insert(result_name); + } + } + + /// Some functions like ignore() or getTypeName() always return constant result even if arguments are not constant. + /// We can't do constant folding, but can specify in sample block that function result is constant to avoid + /// unnecessary materialization. + auto & res = sample_block.getByPosition(result_position); + if (!res.column && function_base->isSuitableForConstantFolding()) + { + if (auto col = function_base->getResultIfAlwaysReturnsConstantAndHasArguments(sample_block, arguments)) + { + res.column = std::move(col); + names_not_for_constant_folding.insert(result_name); } } @@ -252,10 +277,50 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) case JOIN: { - /// TODO join_use_nulls setting + bool is_null_used_as_default = settings.join_use_nulls; + bool right_or_full_join = join_kind == ASTTableJoin::Kind::Right || join_kind == ASTTableJoin::Kind::Full; + bool left_or_full_join = join_kind == ASTTableJoin::Kind::Left || join_kind == ASTTableJoin::Kind::Full; + + for (auto & col : sample_block) + { + /// Materialize column. + /// Column is not empty if it is constant, but after Join all constants will be materialized. + /// So, we need remove constants from header. + if (col.column) + col.column = nullptr; + + bool make_nullable = is_null_used_as_default && right_or_full_join; + + if (make_nullable && !col.type->isNullable()) + col.type = std::make_shared(col.type); + } for (const auto & col : columns_added_by_join) - sample_block.insert(ColumnWithTypeAndName(nullptr, col.type, col.name)); + { + auto res_type = col.type; + + bool make_nullable = is_null_used_as_default && left_or_full_join; + + if (!make_nullable) + { + /// Keys from right table are usually not stored in Join, but copied from the left one. + /// So, if left key is nullable, let's make right key nullable too. + /// Note: for some join types it's not needed and, probably, may be removed. + /// Note: changing this code, take into account the implementation in Join.cpp. + auto it = std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name); + if (it != join_key_names_right.end()) + { + auto pos = it - join_key_names_right.begin(); + const auto & left_key_name = join_key_names_left[pos]; + make_nullable = sample_block.getByName(left_key_name).type->isNullable(); + } + } + + if (make_nullable && !res_type->isNullable()) + res_type = std::make_shared(res_type); + + sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name)); + } break; } @@ -683,7 +748,7 @@ void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) for (const auto & name_with_alias : action.projection) new_names.emplace_back(name_with_alias.second); - action.prepare(sample_block, settings); + action.prepare(sample_block, settings, names_not_for_constant_folding); actions.push_back(action); } @@ -915,7 +980,7 @@ void ExpressionActions::finalize(const Names & output_columns) if (action.type == ExpressionAction::APPLY_FUNCTION && sample_block.has(out)) { auto & result = sample_block.getByName(out); - if (result.column) + if (result.column && names_not_for_constant_folding.count(result.name) == 0) { action.type = ExpressionAction::ADD_COLUMN; action.result_type = result.type; @@ -1262,6 +1327,7 @@ bool ExpressionAction::operator==(const ExpressionAction & other) const && array_join_is_left == other.array_join_is_left && join == other.join && join_key_names_left == other.join_key_names_left + && join_key_names_right == other.join_key_names_right && columns_added_by_join == other.columns_added_by_join && projection == other.projection && is_function_compiled == other.is_function_compiled; diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index f280e723dd5..62d50131b45 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -10,6 +10,7 @@ #include "config_core.h" #include #include +#include namespace DB @@ -104,7 +105,9 @@ public: /// For JOIN std::shared_ptr join; + ASTTableJoin::Kind join_kind; Names join_key_names_left; + Names join_key_names_right; NamesAndTypesList columns_added_by_join; /// For PROJECT. @@ -121,7 +124,8 @@ public: static ExpressionAction project(const Names & projected_columns_); static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_); static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context); - static ExpressionAction ordinaryJoin(std::shared_ptr join_, const Names & join_key_names_left, + static ExpressionAction ordinaryJoin(const ASTTableJoin & join_params, std::shared_ptr join_, + const Names & join_key_names_left, const Names & join_key_names_right, const NamesAndTypesList & columns_added_by_join_); /// Which columns necessary to perform this action. @@ -139,7 +143,7 @@ public: private: friend class ExpressionActions; - void prepare(Block & sample_block, const Settings & settings); + void prepare(Block & sample_block, const Settings & settings, NameSet & names_not_for_constant_folding); void execute(Block & block, bool dry_run) const; void executeOnTotals(Block & block) const; }; @@ -263,6 +267,8 @@ private: Actions actions; /// The example of result (output) block. Block sample_block; + /// Columns which can't be used for constant folding. + NameSet names_not_for_constant_folding; Settings settings; #if USE_EMBEDDED_COMPILER diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 29464856316..9836c9eff37 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -140,7 +140,7 @@ void ExpressionAnalyzer::analyzeAggregation() for (const auto & key_ast : analyzedJoin().key_asts_left) getRootActions(key_ast, true, temp_actions); - addJoinAction(temp_actions); + addJoinAction(table_join, temp_actions); } } @@ -424,9 +424,9 @@ static void appendRequiredColumns( } /// It's possible to set nullptr as join for only_types mode -void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, JoinPtr join) const +void ExpressionAnalyzer::addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join) const { - actions->add(ExpressionAction::ordinaryJoin(join, analyzedJoin().key_names_left, columnsAddedByJoin())); + actions->add(ExpressionAction::ordinaryJoin(join_params, std::move(join), analyzedJoin().key_names_left, analyzedJoin().key_names_right, columnsAddedByJoin())); } bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) @@ -443,8 +443,10 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b initChain(chain, sourceColumns()); ExpressionActionsChain::Step & step = chain.steps.back(); + auto & join_params = ast_join->table_join->as(); + getRootActions(left_keys_list, only_types, step.actions); - addJoinAction(step.actions, subquery_for_set.join); + addJoinAction(join_params, step.actions, subquery_for_set.join); return true; } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 0b71b4ce54f..2eafe4b85f0 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -131,7 +131,7 @@ protected: void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; - void addJoinAction(ExpressionActionsPtr & actions, JoinPtr join = {}) const; + void addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join = {}) const; void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 31a69086f5c..cc8466bebc1 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -400,6 +400,10 @@ public: return count; } +#if !__clang__ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-variable" +#endif bool hasCurrentlyLoadedObjects() const { std::lock_guard lock{mutex}; @@ -408,6 +412,9 @@ public: return true; return false; } +#if !__clang__ +#pragma GCC diagnostic pop +#endif /// Starts loading of a specified object. void load(const String & name) diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 8751ff067b1..a3e6824c3a5 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -8,7 +8,9 @@ #include #include #include +#include #include +#include #include @@ -48,6 +50,7 @@ BlockIO InterpreterAlterQuery::execute() AlterCommands alter_commands; PartitionCommands partition_commands; MutationCommands mutation_commands; + LiveViewCommands live_view_commands; for (ASTAlterCommand * command_ast : alter.command_list->commands) { if (auto alter_command = AlterCommand::parse(command_ast)) @@ -56,13 +59,16 @@ BlockIO InterpreterAlterQuery::execute() partition_commands.emplace_back(std::move(*partition_command)); else if (auto mut_command = MutationCommand::parse(command_ast)) mutation_commands.emplace_back(std::move(*mut_command)); + else if (auto live_view_command = LiveViewCommand::parse(command_ast)) + live_view_commands.emplace_back(std::move(*live_view_command)); else throw Exception("Wrong parameter type in ALTER query", ErrorCodes::LOGICAL_ERROR); } if (!mutation_commands.empty()) { - MutationsInterpreter(table, mutation_commands, context).validate(); + auto table_lock_holder = table->lockStructureForShare(false /* because mutation is executed asyncronously */, context.getCurrentQueryId()); + MutationsInterpreter(table, mutation_commands, context).validate(table_lock_holder); table->mutate(mutation_commands, context); } @@ -72,6 +78,21 @@ BlockIO InterpreterAlterQuery::execute() table->alterPartition(query_ptr, partition_commands, context); } + if (!live_view_commands.empty()) + { + live_view_commands.validate(*table); + for (const LiveViewCommand & command : live_view_commands) + { + auto live_view = std::dynamic_pointer_cast(table); + switch (command.type) + { + case LiveViewCommand::REFRESH: + live_view->refresh(context); + break; + } + } + } + if (!alter_commands.empty()) { auto table_lock_holder = table->lockAlterIntention(context.getCurrentQueryId()); diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 0467e91c6d1..68390160b71 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -442,7 +442,7 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const return; } - if (create.temporary) + if (create.temporary && !create.is_live_view) { auto engine_ast = std::make_shared(); engine_ast->name = "Memory"; @@ -465,6 +465,11 @@ void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const "Cannot CREATE a table AS " + as_database_name + "." + as_table_name + ", it is a View", ErrorCodes::INCORRECT_QUERY); + if (as_create.is_live_view) + throw Exception( + "Cannot CREATE a table AS " + as_database_name + "." + as_table_name + ", it is a Live View", + ErrorCodes::INCORRECT_QUERY); + create.set(create.storage, as_create.storage->ptr()); } } @@ -482,7 +487,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) } /// Temporary tables are created out of databases. - if (create.temporary && !create.database.empty()) + if (create.temporary && !create.database.empty() && !create.is_live_view) throw Exception("Temporary tables cannot be inside a database. You should not specify a database for a temporary table.", ErrorCodes::BAD_DATABASE_FOR_TEMPORARY_TABLE); @@ -505,7 +510,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.to_database.empty()) create.to_database = current_database; - if (create.select && (create.is_view || create.is_materialized_view)) + if (create.select && (create.is_view || create.is_materialized_view || create.is_live_view)) { AddDefaultDatabaseVisitor visitor(current_database); visitor.visit(*create.select); @@ -565,7 +570,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) String data_path; DatabasePtr database; - if (!create.temporary) + if (!create.temporary || create.is_live_view) { database = context.getDatabase(database_name); data_path = database->getDataPath(); @@ -611,7 +616,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) false); } - if (create.temporary) + if (create.temporary && !create.is_live_view) context.getSessionContext().addExternalTable(table_name, res, query_ptr); else database->createTable(context, table_name, res, query_ptr); @@ -630,7 +635,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) /// If the query is a CREATE SELECT, insert the data into the table. if (create.select && !create.attach - && !create.is_view && (!create.is_materialized_view || create.is_populate)) + && !create.is_view && !create.is_live_view && (!create.is_materialized_view || create.is_populate)) { auto insert = std::make_shared(); diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 54611860227..eaba1d1d9dc 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -35,6 +36,7 @@ #include #include #include +#include #include @@ -173,6 +175,10 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & throwIfNoAccess(context); return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } else throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); } diff --git a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp index 7326bb62924..69d601ca462 100644 --- a/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp +++ b/dbms/src/Interpreters/InterpreterOptimizeQuery.cpp @@ -23,7 +23,6 @@ BlockIO InterpreterOptimizeQuery::execute() return executeDDLQueryOnCluster(query_ptr, context, {ast.database}); StoragePtr table = context.getTable(ast.database, ast.table); - auto table_lock = table->lockStructureForShare(true, context.getCurrentQueryId()); table->optimize(query_ptr, ast.partition, ast.final, ast.deduplicate, context); return {}; } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 354da206635..27be35ad57f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -82,6 +82,8 @@ #include #include #include +#include +#include namespace DB @@ -271,7 +273,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( String database_name; String table_name; - getDatabaseAndTableNames(database_name, table_name); + getDatabaseAndTableNames(query, database_name, table_name, context); if (auto view_source = context.getViewSource()) { @@ -345,17 +347,20 @@ InterpreterSelectQuery::InterpreterSelectQuery( source_header = storage->getSampleBlockForColumns(required_columns); /// Calculate structure of the result. + result_header = getSampleBlockImpl(); + for (auto & col : result_header) { - Pipeline pipeline; - executeImpl(pipeline, nullptr, true); - result_header = pipeline.firstStream()->getHeader(); + if (!col.column) + col.column = col.type->createColumn(); + else if (isColumnConst(*col.column) && !col.column->empty()) + col.column = col.column->cloneEmpty(); } } -void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name) +void InterpreterSelectQuery::getDatabaseAndTableNames(const ASTSelectQuery & query, String & database_name, String & table_name, const Context & context) { - if (auto db_and_table = getDatabaseAndTable(getSelectQuery(), 0)) + if (auto db_and_table = getDatabaseAndTable(query, 0)) { table_name = db_and_table->table; database_name = db_and_table->database; @@ -381,8 +386,8 @@ Block InterpreterSelectQuery::getSampleBlock() BlockIO InterpreterSelectQuery::execute() { Pipeline pipeline; - executeImpl(pipeline, input, options.only_analyze); - executeUnion(pipeline); + executeImpl(pipeline, input); + executeUnion(pipeline, getSampleBlock()); BlockIO res; res.in = pipeline.firstStream(); @@ -392,28 +397,104 @@ BlockIO InterpreterSelectQuery::execute() BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams() { Pipeline pipeline; - executeImpl(pipeline, input, options.only_analyze); + executeImpl(pipeline, input); + unifyStreams(pipeline, getSampleBlock()); return pipeline.streams; } QueryPipeline InterpreterSelectQuery::executeWithProcessors() { QueryPipeline query_pipeline; - executeImpl(query_pipeline, input, options.only_analyze); + executeImpl(query_pipeline, input); return query_pipeline; } + +Block InterpreterSelectQuery::getSampleBlockImpl() +{ + FilterInfoPtr filter_info; + + /// Need to create sets before analyzeExpressions(). Otherwise some sets for index won't be created. + query_analyzer->makeSetsForIndex(getSelectQuery().where()); + query_analyzer->makeSetsForIndex(getSelectQuery().prewhere()); + + auto analysis_result = analyzeExpressions( + getSelectQuery(), + *query_analyzer, + QueryProcessingStage::Enum::FetchColumns, + options.to_stage, + context, + storage, + true, + filter_info); + + if (options.to_stage == QueryProcessingStage::Enum::FetchColumns) + { + auto header = source_header; + + if (analysis_result.prewhere_info) + { + analysis_result.prewhere_info->prewhere_actions->execute(header); + header = materializeBlock(header); + if (analysis_result.prewhere_info->remove_prewhere_column) + header.erase(analysis_result.prewhere_info->prewhere_column_name); + } + return header; + } + + if (options.to_stage == QueryProcessingStage::Enum::WithMergeableState) + { + if (!analysis_result.need_aggregate) + return analysis_result.before_order_and_select->getSampleBlock(); + + auto header = analysis_result.before_aggregation->getSampleBlock(); + + Names key_names; + AggregateDescriptions aggregates; + query_analyzer->getAggregateInfo(key_names, aggregates); + + Block res; + + for (auto & key : key_names) + res.insert({nullptr, header.getByName(key).type, key}); + + for (auto & aggregate : aggregates) + { + size_t arguments_size = aggregate.argument_names.size(); + DataTypes argument_types(arguments_size); + for (size_t j = 0; j < arguments_size; ++j) + argument_types[j] = header.getByName(aggregate.argument_names[j]).type; + + DataTypePtr type = std::make_shared(aggregate.function, argument_types, aggregate.parameters); + + res.insert({nullptr, type, aggregate.column_name}); + } + + return res; + } + + return analysis_result.final_projection->getSampleBlock(); +} + InterpreterSelectQuery::AnalysisResult -InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run, const FilterInfoPtr & filter_info) +InterpreterSelectQuery::analyzeExpressions( + const ASTSelectQuery & query, + SelectQueryExpressionAnalyzer & query_analyzer, + QueryProcessingStage::Enum from_stage, + QueryProcessingStage::Enum to_stage, + const Context & context, + const StoragePtr & storage, + bool only_types, + const FilterInfoPtr & filter_info) { AnalysisResult res; /// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing. res.first_stage = from_stage < QueryProcessingStage::WithMergeableState - && options.to_stage >= QueryProcessingStage::WithMergeableState; + && to_stage >= QueryProcessingStage::WithMergeableState; /// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing. res.second_stage = from_stage <= QueryProcessingStage::WithMergeableState - && options.to_stage > QueryProcessingStage::WithMergeableState; + && to_stage > QueryProcessingStage::WithMergeableState; /** First we compose a chain of actions and remember the necessary steps from it. * Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and @@ -468,8 +549,6 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage { ExpressionActionsChain chain(context); - auto & query = getSelectQuery(); - Names additional_required_columns_after_prewhere; if (storage && query.sample_size()) @@ -486,14 +565,14 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage columns_for_final.begin(), columns_for_final.end()); } - if (storage && context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) + if (storage && filter_info) { has_filter = true; /// XXX: aggregated copy-paste from ExpressionAnalyzer::appendSmth() if (chain.steps.empty()) { - chain.steps.emplace_back(std::make_shared(source_columns, context)); + chain.steps.emplace_back(std::make_shared(NamesAndTypesList(), context)); } ExpressionActionsChain::Step & step = chain.steps.back(); @@ -506,7 +585,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage chain.addStep(); } - if (query_analyzer->appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere)) + if (query_analyzer.appendPrewhere(chain, !res.first_stage, additional_required_columns_after_prewhere)) { has_prewhere = true; @@ -516,11 +595,11 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage chain.addStep(); } - res.need_aggregate = query_analyzer->hasAggregation(); + res.need_aggregate = query_analyzer.hasAggregation(); - query_analyzer->appendArrayJoin(chain, dry_run || !res.first_stage); + query_analyzer.appendArrayJoin(chain, only_types || !res.first_stage); - if (query_analyzer->appendJoin(chain, dry_run || !res.first_stage)) + if (query_analyzer.appendJoin(chain, only_types || !res.first_stage)) { res.before_join = chain.getLastActions(); if (!res.hasJoin()) @@ -528,7 +607,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage chain.addStep(); } - if (query_analyzer->appendWhere(chain, dry_run || !res.first_stage)) + if (query_analyzer.appendWhere(chain, only_types || !res.first_stage)) { where_step_num = chain.steps.size() - 1; has_where = res.has_where = true; @@ -538,13 +617,13 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage if (res.need_aggregate) { - query_analyzer->appendGroupBy(chain, dry_run || !res.first_stage); - query_analyzer->appendAggregateFunctionsArguments(chain, dry_run || !res.first_stage); + query_analyzer.appendGroupBy(chain, only_types || !res.first_stage); + query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !res.first_stage); res.before_aggregation = chain.getLastActions(); finalizeChain(chain); - if (query_analyzer->appendHaving(chain, dry_run || !res.second_stage)) + if (query_analyzer.appendHaving(chain, only_types || !res.second_stage)) { res.has_having = true; res.before_having = chain.getLastActions(); @@ -553,20 +632,20 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage } /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. - query_analyzer->appendSelect(chain, dry_run || (res.need_aggregate ? !res.second_stage : !res.first_stage)); + query_analyzer.appendSelect(chain, only_types || (res.need_aggregate ? !res.second_stage : !res.first_stage)); res.selected_columns = chain.getLastStep().required_output; - res.has_order_by = query_analyzer->appendOrderBy(chain, dry_run || (res.need_aggregate ? !res.second_stage : !res.first_stage)); + res.has_order_by = query_analyzer.appendOrderBy(chain, only_types || (res.need_aggregate ? !res.second_stage : !res.first_stage)); res.before_order_and_select = chain.getLastActions(); chain.addStep(); - if (query_analyzer->appendLimitBy(chain, dry_run || !res.second_stage)) + if (query_analyzer.appendLimitBy(chain, only_types || !res.second_stage)) { res.has_limit_by = true; res.before_limit_by = chain.getLastActions(); chain.addStep(); } - query_analyzer->appendProjectResult(chain); + query_analyzer.appendProjectResult(chain); res.final_projection = chain.getLastActions(); finalizeChain(chain); @@ -580,7 +659,7 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage if (res.has_having) res.before_having->prependProjectInput(); - res.subqueries_for_sets = query_analyzer->getSubqueriesForSets(); + res.subqueries_for_sets = query_analyzer.getSubqueriesForSets(); /// Check that PREWHERE doesn't contain unusual actions. Unusual actions are that can change number of rows. if (res.prewhere_info) @@ -747,7 +826,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons template -void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run) +void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input) { /** Streams of data. When the query is executed in parallel, we have several data streams. * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then @@ -771,7 +850,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS /// Turn off, if the table filter is applied. if (storage && !context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) { - if (!dry_run) + if (!options.only_analyze) from_stage = storage->getQueryProcessingStage(context); query_analyzer->makeSetsForIndex(query.where()); @@ -811,14 +890,22 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); } - if (dry_run) + if (options.only_analyze) { if constexpr (pipeline_with_processors) pipeline.init({std::make_shared(source_header)}); else pipeline.streams.emplace_back(std::make_shared(source_header)); - expressions = analyzeExpressions(QueryProcessingStage::FetchColumns, true, filter_info); + expressions = analyzeExpressions( + getSelectQuery(), + *query_analyzer, + QueryProcessingStage::FetchColumns, + options.to_stage, + context, + storage, + true, + filter_info); if (storage && expressions.filter_info && expressions.prewhere_info) throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); @@ -850,7 +937,15 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS pipeline.streams.push_back(prepared_input); } - expressions = analyzeExpressions(from_stage, false, filter_info); + expressions = analyzeExpressions( + getSelectQuery(), + *query_analyzer, + from_stage, + options.to_stage, + context, + storage, + false, + filter_info); if (from_stage == QueryProcessingStage::WithMergeableState && options.to_stage == QueryProcessingStage::WithMergeableState) @@ -1097,7 +1192,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if constexpr (pipeline_with_processors) pipeline.resize(1); else - executeUnion(pipeline); + executeUnion(pipeline, {}); } /** If there was more than one stream, @@ -1696,7 +1791,7 @@ void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool ov if (!settings.distributed_aggregation_memory_efficient) { /// We union several sources into one, parallelizing the work. - executeUnion(pipeline); + executeUnion(pipeline, {}); /// Now merge the aggregated blocks pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params, final, settings.max_threads); @@ -1798,7 +1893,7 @@ void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const Expre void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { - executeUnion(pipeline); + executeUnion(pipeline, {}); const Settings & settings = context.getSettingsRef(); @@ -1827,7 +1922,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bo void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator) { - executeUnion(pipeline); + executeUnion(pipeline, {}); Names key_names; AggregateDescriptions aggregates; @@ -1972,7 +2067,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so }); /// If there are several streams, we merge them into one - executeUnion(pipeline); + executeUnion(pipeline, {}); /// Merge the sorted blocks. pipeline.firstStream() = std::make_shared( @@ -2032,7 +2127,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) /// If there are several streams, then we merge them into one if (pipeline.hasMoreThanOneStream()) { - unifyStreams(pipeline); + unifyStreams(pipeline, pipeline.firstStream()->getHeader()); /** MergingSortedBlockInputStream reads the sources sequentially. * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. @@ -2136,12 +2231,15 @@ void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool befo } -void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) +void InterpreterSelectQuery::executeUnion(Pipeline & pipeline, Block header) { /// If there are still several streams, then we combine them into one if (pipeline.hasMoreThanOneStream()) { - unifyStreams(pipeline); + if (!header) + header = pipeline.firstStream()->getHeader(); + + unifyStreams(pipeline, std::move(header)); pipeline.firstStream() = std::make_shared(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); pipeline.stream_with_non_joined_data = nullptr; @@ -2351,7 +2449,7 @@ void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { - executeUnion(pipeline); + executeUnion(pipeline, {}); pipeline.firstStream() = std::make_shared( pipeline.firstStream(), subqueries_for_sets, context); } @@ -2369,20 +2467,22 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip } -void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline) +void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header) { - if (pipeline.hasMoreThanOneStream()) + /// Unify streams in case they have different headers. + + /// TODO: remove previos addition of _dummy column. + if (header.columns() > 1 && header.has("_dummy")) + header.erase("_dummy"); + + for (size_t i = 0; i < pipeline.streams.size(); ++i) { - /// Unify streams in case they have different headers. - auto first_header = pipeline.streams.at(0)->getHeader(); - for (size_t i = 1; i < pipeline.streams.size(); ++i) - { - auto & stream = pipeline.streams[i]; - auto header = stream->getHeader(); - auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; - if (!blocksHaveEqualStructure(first_header, header)) - stream = std::make_shared(context, stream, first_header, mode); - } + auto & stream = pipeline.streams[i]; + auto stream_header = stream->getHeader(); + auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; + + if (!blocksHaveEqualStructure(header, stream_header)) + stream = std::make_shared(context, stream, header, mode); } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 103d7578464..3a441445c9b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -90,6 +90,7 @@ private: ASTSelectQuery & getSelectQuery() { return query_ptr->as(); } + Block getSampleBlockImpl(); struct Pipeline { @@ -135,7 +136,7 @@ private: }; template - void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run); + void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input); struct AnalysisResult { @@ -172,12 +173,19 @@ private: FilterInfoPtr filter_info; }; - AnalysisResult analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run, const FilterInfoPtr & filter_info); - + static AnalysisResult analyzeExpressions( + const ASTSelectQuery & query, + SelectQueryExpressionAnalyzer & query_analyzer, + QueryProcessingStage::Enum from_stage, + QueryProcessingStage::Enum to_stage, + const Context & context, + const StoragePtr & storage, + bool only_types, + const FilterInfoPtr & filter_info); /** From which table to read. With JOIN, the "left" table is returned. */ - void getDatabaseAndTableNames(String & database_name, String & table_name); + static void getDatabaseAndTableNames(const ASTSelectQuery & query, String & database_name, String & table_name, const Context & context); /// Different stages of query execution. @@ -198,7 +206,7 @@ private: void executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info); void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); - void executeUnion(Pipeline & pipeline); + void executeUnion(Pipeline & pipeline, Block header); /// If header is not empty, convert streams structure to it. void executeLimitBy(Pipeline & pipeline); void executeLimit(Pipeline & pipeline); void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression); @@ -222,8 +230,8 @@ private: void executeExtremes(QueryPipeline & pipeline); void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, std::unordered_map & subqueries_for_sets); - /// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header. - void unifyStreams(Pipeline & pipeline); + /// Add ConvertingBlockInputStream to specified header. + void unifyStreams(Pipeline & pipeline, Block header); enum class Modificator { @@ -246,7 +254,6 @@ private: const SelectQueryOptions options; ASTPtr query_ptr; Context context; - NamesAndTypesList source_columns; SyntaxAnalyzerResultPtr syntax_analyzer_result; std::unique_ptr query_analyzer; SelectQueryInfo query_info; diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.cpp b/dbms/src/Interpreters/InterpreterWatchQuery.cpp new file mode 100644 index 00000000000..3ba8e2eadaa --- /dev/null +++ b/dbms/src/Interpreters/InterpreterWatchQuery.cpp @@ -0,0 +1,108 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_STORAGE; + extern const int UNKNOWN_TABLE; + extern const int TOO_MANY_COLUMNS; +} + +BlockInputStreamPtr InterpreterWatchQuery::executeImpl() +{ + return std::make_shared(Block()); +} + +BlockIO InterpreterWatchQuery::execute() +{ + BlockIO res; + const ASTWatchQuery & query = typeid_cast(*query_ptr); + String database; + String table; + /// Get database + if (!query.database.empty()) + database = query.database; + else + database = context.getCurrentDatabase(); + + /// Get table + table = query.table; + + /// Get storage + storage = context.tryGetTable(database, table); + + if (!storage) + throw Exception("Table " + backQuoteIfNeed(database) + "." + + backQuoteIfNeed(table) + " doesn't exist.", + ErrorCodes::UNKNOWN_TABLE); + + /// List of columns to read to execute the query. + Names required_columns = storage->getColumns().getNamesOfPhysical(); + + /// Get context settings for this query + const Settings & settings = context.getSettingsRef(); + + /// Limitation on the number of columns to read. + if (settings.max_columns_to_read && required_columns.size() > settings.max_columns_to_read) + throw Exception("Limit for number of columns to read exceeded. " + "Requested: " + std::to_string(required_columns.size()) + + ", maximum: " + settings.max_columns_to_read.toString(), + ErrorCodes::TOO_MANY_COLUMNS); + + size_t max_block_size = settings.max_block_size; + size_t max_streams = 1; + + /// Define query info + SelectQueryInfo query_info; + query_info.query = query_ptr; + + /// From stage + QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; + QueryProcessingStage::Enum to_stage = QueryProcessingStage::Complete; + + /// Watch storage + streams = storage->watch(required_columns, query_info, context, from_stage, max_block_size, max_streams); + + /// Constraints on the result, the quota on the result, and also callback for progress. + if (IBlockInputStream * stream = dynamic_cast(streams[0].get())) + { + /// Constraints apply only to the final result. + if (to_stage == QueryProcessingStage::Complete) + { + IBlockInputStream::LocalLimits limits; + limits.mode = IBlockInputStream::LIMITS_CURRENT; + limits.size_limits.max_rows = settings.max_result_rows; + limits.size_limits.max_bytes = settings.max_result_bytes; + limits.size_limits.overflow_mode = settings.result_overflow_mode; + + stream->setLimits(limits); + stream->setQuota(context.getQuota()); + } + } + + res.in = streams[0]; + + return res; +} + + +} diff --git a/dbms/src/Interpreters/InterpreterWatchQuery.h b/dbms/src/Interpreters/InterpreterWatchQuery.h new file mode 100644 index 00000000000..9315ee1f889 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterWatchQuery.h @@ -0,0 +1,50 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class IAST; +using ASTPtr = std::shared_ptr; +using StoragePtr = std::shared_ptr; + +class InterpreterWatchQuery : public IInterpreter +{ +public: + InterpreterWatchQuery(const ASTPtr & query_ptr_, Context & context_) + : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; + Context & context; + + BlockInputStreamPtr executeImpl(); + /// Table from where to read data, if not subquery. + StoragePtr storage; + /// Streams of read data + BlockInputStreams streams; +}; + + +} diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index fcff80aad62..4756a5680ef 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -126,6 +126,8 @@ public: bool empty() { return type == Type::EMPTY; } + bool isNullUsedAsDefault() const { return use_nulls; } + /** Set information about structure of right hand of JOIN (joined data). * You must call this method before subsequent calls to insertFromBlock. */ @@ -168,6 +170,7 @@ public: size_t getTotalByteCount() const; ASTTableJoin::Kind getKind() const { return kind; } + ASTTableJoin::Strictness getStrictness() const { return strictness; } AsofRowRefs::Type getAsofType() const { return *asof_type; } bool anyTakeLastRow() const { return any_take_last_row; } diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 160d0bc8023..27fb48c7b5b 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -458,15 +458,16 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::ve return in; } -void MutationsInterpreter::validate() +void MutationsInterpreter::validate(TableStructureReadLockHolder &) { prepare(/* dry_run = */ true); - Block first_stage_header = interpreter_select->getSampleBlock(); + /// Do not use getSampleBlock in order to check the whole pipeline. + Block first_stage_header = interpreter_select->execute().in->getHeader(); BlockInputStreamPtr in = std::make_shared(first_stage_header); addStreamsForLaterStages(stages, in)->getHeader(); } -BlockInputStreamPtr MutationsInterpreter::execute() +BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder &) { prepare(/* dry_run = */ false); BlockInputStreamPtr in = interpreter_select->execute().in; diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index 3fa8961e8f7..4b9a295b3d1 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -25,13 +25,13 @@ public: { } - void validate(); + void validate(TableStructureReadLockHolder & table_lock_holder); /// Return false if the data isn't going to be changed by mutations. bool isStorageTouchedByMutations() const; /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. - BlockInputStreamPtr execute(); + BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder); /// Only changed columns. const Block & getUpdatedHeader() const; @@ -44,7 +44,6 @@ private: std::unique_ptr prepareInterpreterSelect(std::vector & prepared_stages, bool dry_run); BlockInputStreamPtr addStreamsForLaterStages(const std::vector & prepared_stages, BlockInputStreamPtr in) const; -private: StoragePtr storage; std::vector commands; const Context & context; diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index 844c53c79ac..ffa94f3d700 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -102,8 +102,24 @@ void QueryNormalizer::visit(ASTIdentifier & node, ASTPtr & ast, Data & data) /// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column"). auto it_alias = data.aliases.find(node.name); - if (IdentifierSemantic::canBeAlias(node) && it_alias != data.aliases.end() && current_alias != node.name) + if (it_alias != data.aliases.end() && current_alias != node.name) { + if (!IdentifierSemantic::canBeAlias(node)) + { + /// This means that column had qualified name, which was translated (so, canBeAlias() returns false). + /// But there is an alias with the same name. So, let's use original name for that column. + /// If alias wasn't set, use original column name as alias. + /// That helps to avoid result set with columns which have same names but different values. + if (node.alias.empty()) + { + node.name.swap(node.alias); + node.restoreCompoundName(); + node.name.swap(node.alias); + } + + return; + } + auto & alias_node = it_alias->second; /// Let's replace it with the corresponding tree node. diff --git a/dbms/src/NOTICE b/dbms/src/NOTICE new file mode 100644 index 00000000000..d0d3efe3f8e --- /dev/null +++ b/dbms/src/NOTICE @@ -0,0 +1,41 @@ +-- +The following notice shall be applied to the files listed below. + +Some modifications Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +Common/ErrorCodes.cpp +Common/UInt128.h +Core/Block.h +Core/Defines.h +Core/Settings.h +DataStreams/PushingToViewsBlockOutputStream.cpp +DataStreams/PushingToViewsBlockOutputStream.h +DataStreams/copyData.cpp +Databases/DatabasesCommon.cpp +IO/WriteBufferValidUTF8.cpp +Interpreters/InterpreterAlterQuery.cpp +Interpreters/InterpreterCreateQuery.cpp +Interpreters/InterpreterFactory.cpp +Parsers/ASTAlterQuery.cpp +Parsers/ASTAlterQuery.h +Parsers/ASTCreateQuery.cpp +Parsers/ASTCreateQuery.h +Parsers/ParserAlterQuery.cpp +Parsers/ParserAlterQuery.h +Parsers/ParserCreateQuery.cpp +Parsers/ParserCreateQuery.h +Parsers/ParserQueryWithOutput.cpp +Storages/IStorage.h +Storages/StorageFactory.cpp +Storages/registerStorages.cpp +-- diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index a2cda316046..67b72eb75bb 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -45,6 +45,11 @@ ASTPtr ASTAlterCommand::clone() const res->ttl = ttl->clone(); res->children.push_back(res->ttl); } + if (values) + { + res->values = values->clone(); + res->children.push_back(res->values); + } return res; } @@ -200,6 +205,46 @@ void ASTAlterCommand::formatImpl( settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY TTL " << (settings.hilite ? hilite_none : ""); ttl->formatImpl(settings, state, frame); } + else if (type == ASTAlterCommand::LIVE_VIEW_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_ADD) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ADD " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_DROP) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "DROP " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_MODIFY) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "MODIFY " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_SUSPEND) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "SUSPEND " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_RESUME) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "RESUME " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } + else if (type == ASTAlterCommand::LIVE_CHANNEL_REFRESH) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "REFRESH " << (settings.hilite ? hilite_none : ""); + + values->formatImpl(settings, state, frame); + } else throw Exception("Unexpected type of ALTER", ErrorCodes::UNEXPECTED_AST_STRUCTURE); } @@ -252,7 +297,12 @@ void ASTAlterQuery::formatQueryImpl(const FormatSettings & settings, FormatState std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); - settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); + if (is_live_view) + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE VIEW " << (settings.hilite ? hilite_none : ""); + else if (is_live_channel) + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER LIVE CHANNEL " << (settings.hilite ? hilite_none : ""); + else + settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str << "ALTER TABLE " << (settings.hilite ? hilite_none : ""); if (!table.empty()) { diff --git a/dbms/src/Parsers/ASTAlterQuery.h b/dbms/src/Parsers/ASTAlterQuery.h index f660dc7b926..2edabcdc70d 100644 --- a/dbms/src/Parsers/ASTAlterQuery.h +++ b/dbms/src/Parsers/ASTAlterQuery.h @@ -15,6 +15,15 @@ namespace DB * MODIFY COLUMN col_name type, * DROP PARTITION partition, * COMMENT_COLUMN col_name 'comment', + * ALTER LIVE VIEW [db.]name_type + * REFRESH + * ALTER CHANNEL [db.]name_type + * ADD live_view,... + * DROP live_view,... + * SUSPEND live_view,... + * RESUME live_view,... + * REFRESH live_view,... + * MODIFY live_view,... */ class ASTAlterCommand : public IAST @@ -44,6 +53,15 @@ public: UPDATE, NO_TYPE, + + LIVE_VIEW_REFRESH, + + LIVE_CHANNEL_ADD, + LIVE_CHANNEL_DROP, + LIVE_CHANNEL_SUSPEND, + LIVE_CHANNEL_RESUME, + LIVE_CHANNEL_REFRESH, + LIVE_CHANNEL_MODIFY }; Type type = NO_TYPE; @@ -91,6 +109,10 @@ public: /// For MODIFY TTL query ASTPtr ttl; + /** In ALTER CHANNEL, ADD, DROP, SUSPEND, RESUME, REFRESH, MODIFY queries, the list of live views is stored here + */ + ASTPtr values; + bool detach = false; /// true for DETACH PARTITION bool part = false; /// true for ATTACH PART @@ -147,6 +169,9 @@ protected: class ASTAlterQuery : public ASTQueryWithTableAndOutput, public ASTQueryWithOnCluster { public: + bool is_live_view{false}; /// true for ALTER LIVE VIEW + bool is_live_channel{false}; /// true for ALTER LIVE CHANNEL + ASTAlterCommandList * command_list = nullptr; String getID(char) const override; diff --git a/dbms/src/Parsers/ASTCreateQuery.cpp b/dbms/src/Parsers/ASTCreateQuery.cpp index d8656d171e5..80309d5ef91 100644 --- a/dbms/src/Parsers/ASTCreateQuery.cpp +++ b/dbms/src/Parsers/ASTCreateQuery.cpp @@ -173,6 +173,8 @@ ASTPtr ASTCreateQuery::clone() const res->set(res->storage, storage->clone()); if (select) res->set(res->select, select->clone()); + if (tables) + res->set(res->tables, tables->clone()); cloneOutputOptions(*res); @@ -204,6 +206,11 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat what = "VIEW"; if (is_materialized_view) what = "MATERIALIZED VIEW"; + if (is_live_view) + what = "LIVE VIEW"; + if (is_live_channel) + what = "LIVE CHANNEL"; + settings.ostr << (settings.hilite ? hilite_keyword : "") @@ -257,6 +264,12 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat settings.ostr << (settings.hilite ? hilite_keyword : "") << " AS" << settings.nl_or_ws << (settings.hilite ? hilite_none : ""); select->formatImpl(settings, state, frame); } + + if (tables) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH " << (settings.hilite ? hilite_none : ""); + tables->formatImpl(settings, state, frame); + } } } diff --git a/dbms/src/Parsers/ASTCreateQuery.h b/dbms/src/Parsers/ASTCreateQuery.h index 7e121e4d2d9..8078e61d3e4 100644 --- a/dbms/src/Parsers/ASTCreateQuery.h +++ b/dbms/src/Parsers/ASTCreateQuery.h @@ -55,9 +55,12 @@ public: bool if_not_exists{false}; bool is_view{false}; bool is_materialized_view{false}; + bool is_live_view{false}; + bool is_live_channel{false}; bool is_populate{false}; bool replace_view{false}; /// CREATE OR REPLACE VIEW ASTColumns * columns_list = nullptr; + ASTExpressionList *tables = nullptr; String to_database; /// For CREATE MATERIALIZED VIEW mv TO table. String to_table; ASTStorage * storage = nullptr; diff --git a/dbms/src/Parsers/ASTWatchQuery.h b/dbms/src/Parsers/ASTWatchQuery.h new file mode 100644 index 00000000000..06d1460f038 --- /dev/null +++ b/dbms/src/Parsers/ASTWatchQuery.h @@ -0,0 +1,59 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + + +namespace DB +{ + +class ASTWatchQuery : public ASTQueryWithTableAndOutput +{ + +public: + ASTPtr limit_length; + bool is_watch_events; + + ASTWatchQuery() = default; + String getID(char) const override { return "WatchQuery_" + database + "_" + table; } + + ASTPtr clone() const override + { + std::shared_ptr res = std::make_shared(*this); + res->children.clear(); + cloneOutputOptions(*res); + return res; + } + +protected: + void formatQueryImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override + { + std::string indent_str = s.one_line ? "" : std::string(4 * frame.indent, ' '); + + s.ostr << (s.hilite ? hilite_keyword : "") << "WATCH" << " " << (s.hilite ? hilite_none : "") + << (!database.empty() ? backQuoteIfNeed(database) + "." : "") << backQuoteIfNeed(table); + + if (is_watch_events) + { + s.ostr << " " << (s.hilite ? hilite_keyword : "") << "EVENTS" << (s.hilite ? hilite_none : ""); + } + + if (limit_length) + { + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "LIMIT " << (s.hilite ? hilite_none : ""); + limit_length->formatImpl(s, state, frame); + } + } +}; + +} diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index 3ff2126bff0..0045769ca4d 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -34,6 +34,13 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserKeyword s_clear_index("CLEAR INDEX"); ParserKeyword s_materialize_index("MATERIALIZE INDEX"); + ParserKeyword s_add("ADD"); + ParserKeyword s_drop("DROP"); + ParserKeyword s_suspend("SUSPEND"); + ParserKeyword s_resume("RESUME"); + ParserKeyword s_refresh("REFRESH"); + ParserKeyword s_modify("MODIFY"); + ParserKeyword s_attach_partition("ATTACH PARTITION"); ParserKeyword s_detach_partition("DETACH PARTITION"); ParserKeyword s_drop_partition("DROP PARTITION"); @@ -65,268 +72,328 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected ParserList parser_assignment_list( std::make_unique(), std::make_unique(TokenType::Comma), /* allow_empty = */ false); + ParserNameList values_p; - if (s_add_column.ignore(pos, expected)) + if (is_live_view) { - if (s_if_not_exists.ignore(pos, expected)) - command->if_not_exists = true; - - if (!parser_col_decl.parse(pos, command->col_decl, expected)) - return false; - - if (s_after.ignore(pos, expected)) + if (s_refresh.ignore(pos, expected)) { + command->type = ASTAlterCommand::LIVE_VIEW_REFRESH; + } + else + return false; + } + else if (is_live_channel) + { + if (s_add.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_ADD; + } + else if (s_drop.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_DROP; + } + else if (s_suspend.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_SUSPEND; + } + else if (s_resume.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_RESUME; + } + else if (s_refresh.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_REFRESH; + } + else if (s_modify.ignore(pos, expected)) + { + if (!values_p.parse(pos, command->values, expected)) + return false; + + command->type = ASTAlterCommand::LIVE_CHANNEL_MODIFY; + } + else + return false; + } + else + { + if (s_add_column.ignore(pos, expected)) + { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; + + if (!parser_col_decl.parse(pos, command->col_decl, expected)) + return false; + + if (s_after.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->column, expected)) + return false; + } + + command->type = ASTAlterCommand::ADD_COLUMN; + } + else if (s_drop_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_PARTITION; + } + else if (s_drop_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + if (!parser_name.parse(pos, command->column, expected)) return false; + + command->type = ASTAlterCommand::DROP_COLUMN; + command->detach = false; } - - command->type = ASTAlterCommand::ADD_COLUMN; - } - else if (s_drop_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - command->type = ASTAlterCommand::DROP_PARTITION; - } - else if (s_drop_column.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->column, expected)) - return false; - - command->type = ASTAlterCommand::DROP_COLUMN; - command->detach = false; - } - else if (s_clear_column.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->column, expected)) - return false; - - command->type = ASTAlterCommand::DROP_COLUMN; - command->clear_column = true; - command->detach = false; - - if (s_in_partition.ignore(pos, expected)) + else if (s_clear_column.ignore(pos, expected)) { - if (!parser_partition.parse(pos, command->partition, expected)) + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->column, expected)) return false; + + command->type = ASTAlterCommand::DROP_COLUMN; + command->clear_column = true; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } } - } - else if (s_add_index.ignore(pos, expected)) - { - if (s_if_not_exists.ignore(pos, expected)) - command->if_not_exists = true; - - if (!parser_idx_decl.parse(pos, command->index_decl, expected)) - return false; - - if (s_after.ignore(pos, expected)) + else if (s_add_index.ignore(pos, expected)) { + if (s_if_not_exists.ignore(pos, expected)) + command->if_not_exists = true; + + if (!parser_idx_decl.parse(pos, command->index_decl, expected)) + return false; + + if (s_after.ignore(pos, expected)) + { + if (!parser_name.parse(pos, command->index, expected)) + return false; + } + + command->type = ASTAlterCommand::ADD_INDEX; + } + else if (s_drop_index.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + if (!parser_name.parse(pos, command->index, expected)) return false; + + command->type = ASTAlterCommand::DROP_INDEX; + command->detach = false; } - - command->type = ASTAlterCommand::ADD_INDEX; - } - else if (s_drop_index.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->index, expected)) - return false; - - command->type = ASTAlterCommand::DROP_INDEX; - command->detach = false; - } - else if (s_clear_index.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->index, expected)) - return false; - - command->type = ASTAlterCommand::DROP_INDEX; - command->clear_index = true; - command->detach = false; - - if (!s_in_partition.ignore(pos, expected)) - return false; - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - } - else if (s_materialize_index.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->index, expected)) - return false; - - command->type = ASTAlterCommand::MATERIALIZE_INDEX; - command->detach = false; - - if (s_in_partition.ignore(pos, expected)) + else if (s_clear_index.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->index, expected)) + return false; + + command->type = ASTAlterCommand::DROP_INDEX; + command->clear_index = true; + command->detach = false; + + if (!s_in_partition.ignore(pos, expected)) + return false; if (!parser_partition.parse(pos, command->partition, expected)) return false; } - } - else if (s_detach_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - command->type = ASTAlterCommand::DROP_PARTITION; - command->detach = true; - } - else if (s_attach_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - if (s_from.ignore(pos)) + else if (s_materialize_index.ignore(pos, expected)) { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->index, expected)) + return false; + + command->type = ASTAlterCommand::MATERIALIZE_INDEX; + command->detach = false; + + if (s_in_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + } + } + else if (s_detach_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::DROP_PARTITION; + command->detach = true; + } + else if (s_attach_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + if (s_from.ignore(pos)) + { + if (!parseDatabaseAndTableName(pos, expected, command->from_database, command->from_table)) + return false; + + command->replace = false; + command->type = ASTAlterCommand::REPLACE_PARTITION; + } + else + { + command->type = ASTAlterCommand::ATTACH_PARTITION; + } + } + else if (s_replace_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + if (!s_from.ignore(pos, expected)) + return false; + if (!parseDatabaseAndTableName(pos, expected, command->from_database, command->from_table)) return false; - command->replace = false; + command->replace = true; command->type = ASTAlterCommand::REPLACE_PARTITION; } - else + else if (s_attach_part.ignore(pos, expected)) { + if (!parser_string_literal.parse(pos, command->partition, expected)) + return false; + + command->part = true; command->type = ASTAlterCommand::ATTACH_PARTITION; } - } - else if (s_replace_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - if (!s_from.ignore(pos, expected)) - return false; - - if (!parseDatabaseAndTableName(pos, expected, command->from_database, command->from_table)) - return false; - - command->replace = true; - command->type = ASTAlterCommand::REPLACE_PARTITION; - } - else if (s_attach_part.ignore(pos, expected)) - { - if (!parser_string_literal.parse(pos, command->partition, expected)) - return false; - - command->part = true; - command->type = ASTAlterCommand::ATTACH_PARTITION; - } - else if (s_fetch_partition.ignore(pos, expected)) - { - if (!parser_partition.parse(pos, command->partition, expected)) - return false; - - if (!s_from.ignore(pos, expected)) - return false; - - ASTPtr ast_from; - if (!parser_string_literal.parse(pos, ast_from, expected)) - return false; - - command->from = ast_from->as().value.get(); - command->type = ASTAlterCommand::FETCH_PARTITION; - } - else if (s_freeze.ignore(pos, expected)) - { - if (s_partition.ignore(pos, expected)) + else if (s_fetch_partition.ignore(pos, expected)) { if (!parser_partition.parse(pos, command->partition, expected)) return false; - command->type = ASTAlterCommand::FREEZE_PARTITION; + if (!s_from.ignore(pos, expected)) + return false; + + ASTPtr ast_from; + if (!parser_string_literal.parse(pos, ast_from, expected)) + return false; + + command->from = ast_from->as().value.get(); + command->type = ASTAlterCommand::FETCH_PARTITION; + } + else if (s_freeze.ignore(pos, expected)) + { + if (s_partition.ignore(pos, expected)) + { + if (!parser_partition.parse(pos, command->partition, expected)) + return false; + + command->type = ASTAlterCommand::FREEZE_PARTITION; + } + else + { + command->type = ASTAlterCommand::FREEZE_ALL; + } + + /// WITH NAME 'name' - place local backup to directory with specified name + if (s_with.ignore(pos, expected)) + { + if (!s_name.ignore(pos, expected)) + return false; + + ASTPtr ast_with_name; + if (!parser_string_literal.parse(pos, ast_with_name, expected)) + return false; + + command->with_name = ast_with_name->as().value.get(); + } + } + else if (s_modify_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) + return false; + + command->type = ASTAlterCommand::MODIFY_COLUMN; + } + else if (s_modify_order_by.ignore(pos, expected)) + { + if (!parser_exp_elem.parse(pos, command->order_by, expected)) + return false; + + command->type = ASTAlterCommand::MODIFY_ORDER_BY; + } + else if (s_delete_where.ignore(pos, expected)) + { + if (!parser_exp_elem.parse(pos, command->predicate, expected)) + return false; + + command->type = ASTAlterCommand::DELETE; + } + else if (s_update.ignore(pos, expected)) + { + if (!parser_assignment_list.parse(pos, command->update_assignments, expected)) + return false; + + if (!s_where.ignore(pos, expected)) + return false; + + if (!parser_exp_elem.parse(pos, command->predicate, expected)) + return false; + + command->type = ASTAlterCommand::UPDATE; + } + else if (s_comment_column.ignore(pos, expected)) + { + if (s_if_exists.ignore(pos, expected)) + command->if_exists = true; + + if (!parser_name.parse(pos, command->column, expected)) + return false; + + if (!parser_string_literal.parse(pos, command->comment, expected)) + return false; + + command->type = ASTAlterCommand::COMMENT_COLUMN; + } + else if (s_modify_ttl.ignore(pos, expected)) + { + if (!parser_exp_elem.parse(pos, command->ttl, expected)) + return false; + command->type = ASTAlterCommand::MODIFY_TTL; } else - { - command->type = ASTAlterCommand::FREEZE_ALL; - } - - /// WITH NAME 'name' - place local backup to directory with specified name - if (s_with.ignore(pos, expected)) - { - if (!s_name.ignore(pos, expected)) - return false; - - ASTPtr ast_with_name; - if (!parser_string_literal.parse(pos, ast_with_name, expected)) - return false; - - command->with_name = ast_with_name->as().value.get(); - } + return false; } - else if (s_modify_column.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_modify_col_decl.parse(pos, command->col_decl, expected)) - return false; - - command->type = ASTAlterCommand::MODIFY_COLUMN; - } - else if (s_modify_order_by.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->order_by, expected)) - return false; - - command->type = ASTAlterCommand::MODIFY_ORDER_BY; - } - else if (s_delete_where.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->predicate, expected)) - return false; - - command->type = ASTAlterCommand::DELETE; - } - else if (s_update.ignore(pos, expected)) - { - if (!parser_assignment_list.parse(pos, command->update_assignments, expected)) - return false; - - if (!s_where.ignore(pos, expected)) - return false; - - if (!parser_exp_elem.parse(pos, command->predicate, expected)) - return false; - - command->type = ASTAlterCommand::UPDATE; - } - else if (s_comment_column.ignore(pos, expected)) - { - if (s_if_exists.ignore(pos, expected)) - command->if_exists = true; - - if (!parser_name.parse(pos, command->column, expected)) - return false; - - if (!parser_string_literal.parse(pos, command->comment, expected)) - return false; - - command->type = ASTAlterCommand::COMMENT_COLUMN; - } - else if (s_modify_ttl.ignore(pos, expected)) - { - if (!parser_exp_elem.parse(pos, command->ttl, expected)) - return false; - command->type = ASTAlterCommand::MODIFY_TTL; - } - else - return false; if (command->col_decl) command->children.push_back(command->col_decl); @@ -340,6 +407,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected command->children.push_back(command->predicate); if (command->update_assignments) command->children.push_back(command->update_assignments); + if (command->values) + command->children.push_back(command->values); if (command->comment) command->children.push_back(command->comment); if (command->ttl) @@ -355,7 +424,7 @@ bool ParserAlterCommandList::parseImpl(Pos & pos, ASTPtr & node, Expected & expe node = command_list; ParserToken s_comma(TokenType::Comma); - ParserAlterCommand p_command; + ParserAlterCommand p_command(is_live_view, is_live_channel); do { @@ -404,8 +473,30 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) node = query; ParserKeyword s_alter_table("ALTER TABLE"); + ParserKeyword s_alter_live_view("ALTER LIVE VIEW"); + ParserKeyword s_alter_live_channel("ALTER LIVE CHANNEL"); + + bool is_live_view = false; + bool is_live_channel = false; + if (!s_alter_table.ignore(pos, expected)) - return false; + { + if (!s_alter_live_view.ignore(pos, expected)) + { + if (!s_alter_live_channel.ignore(pos, expected)) + return false; + else + is_live_channel = true; + } + else + is_live_view = true; + } + + if (is_live_view) + query->is_live_view = true; + + if (is_live_channel) + query->is_live_channel = true; if (!parseDatabaseAndTableName(pos, expected, query->database, query->table)) return false; @@ -418,7 +509,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } query->cluster = cluster_str; - ParserAlterCommandList p_command_list; + ParserAlterCommandList p_command_list(is_live_view, is_live_channel); ASTPtr command_list; if (!p_command_list.parse(pos, command_list, expected)) return false; diff --git a/dbms/src/Parsers/ParserAlterQuery.h b/dbms/src/Parsers/ParserAlterQuery.h index 282a4277e17..13f5681a9da 100644 --- a/dbms/src/Parsers/ParserAlterQuery.h +++ b/dbms/src/Parsers/ParserAlterQuery.h @@ -19,6 +19,15 @@ namespace DB * [FREEZE [PARTITION] [WITH NAME name]] * [DELETE WHERE ...] * [UPDATE col_name = expr, ... WHERE ...] + * ALTER LIVE VIEW [db.name] + * [REFRESH] + * ALTER LIVE CHANNEL [db.name] [ON CLUSTER cluster] + * [ADD live_view, ...] + * [DROP live_view, ...] + * [SUSPEND live_view, ...] + * [RESUME live_view, ...] + * [REFRESH live_view, ...] + * [MODIFY live_view, ...] */ class ParserAlterQuery : public IParserBase @@ -34,6 +43,12 @@ class ParserAlterCommandList : public IParserBase protected: const char * getName() const { return "a list of ALTER commands"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + +public: + bool is_live_view; + bool is_live_channel; + + ParserAlterCommandList(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {} }; @@ -42,6 +57,12 @@ class ParserAlterCommand : public IParserBase protected: const char * getName() const { return "ALTER command"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); + +public: + bool is_live_view; + bool is_live_channel; + + ParserAlterCommand(bool is_live_view_ = false, bool is_live_channel_ = false) : is_live_view(is_live_view_), is_live_channel(is_live_channel_) {} }; diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 2e7c03a66b9..1a00e18e600 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -94,6 +94,12 @@ bool ParserColumnDeclarationList::parseImpl(Pos & pos, ASTPtr & node, Expected & .parse(pos, node, expected); } +bool ParserNameList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + return ParserList(std::make_unique(), std::make_unique(TokenType::Comma), false) + .parse(pos, node, expected); +} + bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_type("TYPE"); @@ -309,7 +315,10 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_if_not_exists("IF NOT EXISTS"); ParserKeyword s_as("AS"); ParserKeyword s_view("VIEW"); + ParserKeyword s_with("WITH"); ParserKeyword s_materialized("MATERIALIZED"); + ParserKeyword s_live("LIVE"); + ParserKeyword s_channel("CHANNEL"); ParserKeyword s_populate("POPULATE"); ParserKeyword s_or_replace("OR REPLACE"); ParserToken s_dot(TokenType::Dot); @@ -320,6 +329,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserColumnsOrIndicesDeclarationList columns_or_indices_p; ParserSelectWithUnionQuery select_p; ParserFunction table_function_p; + ParserNameList names_p; ASTPtr database; ASTPtr table; @@ -331,11 +341,15 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ASTPtr as_table; ASTPtr as_table_function; ASTPtr select; + ASTPtr tables; + String cluster_str; bool attach = false; bool if_not_exists = false; bool is_view = false; bool is_materialized_view = false; + bool is_live_view = false; + bool is_live_channel = false; bool is_populate = false; bool is_temporary = false; bool replace_view = false; @@ -431,6 +445,79 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } } + else if (s_live.ignore(pos, expected)) + { + if (s_channel.ignore(pos, expected)) + is_live_channel = true; + else if (s_view.ignore(pos, expected)) + is_live_view = true; + else + return false; + + if (s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + + if (!name_p.parse(pos, table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + database = table; + if (!name_p.parse(pos, table, expected)) + return false; + } + + if (ParserKeyword{"ON"}.ignore(pos, expected)) + { + if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected)) + return false; + } + + if (!is_live_channel) + { + // TO [db.]table + if (ParserKeyword{"TO"}.ignore(pos, expected)) + { + if (!name_p.parse(pos, to_table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + to_database = to_table; + if (!name_p.parse(pos, to_table, expected)) + return false; + } + } + } + + /// Optional - a list of columns can be specified. It must fully comply with SELECT. + if (s_lparen.ignore(pos, expected)) + { + if (!columns_or_indices_p.parse(pos, columns_list, expected)) + return false; + + if (!s_rparen.ignore(pos, expected)) + return false; + } + + if (is_live_channel) + { + if (s_with.ignore(pos, expected)) + { + if (!names_p.parse(pos, tables, expected)) + return false; + } + } + else + { + /// AS SELECT ... + if (!s_as.ignore(pos, expected)) + return false; + + if (!select_p.parse(pos, select, expected)) + return false; + } + } else if (is_temporary) return false; else if (s_database.ignore(pos, expected)) @@ -538,6 +625,8 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->if_not_exists = if_not_exists; query->is_view = is_view; query->is_materialized_view = is_materialized_view; + query->is_live_view = is_live_view; + query->is_live_channel = is_live_channel; query->is_populate = is_populate; query->temporary = is_temporary; query->replace_view = replace_view; @@ -551,6 +640,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) query->set(query->columns_list, columns_list); query->set(query->storage, storage); + query->set(query->tables, tables); tryGetIdentifierNameInto(as_database, query->as_database); tryGetIdentifierNameInto(as_table, query->as_table); diff --git a/dbms/src/Parsers/ParserCreateQuery.h b/dbms/src/Parsers/ParserCreateQuery.h index d95e66566b2..076b0d94397 100644 --- a/dbms/src/Parsers/ParserCreateQuery.h +++ b/dbms/src/Parsers/ParserCreateQuery.h @@ -91,6 +91,14 @@ protected: bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); }; +/** List of table names. */ +class ParserNameList : public IParserBase +{ +protected: + const char * getName() const { return "name list"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + template class IParserColumnDeclaration : public IParserBase @@ -300,7 +308,7 @@ protected: * CREATE|ATTACH DATABASE db [ENGINE = engine] * * Or: - * CREATE [OR REPLACE]|ATTACH [MATERIALIZED] VIEW [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... + * CREATE[OR REPLACE]|ATTACH [[MATERIALIZED] VIEW] | [[TEMPORARY] LIVE [CHANNEL] | [VIEW]] [IF NOT EXISTS] [db.]name [TO [db.]name] [ENGINE = engine] [POPULATE] AS SELECT ... */ class ParserCreateQuery : public IParserBase { diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index c41e0946a96..1c44c639848 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -32,6 +33,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserCheckQuery check_p; ParserOptimizeQuery optimize_p; ParserKillQueryQuery kill_query_p; + ParserWatchQuery watch_p; ASTPtr query; @@ -57,7 +59,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || drop_p.parse(pos, query, expected) || check_p.parse(pos, query, expected) || kill_query_p.parse(pos, query, expected) - || optimize_p.parse(pos, query, expected); + || optimize_p.parse(pos, query, expected) + || watch_p.parse(pos, query, expected); if (!parsed) return false; diff --git a/dbms/src/Parsers/ParserWatchQuery.cpp b/dbms/src/Parsers/ParserWatchQuery.cpp new file mode 100644 index 00000000000..5171f4f2536 --- /dev/null +++ b/dbms/src/Parsers/ParserWatchQuery.cpp @@ -0,0 +1,77 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserWatchQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + ParserKeyword s_watch("WATCH"); + ParserToken s_dot(TokenType::Dot); + ParserIdentifier name_p; + ParserKeyword s_events("EVENTS"); + ParserKeyword s_limit("LIMIT"); + + ASTPtr database; + ASTPtr table; + auto query = std::make_shared(); + + if (!s_watch.ignore(pos, expected)) + { + return false; + } + + if (!name_p.parse(pos, table, expected)) + return false; + + if (s_dot.ignore(pos, expected)) + { + database = table; + if (!name_p.parse(pos, table, expected)) + return false; + } + + /// EVENTS + if (s_events.ignore(pos, expected)) + { + query->is_watch_events = true; + } + + /// LIMIT length + if (s_limit.ignore(pos, expected)) + { + ParserNumber num; + + if (!num.parse(pos, query->limit_length, expected)) + return false; + } + + if (database) + query->database = getIdentifierName(database); + + if (table) + query->table = getIdentifierName(table); + + node = query; + + return true; +} + + +} diff --git a/dbms/src/Parsers/ParserWatchQuery.h b/dbms/src/Parsers/ParserWatchQuery.h new file mode 100644 index 00000000000..330f0a432df --- /dev/null +++ b/dbms/src/Parsers/ParserWatchQuery.h @@ -0,0 +1,30 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include + + +namespace DB +{ + +/** Query like this: + * WATCH [db.]table EVENTS + */ +class ParserWatchQuery : public IParserBase +{ +protected: + const char * getName() const { return "WATCH query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index d3931e080f5..d2b6937cd01 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -29,8 +29,9 @@ protected: void consumeTotals(Chunk) override {} void consumeExtremes(Chunk) override {} -private: size_t field_number = 0; + +private: Names fields; FormatSettings settings; diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp new file mode 100644 index 00000000000..33fc1c9920b --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +void JSONEachRowWithProgressRowOutputFormat::writeRowStartDelimiter() +{ + writeCString("{\"row\":{", out); +} + +void JSONEachRowWithProgressRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("}}\n", out); + field_number = 0; +} + +void JSONEachRowWithProgressRowOutputFormat::onProgress(const Progress & value) +{ + progress.incrementPiecewiseAtomically(value); + writeCString("{\"progress\":", out); + progress.writeJSON(out); + writeCString("}\n", out); +} + + +void registerOutputFormatProcessorJSONEachRowWithProgress(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONEachRowWithProgress", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h new file mode 100644 index 00000000000..3062d664199 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h @@ -0,0 +1,20 @@ +#pragma once +#include + +namespace DB +{ + +class JSONEachRowWithProgressRowOutputFormat : public JSONEachRowRowOutputFormat +{ +public: + using JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat; + + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void onProgress(const Progress & value) override; + +private: + Progress progress; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 3b7ab9e6c31..4f42597eac5 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -111,7 +111,7 @@ static void fillColumnWithStringData(std::shared_ptr & arrow_colu static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = static_cast &>(*internal_column).getData(); - column_data.resize(arrow_column->length()); + column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i) { @@ -120,7 +120,7 @@ static void fillColumnWithBooleanData(std::shared_ptr & arrow_col std::shared_ptr buffer = chunk.data()->buffers[1]; for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) - column_data[bool_i] = chunk.Value(bool_i); + column_data.emplace_back(chunk.Value(bool_i)); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 88b1e3662ca..233e7aa25cb 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -76,7 +76,7 @@ public: /// The name of the table. virtual std::string getTableName() const = 0; - virtual std::string getDatabaseName() const = 0; + virtual std::string getDatabaseName() const { return {}; } /// Returns true if the storage receives data from a remote server or servers. virtual bool isRemote() const { return false; } @@ -102,7 +102,8 @@ public: virtual ColumnSizeByName getColumnSizes() const { return {}; } public: /// thread-unsafe part. lockStructure must be acquired - const ColumnsDescription & getColumns() const; /// returns combined set of columns + virtual const ColumnsDescription & getColumns() const; /// returns combined set of columns + virtual void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. const ColumnsDescription & getVirtuals() const; const IndicesDescription & getIndices() const; @@ -132,7 +133,6 @@ public: /// thread-unsafe part. lockStructure must be acquired void check(const Block & block, bool need_all = false) const; protected: /// still thread-unsafe part. - void setColumns(ColumnsDescription columns_); /// sets only real columns, possibly overwrites virtual ones. void setIndices(IndicesDescription indices_); /// Returns whether the column is virtual - by default all columns are real. @@ -172,6 +172,36 @@ public: */ virtual QueryProcessingStage::Enum getQueryProcessingStage(const Context &) const { return QueryProcessingStage::FetchColumns; } + /** Watch live changes to the table. + * Accepts a list of columns to read, as well as a description of the query, + * from which information can be extracted about how to retrieve data + * (indexes, locks, etc.) + * Returns a stream with which you can read data sequentially + * or multiple streams for parallel data reading. + * The `processed_stage` info is also written to what stage the request was processed. + * (Normally, the function only reads the columns from the list, but in other cases, + * for example, the request can be partially processed on a remote server.) + * + * context contains settings for one query. + * Usually Storage does not care about these settings, since they are used in the interpreter. + * But, for example, for distributed query processing, the settings are passed to the remote server. + * + * num_streams - a recommendation, how many streams to return, + * if the storage can return a different number of streams. + * + * It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). + */ + virtual BlockInputStreams watch( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum & /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) + { + throw Exception("Method watch is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + /** Read a set of columns from the table. * Accepts a list of columns to read, as well as a description of the query, * from which information can be extracted about how to retrieve data @@ -296,7 +326,7 @@ public: return {}; } - bool is_dropped{false}; + std::atomic is_dropped{false}; /// Does table support index for IN sections virtual bool supportsIndexForIn() const { return false; } diff --git a/dbms/src/Storages/LiveViewCommands.h b/dbms/src/Storages/LiveViewCommands.h new file mode 100644 index 00000000000..35015a7e5aa --- /dev/null +++ b/dbms/src/Storages/LiveViewCommands.h @@ -0,0 +1,65 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ + +#pragma once + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_STORAGE; +} + +struct LiveViewCommand +{ + enum Type + { + REFRESH + }; + + Type type; + + ASTPtr values; + + static LiveViewCommand refresh(const ASTPtr & values) + { + LiveViewCommand res; + res.type = REFRESH; + res.values = values; + return res; + } + + static std::optional parse(ASTAlterCommand * command) + { + if (command->type == ASTAlterCommand::LIVE_VIEW_REFRESH) + return refresh(command->values); + return {}; + } +}; + + +class LiveViewCommands : public std::vector +{ +public: + void validate(const IStorage & table) + { + if (!empty() && !dynamic_cast(&table)) + throw Exception("Wrong storage type. Must be StorageLiveView", DB::ErrorCodes::UNKNOWN_STORAGE); + } +}; + +} diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 74193fa7156..9c9dbe2dfb6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -521,7 +521,7 @@ public: /// parts should be sorted. MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mergePartsToTemporaryPart( - const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, + const FutureMergedMutatedPart & future_part, MergeList::Entry & merge_entry, TableStructureReadLockHolder &, time_t time_of_merge, DiskSpaceMonitor::Reservation * disk_reservation, bool deduplicate, bool force_ttl) { static const String TMP_PREFIX = "tmp_merge_"; @@ -883,7 +883,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor const FutureMergedMutatedPart & future_part, const std::vector & commands, MergeListEntry & merge_entry, - const Context & context) + const Context & context, + TableStructureReadLockHolder & table_lock_holder) { auto check_not_cancelled = [&]() { @@ -918,7 +919,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor command.partition, context_for_reading); }); - MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading); if (!mutations_interpreter.isStorageTouchedByMutations()) @@ -949,7 +949,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor Poco::File(new_part_tmp_path).createDirectories(); - auto in = mutations_interpreter.execute(); + auto in = mutations_interpreter.execute(table_lock_holder); const auto & updated_header = mutations_interpreter.getUpdatedHeader(); NamesAndTypesList all_columns = data.getColumns().getAllPhysical(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 43e3cd15910..914bbc8d06c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -94,14 +94,14 @@ public: */ MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart( const FutureMergedMutatedPart & future_part, - MergeListEntry & merge_entry, time_t time_of_merge, + MergeListEntry & merge_entry, TableStructureReadLockHolder & table_lock_holder, time_t time_of_merge, DiskSpaceMonitor::Reservation * disk_reservation, bool deduplication, bool force_ttl); /// Mutate a single data part with the specified commands. Will create and return a temporary part. MergeTreeData::MutableDataPartPtr mutatePartToTemporaryPart( const FutureMergedMutatedPart & future_part, const std::vector & commands, - MergeListEntry & merge_entry, const Context & context); + MergeListEntry & merge_entry, const Context & context, TableStructureReadLockHolder & table_lock_holder); MergeTreeData::DataPartPtr renameMergedTemporaryPart( MergeTreeData::MutableDataPartPtr & new_data_part, diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 665e8c9bd5c..28a6d017f22 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -1281,9 +1281,9 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep } if (!finished.empty()) + { zookeeper->set(replica_path + "/mutation_pointer", finished.back()->znode_name); - { std::lock_guard lock(state_mutex); mutation_pointer = finished.back()->znode_name; @@ -1655,7 +1655,7 @@ bool ReplicatedMergeTreeMergePredicate::operator()( std::optional ReplicatedMergeTreeMergePredicate::getDesiredMutationVersion(const MergeTreeData::DataPartPtr & part) const { /// Assigning mutations is easier than assigning merges because mutations appear in the same order as - /// the order of their version numbers (see StorageReplicatedMergeTree::mutate()). + /// the order of their version numbers (see StorageReplicatedMergeTree::mutate). /// This means that if we have loaded the mutation with version number X then all mutations with /// the version numbers less than X are also loaded and if there is no merge or mutation assigned to /// the part (checked by querying queue.virtual_parts), we can confidently assign a mutation to diff --git a/dbms/src/Storages/ProxyStorage.h b/dbms/src/Storages/ProxyStorage.h new file mode 100644 index 00000000000..60faa907209 --- /dev/null +++ b/dbms/src/Storages/ProxyStorage.h @@ -0,0 +1,66 @@ +#pragma once + +#include + +namespace DB +{ + +class ProxyStorage : public IStorage +{ +public: + ProxyStorage(StoragePtr storage_, BlockInputStreams streams_, QueryProcessingStage::Enum to_stage_) + : storage(std::move(storage_)), streams(std::move(streams_)), to_stage(to_stage_) {} + +public: + std::string getName() const override { return "ProxyStorage(" + storage->getName() + ")"; } + std::string getTableName() const override { return storage->getTableName(); } + + bool isRemote() const override { return storage->isRemote(); } + bool supportsSampling() const override { return storage->supportsSampling(); } + bool supportsFinal() const override { return storage->supportsFinal(); } + bool supportsPrewhere() const override { return storage->supportsPrewhere(); } + bool supportsReplication() const override { return storage->supportsReplication(); } + bool supportsDeduplication() const override { return storage->supportsDeduplication(); } + + QueryProcessingStage::Enum getQueryProcessingStage(const Context & /*context*/) const override { return to_stage; } + + BlockInputStreams read( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t /*max_block_size*/, + unsigned /*num_streams*/) override + { + return streams; + } + + bool supportsIndexForIn() const override { return storage->supportsIndexForIn(); } + bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context) const override { return storage->mayBenefitFromIndexForIn(left_in_operand, query_context); } + ASTPtr getPartitionKeyAST() const override { return storage->getPartitionKeyAST(); } + ASTPtr getSortingKeyAST() const override { return storage->getSortingKeyAST(); } + ASTPtr getPrimaryKeyAST() const override { return storage->getPrimaryKeyAST(); } + ASTPtr getSamplingKeyAST() const override { return storage->getSamplingKeyAST(); } + Names getColumnsRequiredForPartitionKey() const override { return storage->getColumnsRequiredForPartitionKey(); } + Names getColumnsRequiredForSortingKey() const override { return storage->getColumnsRequiredForSortingKey(); } + Names getColumnsRequiredForPrimaryKey() const override { return storage->getColumnsRequiredForPrimaryKey(); } + Names getColumnsRequiredForSampling() const override { return storage->getColumnsRequiredForSampling(); } + Names getColumnsRequiredForFinal() const override { return storage->getColumnsRequiredForFinal(); } + + const ColumnsDescription & getColumns() const override { return storage->getColumns(); } + void setColumns(ColumnsDescription columns_) override { return storage->setColumns(columns_); } + NameAndTypePair getColumn(const String & column_name) const override { return storage->getColumn(column_name); } + bool hasColumn(const String & column_name) const override { return storage->hasColumn(column_name); } + static StoragePtr createProxyStorage(StoragePtr storage, BlockInputStreams streams, QueryProcessingStage::Enum to_stage) + { + return std::make_shared(std::move(storage), std::move(streams), to_stage); + } +private: + StoragePtr storage; + BlockInputStreams streams; + QueryProcessingStage::Enum to_stage; +}; + + + +} diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index 49c2ebc7286..44c3cae23f2 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -317,8 +317,8 @@ BlockInputStreams StorageDistributed::read( const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table, remote_table_function_ptr); - Block header = materializeBlock( - InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock()); + Block header = + InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage)).getSampleBlock(); ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( diff --git a/dbms/src/Storages/StorageFactory.cpp b/dbms/src/Storages/StorageFactory.cpp index 1dd49e5c99b..7d92ce0ea2c 100644 --- a/dbms/src/Storages/StorageFactory.cpp +++ b/dbms/src/Storages/StorageFactory.cpp @@ -60,6 +60,22 @@ StoragePtr StorageFactory::get( name = "View"; } + else if (query.is_live_view) + { + + if (query.storage) + throw Exception("Specifying ENGINE is not allowed for a LiveView", ErrorCodes::INCORRECT_QUERY); + + name = "LiveView"; + } + else if (query.is_live_channel) + { + + if (query.storage) + throw Exception("Specifying ENGINE is not allowed for a LiveChannel", ErrorCodes::INCORRECT_QUERY); + + name = "LiveChannel"; + } else { /// Check for some special types, that are not allowed to be stored in tables. Example: NULL data type. @@ -115,6 +131,18 @@ StoragePtr StorageFactory::get( "Direct creation of tables with ENGINE MaterializedView is not supported, use CREATE MATERIALIZED VIEW statement", ErrorCodes::INCORRECT_QUERY); } + else if (name == "LiveView") + { + throw Exception( + "Direct creation of tables with ENGINE LiveView is not supported, use CREATE LIVE VIEW statement", + ErrorCodes::INCORRECT_QUERY); + } + else if (name == "LiveChannel") + { + throw Exception( + "Direct creation of tables with ENGINE LiveChannel is not supported, use CREATE LIVE CHANNEL statement", + ErrorCodes::INCORRECT_QUERY); + } } } diff --git a/dbms/src/Storages/StorageLiveView.cpp b/dbms/src/Storages/StorageLiveView.cpp new file mode 100644 index 00000000000..6e42b9780e1 --- /dev/null +++ b/dbms/src/Storages/StorageLiveView.cpp @@ -0,0 +1,479 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; + extern const int TABLE_WAS_NOT_DROPPED; + extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW; +} + +static void extractDependentTable(ASTSelectQuery & query, String & select_database_name, String & select_table_name) +{ + auto db_and_table = getDatabaseAndTable(query, 0); + ASTPtr subquery = extractTableExpression(query, 0); + + if (!db_and_table && !subquery) + return; + + if (db_and_table) + { + select_table_name = db_and_table->table; + + if (db_and_table->database.empty()) + { + db_and_table->database = select_database_name; + AddDefaultDatabaseVisitor visitor(select_database_name); + visitor.visit(query); + } + else + select_database_name = db_and_table->database; + } + else if (auto * ast_select = subquery->as()) + { + if (ast_select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + auto & inner_query = ast_select->list_of_selects->children.at(0); + + extractDependentTable(inner_query->as(), select_database_name, select_table_name); + } + else + throw Exception("Logical error while creating StorageLiveView." + " Could not retrieve table name from select query.", + DB::ErrorCodes::LOGICAL_ERROR); +} + +static void checkAllowedQueries(const ASTSelectQuery & query) +{ + if (query.prewhere() || query.final() || query.sample_size()) + throw Exception("LIVE VIEW cannot have PREWHERE, SAMPLE or FINAL.", DB::ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + ASTPtr subquery = extractTableExpression(query, 0); + if (!subquery) + return; + + if (const auto * ast_select = subquery->as()) + { + if (ast_select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + const auto & inner_query = ast_select->list_of_selects->children.at(0); + + checkAllowedQueries(inner_query->as()); + } +} + +StorageLiveView::StorageLiveView( + const String & table_name_, + const String & database_name_, + Context & local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns_) + : IStorage(columns_), table_name(table_name_), + database_name(database_name_), global_context(local_context.getGlobalContext()) +{ + if (!query.select) + throw Exception("SELECT query is not specified for " + getName(), ErrorCodes::INCORRECT_QUERY); + + /// Default value, if only table name exist in the query + select_database_name = local_context.getCurrentDatabase(); + if (query.select->list_of_selects->children.size() != 1) + throw Exception("UNION is not supported for LIVE VIEW", ErrorCodes::QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW); + + inner_query = query.select->list_of_selects->children.at(0); + + ASTSelectQuery & select_query = typeid_cast(*inner_query); + extractDependentTable(select_query, select_database_name, select_table_name); + + /// If the table is not specified - use the table `system.one` + if (select_table_name.empty()) + { + select_database_name = "system"; + select_table_name = "one"; + } + + global_context.addDependency( + DatabaseAndTableName(select_database_name, select_table_name), + DatabaseAndTableName(database_name, table_name)); + + is_temporary = query.temporary; + temporary_live_view_timeout = local_context.getSettingsRef().temporary_live_view_timeout.totalSeconds(); + + blocks_ptr = std::make_shared(); + blocks_metadata_ptr = std::make_shared(); + active_ptr = std::make_shared(true); +} + +NameAndTypePair StorageLiveView::getColumn(const String & column_name) const +{ + if (column_name == "_version") + return NameAndTypePair("_version", std::make_shared()); + + return IStorage::getColumn(column_name); +} + +bool StorageLiveView::hasColumn(const String & column_name) const +{ + if (column_name == "_version") + return true; + + return IStorage::hasColumn(column_name); +} + +Block StorageLiveView::getHeader() const +{ + if (!sample_block) + { + auto storage = global_context.getTable(select_database_name, select_table_name); + sample_block = InterpreterSelectQuery(inner_query, global_context, storage, + SelectQueryOptions(QueryProcessingStage::Complete)).getSampleBlock(); + sample_block.insert({DataTypeUInt64().createColumnConst( + sample_block.rows(), 0)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + /// convert all columns to full columns + /// in case some of them are constant + for (size_t i = 0; i < sample_block.columns(); ++i) + { + sample_block.safeGetByPosition(i).column = sample_block.safeGetByPosition(i).column->convertToFullColumnIfConst(); + } + } + + return sample_block; +} + +bool StorageLiveView::getNewBlocks() +{ + SipHash hash; + UInt128 key; + BlocksPtr new_blocks = std::make_shared(); + BlocksMetadataPtr new_blocks_metadata = std::make_shared(); + BlocksPtr new_mergeable_blocks = std::make_shared(); + + InterpreterSelectQuery interpreter(inner_query->clone(), global_context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + auto mergeable_stream = std::make_shared(interpreter.execute().in); + + while (Block block = mergeable_stream->read()) + new_mergeable_blocks->push_back(block); + + mergeable_blocks = std::make_shared>(); + mergeable_blocks->push_back(new_mergeable_blocks); + BlockInputStreamPtr from = std::make_shared(std::make_shared(new_mergeable_blocks), mergeable_stream->getHeader()); + auto proxy_storage = ProxyStorage::createProxyStorage(global_context.getTable(select_database_name, select_table_name), {from}, QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(inner_query->clone(), global_context, proxy_storage, SelectQueryOptions(QueryProcessingStage::Complete)); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, global_context.getSettingsRef().min_insert_block_size_rows, global_context.getSettingsRef().min_insert_block_size_bytes); + + while (Block block = data->read()) + { + /// calculate hash before virtual column is added + block.updateHash(hash); + /// add result version meta column + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), getBlocksVersion() + 1)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + new_blocks->push_back(block); + } + + hash.get128(key.low, key.high); + + /// Update blocks only if hash keys do not match + /// NOTE: hash could be different for the same result + /// if blocks are not in the same order + bool updated = false; + { + if (getBlocksHashKey() != key.toHexString()) + { + if (new_blocks->empty()) + { + new_blocks->push_back(getHeader()); + } + new_blocks_metadata->hash = key.toHexString(); + new_blocks_metadata->version = getBlocksVersion() + 1; + (*blocks_ptr) = new_blocks; + (*blocks_metadata_ptr) = new_blocks_metadata; + updated = true; + } + } + return updated; +} + +void StorageLiveView::checkTableCanBeDropped() const +{ + Dependencies dependencies = global_context.getDependencies(database_name, table_name); + if (!dependencies.empty()) + { + DatabaseAndTableName database_and_table_name = dependencies.front(); + throw Exception("Table has dependency " + database_and_table_name.first + "." + database_and_table_name.second, ErrorCodes::TABLE_WAS_NOT_DROPPED); + } +} + +void StorageLiveView::noUsersThread(const UInt64 & timeout) +{ + if (shutdown_called) + return; + + bool drop_table = false; + + { + while (1) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + if (!noUsersThreadWakeUp && !noUsersThreadCondition.tryWait(noUsersThreadMutex, + timeout * 1000)) + { + noUsersThreadWakeUp = false; + if (shutdown_called) + return; + if (hasUsers()) + return; + if (!global_context.getDependencies(database_name, table_name).empty()) + continue; + drop_table = true; + } + break; + } + } + + if (drop_table) + { + if (global_context.tryGetTable(database_name, table_name)) + { + try + { + /// We create and execute `drop` query for this table + auto drop_query = std::make_shared(); + drop_query->database = database_name; + drop_query->table = table_name; + drop_query->kind = ASTDropQuery::Kind::Drop; + ASTPtr ast_drop_query = drop_query; + InterpreterDropQuery drop_interpreter(ast_drop_query, global_context); + drop_interpreter.execute(); + } + catch (...) + { + } + } + } +} + +void StorageLiveView::startNoUsersThread(const UInt64 & timeout) +{ + bool expected = false; + if (!startnousersthread_called.compare_exchange_strong(expected, true)) + return; + + if (is_dropped) + return; + + if (is_temporary) + { + if (no_users_thread.joinable()) + { + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + } + no_users_thread.join(); + } + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = false; + } + if (!is_dropped) + no_users_thread = std::thread(&StorageLiveView::noUsersThread, this, timeout); + } + startnousersthread_called = false; +} + +void StorageLiveView::startup() +{ + startNoUsersThread(temporary_live_view_timeout); +} + +void StorageLiveView::shutdown() +{ + bool expected = false; + if (!shutdown_called.compare_exchange_strong(expected, true)) + return; + + if (no_users_thread.joinable()) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + /// Must detach the no users thread + /// as we can't join it as it will result + /// in a deadlock + no_users_thread.detach(); + } +} + +StorageLiveView::~StorageLiveView() +{ + shutdown(); +} + +void StorageLiveView::drop() +{ + global_context.removeDependency( + DatabaseAndTableName(select_database_name, select_table_name), + DatabaseAndTableName(database_name, table_name)); + Poco::FastMutex::ScopedLock lock(mutex); + is_dropped = true; + condition.broadcast(); +} + +void StorageLiveView::refresh(const Context & context) +{ + auto alter_lock = lockAlterIntention(context.getCurrentQueryId()); + { + Poco::FastMutex::ScopedLock lock(mutex); + if (getNewBlocks()) + condition.broadcast(); + } +} + +BlockInputStreams StorageLiveView::read( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + /// add user to the blocks_ptr + std::shared_ptr stream_blocks_ptr = blocks_ptr; + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + return { std::make_shared(stream_blocks_ptr, getHeader()) }; +} + +BlockInputStreams StorageLiveView::watch( + const Names & /*column_names*/, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t /*max_block_size*/, + const unsigned /*num_streams*/) +{ + ASTWatchQuery & query = typeid_cast(*query_info.query); + + bool has_limit = false; + UInt64 limit = 0; + + if (query.limit_length) + { + has_limit = true; + limit = safeGet(typeid_cast(*query.limit_length).value); + } + + if (query.is_watch_events) + { + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); + + if (no_users_thread.joinable()) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + } + + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + + processed_stage = QueryProcessingStage::Complete; + + return { reader }; + } + else + { + auto reader = std::make_shared(std::static_pointer_cast(shared_from_this()), blocks_ptr, blocks_metadata_ptr, active_ptr, has_limit, limit, context.getSettingsRef().live_view_heartbeat_interval.totalSeconds(), + context.getSettingsRef().temporary_live_view_timeout.totalSeconds()); + + if (no_users_thread.joinable()) + { + Poco::FastMutex::ScopedLock lock(noUsersThreadMutex); + noUsersThreadWakeUp = true; + noUsersThreadCondition.signal(); + } + + { + Poco::FastMutex::ScopedLock lock(mutex); + if (!(*blocks_ptr)) + { + if (getNewBlocks()) + condition.broadcast(); + } + } + + processed_stage = QueryProcessingStage::Complete; + + return { reader }; + } +} + +void registerStorageLiveView(StorageFactory & factory) +{ + factory.registerStorage("LiveView", [](const StorageFactory::Arguments & args) + { + return StorageLiveView::create(args.table_name, args.database_name, args.local_context, args.query, args.columns); + }); +} + +} diff --git a/dbms/src/Storages/StorageLiveView.h b/dbms/src/Storages/StorageLiveView.h new file mode 100644 index 00000000000..6bff9c2dc85 --- /dev/null +++ b/dbms/src/Storages/StorageLiveView.h @@ -0,0 +1,347 @@ +/* Copyright (c) 2018 BlackBerry Limited + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at +http://www.apache.org/licenses/LICENSE-2.0 +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. */ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +class IAST; + +struct BlocksMetadata +{ + String hash; + UInt64 version; +}; + +using ASTPtr = std::shared_ptr; +using BlocksMetadataPtr = std::shared_ptr; +using SipHashPtr = std::shared_ptr; + +class StorageLiveView : public ext::shared_ptr_helper, public IStorage +{ +friend struct ext::shared_ptr_helper; +friend class LiveViewBlockOutputStream; + +public: + ~StorageLiveView() override; + String getName() const override { return "LiveView"; } + String getTableName() const override { return table_name; } + String getDatabaseName() const override { return database_name; } + String getSelectDatabaseName() const { return select_database_name; } + String getSelectTableName() const { return select_table_name; } + + NameAndTypePair getColumn(const String & column_name) const override; + bool hasColumn(const String & column_name) const override; + + // const NamesAndTypesList & getColumnsListImpl() const override { return *columns; } + ASTPtr getInnerQuery() const { return inner_query->clone(); } + + /// It is passed inside the query and solved at its level. + bool supportsSampling() const override { return true; } + bool supportsFinal() const override { return true; } + + /// Mutex for the blocks and ready condition + Poco::FastMutex mutex; + /// New blocks ready condition to broadcast to readers + /// that new blocks are available + Poco::Condition condition; + + bool isTemporary() { return is_temporary; } + + /// Check if we have any readers + /// must be called with mutex locked + bool hasUsers() + { + return blocks_ptr.use_count() > 1; + } + + /// Check we have any active readers + /// must be called with mutex locked + bool hasActiveUsers() + { + return active_ptr.use_count() > 1; + } + /// Background thread for temporary tables + /// which drops this table if there are no users + void startNoUsersThread(const UInt64 & timeout); + Poco::FastMutex noUsersThreadMutex; + bool noUsersThreadWakeUp{false}; + Poco::Condition noUsersThreadCondition; + /// Get blocks hash + /// must be called with mutex locked + String getBlocksHashKey() + { + if (*blocks_metadata_ptr) + return (*blocks_metadata_ptr)->hash; + return ""; + } + /// Get blocks version + /// must be called with mutex locked + UInt64 getBlocksVersion() + { + if (*blocks_metadata_ptr) + return (*blocks_metadata_ptr)->version; + return 0; + } + + /// Reset blocks + /// must be called with mutex locked + void reset() + { + (*blocks_ptr).reset(); + if (*blocks_metadata_ptr) + (*blocks_metadata_ptr)->hash.clear(); + mergeable_blocks.reset(); + } + + void checkTableCanBeDropped() const override; + void drop() override; + void startup() override; + void shutdown() override; + + void refresh(const Context & context); + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + BlockInputStreams watch( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + std::shared_ptr getBlocksPtr() { return blocks_ptr; } + BlocksPtrs getMergeableBlocks() { return mergeable_blocks; } + void setMergeableBlocks(BlocksPtrs blocks) { mergeable_blocks = blocks; } + std::shared_ptr getActivePtr() { return active_ptr; } + + /// Read new data blocks that store query result + bool getNewBlocks(); + + Block getHeader() const; + + static void writeIntoLiveView(StorageLiveView & live_view, + const Block & block, + const Context & context, + BlockOutputStreamPtr & output) + { + /// Check if live view has any readers if not + /// just reset blocks to empty and do nothing else + /// When first reader comes the blocks will be read. + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + if (!live_view.hasActiveUsers()) + { + live_view.reset(); + return; + } + } + + bool is_block_processed = false; + BlockInputStreams from; + BlocksPtrs mergeable_blocks; + BlocksPtr new_mergeable_blocks = std::make_shared(); + + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + + mergeable_blocks = live_view.getMergeableBlocks(); + if (!mergeable_blocks || mergeable_blocks->size() >= context.getGlobalContext().getSettingsRef().max_live_view_insert_blocks_before_refresh) + { + mergeable_blocks = std::make_shared>(); + BlocksPtr base_mergeable_blocks = std::make_shared(); + InterpreterSelectQuery interpreter(live_view.getInnerQuery(), context, SelectQueryOptions(QueryProcessingStage::WithMergeableState), Names()); + auto view_mergeable_stream = std::make_shared( + interpreter.execute().in); + while (Block this_block = view_mergeable_stream->read()) + base_mergeable_blocks->push_back(this_block); + mergeable_blocks->push_back(base_mergeable_blocks); + live_view.setMergeableBlocks(mergeable_blocks); + + /// Create from streams + for (auto & blocks_ : *mergeable_blocks) + { + if (blocks_->empty()) + continue; + auto sample_block = blocks_->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + from.push_back(std::move(stream)); + } + + is_block_processed = true; + } + } + + if (!is_block_processed) + { + auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + BlockInputStreams streams = {std::make_shared(block)}; + auto proxy_storage = std::make_shared(parent_storage, std::move(streams), QueryProcessingStage::FetchColumns); + InterpreterSelectQuery select_block(live_view.getInnerQuery(), + context, proxy_storage, + QueryProcessingStage::WithMergeableState); + auto data_mergeable_stream = std::make_shared( + select_block.execute().in); + while (Block this_block = data_mergeable_stream->read()) + new_mergeable_blocks->push_back(this_block); + + if (new_mergeable_blocks->empty()) + return; + + { + Poco::FastMutex::ScopedLock lock(live_view.mutex); + + mergeable_blocks = live_view.getMergeableBlocks(); + mergeable_blocks->push_back(new_mergeable_blocks); + + /// Create from streams + for (auto & blocks_ : *mergeable_blocks) + { + if (blocks_->empty()) + continue; + auto sample_block = blocks_->front().cloneEmpty(); + BlockInputStreamPtr stream = std::make_shared(std::make_shared(blocks_), sample_block); + from.push_back(std::move(stream)); + } + } + } + + auto parent_storage = context.getTable(live_view.getSelectDatabaseName(), live_view.getSelectTableName()); + auto proxy_storage = std::make_shared(parent_storage, std::move(from), QueryProcessingStage::WithMergeableState); + InterpreterSelectQuery select(live_view.getInnerQuery(), context, proxy_storage, QueryProcessingStage::Complete); + BlockInputStreamPtr data = std::make_shared(select.execute().in); + + /// Squashing is needed here because the view query can generate a lot of blocks + /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY + /// and two-level aggregation is triggered). + data = std::make_shared( + data, context.getGlobalContext().getSettingsRef().min_insert_block_size_rows, context.getGlobalContext().getSettingsRef().min_insert_block_size_bytes); + + copyData(*data, *output); + } + +private: + String select_database_name; + String select_table_name; + String table_name; + String database_name; + ASTPtr inner_query; + Context & global_context; + bool is_temporary {false}; + mutable Block sample_block; + + /// Active users + std::shared_ptr active_ptr; + /// Current data blocks that store query result + std::shared_ptr blocks_ptr; + /// Current data blocks metadata + std::shared_ptr blocks_metadata_ptr; + BlocksPtrs mergeable_blocks; + + void noUsersThread(const UInt64 & timeout); + std::thread no_users_thread; + std::atomic shutdown_called{false}; + std::atomic startnousersthread_called{false}; + UInt64 temporary_live_view_timeout; + + StorageLiveView( + const String & table_name_, + const String & database_name_, + Context & local_context, + const ASTCreateQuery & query, + const ColumnsDescription & columns + ); +}; + +class LiveViewBlockOutputStream : public IBlockOutputStream +{ +public: + explicit LiveViewBlockOutputStream(StorageLiveView & storage_) : storage(storage_) {} + + void writePrefix() override + { + new_blocks = std::make_shared(); + new_blocks_metadata = std::make_shared(); + new_hash = std::make_shared(); + } + + void writeSuffix() override + { + UInt128 key; + String key_str; + + new_hash->get128(key.low, key.high); + key_str = key.toHexString(); + + Poco::FastMutex::ScopedLock lock(storage.mutex); + + if (storage.getBlocksHashKey() != key_str) + { + new_blocks_metadata->hash = key_str; + new_blocks_metadata->version = storage.getBlocksVersion() + 1; + + for (auto & block : *new_blocks) + { + block.insert({DataTypeUInt64().createColumnConst( + block.rows(), new_blocks_metadata->version)->convertToFullColumnIfConst(), + std::make_shared(), + "_version"}); + } + + (*storage.blocks_ptr) = new_blocks; + (*storage.blocks_metadata_ptr) = new_blocks_metadata; + + storage.condition.broadcast(); + } + + new_blocks.reset(); + new_blocks_metadata.reset(); + new_hash.reset(); + } + + void write(const Block & block) override + { + new_blocks->push_back(block); + block.updateHash(*new_hash); + } + + Block getHeader() const override { return storage.getHeader(); } + +private: + BlocksPtr new_blocks; + BlocksMetadataPtr new_blocks_metadata; + SipHashPtr new_hash; + StorageLiveView & storage; +}; + +} diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index d062bb197ca..4003a51e301 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -206,29 +206,35 @@ std::vector StorageMergeTree::prepar const Settings & settings_ = context.getSettingsRef(); size_t thread_pool_size = std::min(parts.size(), settings_.max_alter_threads); - ThreadPool thread_pool(thread_pool_size); + std::optional thread_pool; + + if (thread_pool_size > 1) + thread_pool.emplace(thread_pool_size); for (const auto & part : parts) { transactions.push_back(std::make_unique(part)); - thread_pool.schedule( - [this, & transaction = transactions.back(), & columns_for_parts, & new_indices = new_indices.indices] - { - this->alterDataPart(columns_for_parts, new_indices, false, transaction); - } - ); + auto job = [this, & transaction = transactions.back(), & columns_for_parts, & new_indices = new_indices.indices] + { + this->alterDataPart(columns_for_parts, new_indices, false, transaction); + }; + + if (thread_pool) + thread_pool->schedule(job); + else + job(); } - thread_pool.wait(); + if (thread_pool) + thread_pool->wait(); auto erase_pos = std::remove_if(transactions.begin(), transactions.end(), [](const MergeTreeData::AlterDataPartTransactionPtr & transaction) { return !transaction->isValid(); - } - ); + }); transactions.erase(erase_pos, transactions.end()); return transactions; @@ -596,7 +602,7 @@ bool StorageMergeTree::merge( bool force_ttl = (final && (hasTableTTL() || hasAnyColumnTTL())); new_part = merger_mutator.mergePartsToTemporaryPart( - future_part, *merge_entry, time(nullptr), + future_part, *merge_entry, table_lock_holder, time(nullptr), merging_tagger->reserved_space.get(), deduplicate, force_ttl); merger_mutator.renameMergedTemporaryPart(new_part, future_part.parts, nullptr); removeEmptyColumnsFromPart(new_part); @@ -714,7 +720,7 @@ bool StorageMergeTree::tryMutatePart() try { - new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, global_context); + new_part = merger_mutator.mutatePartToTemporaryPart(future_part, commands, *merge_entry, global_context, table_lock_holder); renameTempPartAndReplace(new_part); tagger->is_successful = true; write_part_log({}); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 3781b9efea5..bc0bf3fdc63 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -91,7 +91,6 @@ namespace ErrorCodes extern const int NOT_FOUND_NODE; extern const int NO_ACTIVE_REPLICAS; extern const int LEADERSHIP_CHANGED; - extern const int TABLE_IS_READ_ONLY; extern const int TABLE_WAS_NOT_DROPPED; extern const int PARTITION_ALREADY_EXISTS; extern const int TOO_MANY_RETRIES_TO_FETCH_PARTS; @@ -1088,7 +1087,7 @@ bool StorageReplicatedMergeTree::tryExecuteMerge(const LogEntry & entry) try { part = merger_mutator.mergePartsToTemporaryPart( - future_merged_part, *merge_entry, entry.create_time, reserved_space.get(), entry.deduplicate, entry.force_ttl); + future_merged_part, *merge_entry, table_lock, entry.create_time, reserved_space.get(), entry.deduplicate, entry.force_ttl); merger_mutator.renameMergedTemporaryPart(part, parts, &transaction); removeEmptyColumnsFromPart(part); @@ -1217,7 +1216,7 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM try { - new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, global_context); + new_part = merger_mutator.mutatePartToTemporaryPart(future_mutated_part, commands, *merge_entry, global_context, table_lock); renameTempPartAndReplace(new_part, nullptr, &transaction); try @@ -3111,9 +3110,10 @@ bool StorageReplicatedMergeTree::optimize(const ASTPtr & query, const ASTPtr & p } } - /// TODO: Bad setting name for such purpose if (query_context.getSettingsRef().replication_alter_partitions_sync != 0) { + /// NOTE Table lock must not be held while waiting. Some combination of R-W-R locks from different threads will yield to deadlock. + /// TODO Check all other "wait" places. for (auto & merge_entry : merge_entries) waitForAllReplicasToProcessLogEntry(merge_entry); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 1d967021e11..c9ec99e9e6a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -470,10 +470,15 @@ private: /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica . + * + * NOTE: This method must be called without table lock held. + * Because it effectively waits for other thread that usually has to also acquire a lock to proceed and this yields deadlock. + * TODO: There are wrong usages of this method that are not fixed yet. */ void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry); /** Wait until the specified replica executes the specified action from the log. + * NOTE: See comment about locks above. */ void waitForReplicaToProcessLogEntry(const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry); diff --git a/dbms/src/Storages/System/StorageSystemNumbers.cpp b/dbms/src/Storages/System/StorageSystemNumbers.cpp index c909a338453..2afe2a7c018 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.cpp +++ b/dbms/src/Storages/System/StorageSystemNumbers.cpp @@ -5,9 +5,13 @@ #include #include + namespace DB { +namespace +{ + class NumbersBlockInputStream : public IBlockInputStream { public: @@ -43,8 +47,64 @@ private: }; -StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_, UInt64 offset_) - : name(name_), multithreaded(multithreaded_), limit(limit_), offset(offset_) +struct NumbersMultiThreadedState +{ + std::atomic counter; + explicit NumbersMultiThreadedState(UInt64 offset) : counter(offset) {} +}; + +using NumbersMultiThreadedStatePtr = std::shared_ptr; + +class NumbersMultiThreadedBlockInputStream : public IBlockInputStream +{ +public: + NumbersMultiThreadedBlockInputStream(NumbersMultiThreadedStatePtr state_, UInt64 block_size_, UInt64 max_counter_) + : state(std::move(state_)), block_size(block_size_), max_counter(max_counter_) {} + + String getName() const override { return "NumbersMt"; } + + Block getHeader() const override + { + return { ColumnWithTypeAndName(ColumnUInt64::create(), std::make_shared(), "number") }; + } + +protected: + Block readImpl() override + { + if (block_size == 0) + return {}; + + UInt64 curr = state->counter.fetch_add(block_size, std::memory_order_acquire); + + if (curr >= max_counter) + return {}; + + if (curr + block_size > max_counter) + block_size = max_counter - curr; + + auto column = ColumnUInt64::create(block_size); + ColumnUInt64::Container & vec = column->getData(); + + UInt64 * pos = vec.data(); + UInt64 * end = &vec[block_size]; + while (pos < end) + *pos++ = curr++; + + return { ColumnWithTypeAndName(std::move(column), std::make_shared(), "number") }; + } + +private: + NumbersMultiThreadedStatePtr state; + + UInt64 block_size; + UInt64 max_counter; +}; + +} + + +StorageSystemNumbers::StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_, UInt64 offset_, bool even_distribution_) + : name(name_), multithreaded(multithreaded_), even_distribution(even_distribution_), limit(limit_), offset(offset_) { setColumns(ColumnsDescription({{"number", std::make_shared()}})); } @@ -69,6 +129,18 @@ BlockInputStreams StorageSystemNumbers::read( num_streams = 1; BlockInputStreams res(num_streams); + + if (num_streams > 1 && !even_distribution && *limit) + { + auto state = std::make_shared(offset); + UInt64 max_counter = offset + *limit; + + for (size_t i = 0; i < num_streams; ++i) + res[i] = std::make_shared(state, max_block_size, max_counter); + + return res; + } + for (size_t i = 0; i < num_streams; ++i) { res[i] = std::make_shared(max_block_size, offset + i * max_block_size, num_streams * max_block_size); diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 452ec5a9ef5..5efc23a1125 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -19,6 +19,9 @@ class Context; * If multithreaded is specified, numbers will be generated in several streams * (and result could be out of order). If both multithreaded and limit are specified, * the table could give you not exactly 1..limit range, but some arbitrary 'limit' numbers. + * + * In multithreaded case, if even_distributed is False, implementation with atomic is used, + * and result is always in [0 ... limit - 1] range. */ class StorageSystemNumbers : public ext::shared_ptr_helper, public IStorage { @@ -38,11 +41,14 @@ public: private: const std::string name; bool multithreaded; + bool even_distribution; std::optional limit; UInt64 offset; protected: - StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0); + /// If even_distribution is true, numbers are distributed evenly between streams. + /// Otherwise, streams concurrently increment atomic. + StorageSystemNumbers(const std::string & name_, bool multithreaded_, std::optional limit_ = std::nullopt, UInt64 offset_ = 0, bool even_distribution_ = true); }; } diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index 991716fe08c..811a031c7a3 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -24,6 +24,8 @@ void registerStorageSet(StorageFactory & factory); void registerStorageJoin(StorageFactory & factory); void registerStorageView(StorageFactory & factory); void registerStorageMaterializedView(StorageFactory & factory); +void registerStorageLiveView(StorageFactory & factory); +//void registerStorageLiveChannel(StorageFactory & factory); #if USE_HDFS void registerStorageHDFS(StorageFactory & factory); @@ -64,6 +66,8 @@ void registerStorages() registerStorageJoin(factory); registerStorageView(factory); registerStorageMaterializedView(factory); + registerStorageLiveView(factory); + //registerStorageLiveChannel(factory); #if USE_HDFS registerStorageHDFS(factory); diff --git a/dbms/src/TableFunctions/TableFunctionNumbers.cpp b/dbms/src/TableFunctions/TableFunctionNumbers.cpp index 94c618bd9b1..14947a40812 100644 --- a/dbms/src/TableFunctions/TableFunctionNumbers.cpp +++ b/dbms/src/TableFunctions/TableFunctionNumbers.cpp @@ -16,34 +16,35 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } - -StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const +template +StoragePtr TableFunctionNumbers::executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const { if (const auto * function = ast_function->as()) { auto arguments = function->arguments->children; if (arguments.size() != 1 && arguments.size() != 2) - throw Exception("Table function 'numbers' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Table function '" + getName() + "' requires 'length' or 'offset, length'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); UInt64 offset = arguments.size() == 2 ? evaluateArgument(context, arguments[0]) : 0; UInt64 length = arguments.size() == 2 ? evaluateArgument(context, arguments[1]) : evaluateArgument(context, arguments[0]); - auto res = StorageSystemNumbers::create(table_name, false, length, offset); + auto res = StorageSystemNumbers::create(table_name, multithreaded, length, offset, false); res->startup(); return res; } - throw Exception("Table function 'numbers' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Table function '" + getName() + "' requires 'limit' or 'offset, limit'.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } void registerTableFunctionNumbers(TableFunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction>(); + factory.registerFunction>(); } - -UInt64 TableFunctionNumbers::evaluateArgument(const Context & context, ASTPtr & argument) const +template +UInt64 TableFunctionNumbers::evaluateArgument(const Context & context, ASTPtr & argument) const { return evaluateConstantExpressionOrIdentifierAsLiteral(argument, context)->as().value.safeGet(); } diff --git a/dbms/src/TableFunctions/TableFunctionNumbers.h b/dbms/src/TableFunctions/TableFunctionNumbers.h index 733b4508f51..e5ab38ccad8 100644 --- a/dbms/src/TableFunctions/TableFunctionNumbers.h +++ b/dbms/src/TableFunctions/TableFunctionNumbers.h @@ -7,14 +7,15 @@ namespace DB { -/* numbers(limit) +/* numbers(limit), numbers_mt(limit) * - the same as SELECT number FROM system.numbers LIMIT limit. * Used for testing purposes, as a simple example of table function. */ +template class TableFunctionNumbers : public ITableFunction { public: - static constexpr auto name = "numbers"; + static constexpr auto name = multithreaded ? "numbers_mt" : "numbers"; std::string getName() const override { return name; } private: StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; diff --git a/dbms/tests/instructions/sanitizers.md b/dbms/tests/instructions/sanitizers.md index 3ec8d30ae09..cfa465fcbda 100644 --- a/dbms/tests/instructions/sanitizers.md +++ b/dbms/tests/instructions/sanitizers.md @@ -40,7 +40,7 @@ ninja ## Start ClickHouse and run tests ``` -sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1,suppressions=../dbms/tests/tsan_suppressions.txt' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml +sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml ``` diff --git a/dbms/tests/performance/general_purpose_hashes.xml b/dbms/tests/performance/general_purpose_hashes.xml index c2462f8710b..b7a1b915ff0 100644 --- a/dbms/tests/performance/general_purpose_hashes.xml +++ b/dbms/tests/performance/general_purpose_hashes.xml @@ -1,18 +1,19 @@ - once + loop - - 10000 + + 3 + 10000 - 5000 - 20000 + 5 + 60000 - + @@ -45,11 +46,11 @@ table - numbers - numbers_mt + numbers(1000000) + numbers_mt(10000000) - SELECT ignore({gp_hash_func}({string})) FROM system.{table} + SELECT count() from {table} where not ignore({gp_hash_func}({string})) diff --git a/dbms/tests/queries/0_stateless/00459_group_array_insert_at.sql b/dbms/tests/queries/0_stateless/00459_group_array_insert_at.sql index 9e2c1ee441e..59ecfc05045 100644 --- a/dbms/tests/queries/0_stateless/00459_group_array_insert_at.sql +++ b/dbms/tests/queries/0_stateless/00459_group_array_insert_at.sql @@ -2,5 +2,5 @@ SELECT groupArrayInsertAt(toString(number), number * 2) FROM (SELECT * FROM syst SELECT groupArrayInsertAt('-')(toString(number), number * 2) FROM (SELECT * FROM system.numbers LIMIT 10); SELECT groupArrayInsertAt([123])(range(number), number * 2) FROM (SELECT * FROM system.numbers LIMIT 10); SELECT number, groupArrayInsertAt(number, number) FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number ORDER BY number; -SELECT k, ignore(groupArrayInsertAt(x, x)) FROM (SELECT dummy AS k, randConstant() % 10 AS x FROM remote('127.0.0.{1,1}', system.one)) GROUP BY k ORDER BY k; +SELECT k, ignore(groupArrayInsertAt(x, x)) FROM (SELECT dummy AS k, (randConstant() * 10) % 10 AS x FROM remote('127.0.0.{1,1}', system.one)) GROUP BY k ORDER BY k; SELECT k, groupArrayInsertAt('-', 10)(toString(x), x) FROM (SELECT number AS k, number AS x FROM system.numbers LIMIT 11) GROUP BY k ORDER BY k; diff --git a/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh b/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh index 9081580ddfc..170259f0e24 100755 --- a/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh +++ b/dbms/tests/queries/0_stateless/00704_drop_truncate_memory_table.sh @@ -20,7 +20,7 @@ INSERT INTO memory SELECT * FROM numbers(1000);" ${CLICKHOUSE_CLIENT} --multiquery --query=" SET max_threads = 1; -SELECT count() FROM memory WHERE NOT ignore(sleep(0.0001));" 2>&1 | grep -c -P '^1000$|Table .+? doesn.t exist' & +SELECT count() FROM memory WHERE NOT ignore(sleep(0.0001));" 2>&1 | grep -c -P '^1000$|^0$|Table .+? doesn.t exist' & sleep 0.05; diff --git a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl index dd12d9ffdfa..101245b9f60 100755 --- a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl +++ b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.pl @@ -133,7 +133,6 @@ sub main { split /[\s;,]+/, $ENV{SQL_FUZZY_FUNCTIONS} || file_read($ENV{SQL_FUZZY_FILE_FUNCTIONS} || 'clickhouse-functions') - || '__inner_restore_projection__ __inner_build_projection_composition__ convertCharset one_or_zero findClusterValue findClusterIndex toNullable coalesce isNotNull geoToH3 pointInEllipses transform pow acos asin tan cos tgamma lgamma erfc erf sqrt log10 exp10 e visitParamExtractFloat visitParamExtractUInt decodeURLComponent cutURLParameter cutQueryStringAndFragment cutFragment cutWWW URLPathHierarchy URLHierarchy extractURLParameterNames extractURLParameter queryStringAndFragment pathFull sin topLevelDomain domainWithoutWWW domain protocol greatCircleDistance extract match positionCaseInsensitiveUTF8 positionCaseInsensitive positionUTF8 position replaceRegexpAll replaceRegexpOne arrayStringConcat splitByString splitByChar alphaTokens endsWith startsWith appendTrailingCharIfAbsent substringUTF8 concatAssumeInjective reverseUTF8 upperUTF8 __inner_project__ upper lower CRC32 length notEmpty trunc round roundAge roundDuration roundToExp2 reinterpretAsString reinterpretAsDateTime reinterpretAsDate reinterpretAsFloat64 reinterpretAsFloat32 reinterpretAsInt64 reinterpretAsInt8 reinterpretAsUInt32 toStartOfFiveMinute toISOYear toISOWeek concat toDecimal64 ifNull toStartOfDay toSecond addSeconds sleepEachRow materialize visitParamExtractInt toStartOfMinute toDayOfWeek toDayOfMonth bitShiftLeft emptyArrayUInt8 parseDateTimeBestEffort toTime toDateTimeOrNull toFloat32OrNull toInt16 IPv6NumToString atan substring arrayIntersect isInfinite toRelativeHourNum hex arrayEnumerateDense toUInt8OrZero toRelativeSecondNum toUInt64OrNull MACNumToString toInt32OrNull toDayOfYear toUnixTimestamp toString toDateOrZero subtractDays toMinute murmurHash3_64 murmurHash2_32 toUInt64 toUInt8 dictGetDateTime empty isFinite caseWithoutExpression caseWithoutExpr visitParamExtractRaw queryString dictGetInt32OrDefault caseWithExpression toInt8OrZero multiIf if intExp10 bitShiftRight less toUInt8OrNull toInt8OrNull bitmaskToArray toIntervalYear toFloat64OrZero dateDiff generateUUIDv4 arrayPopBack toIntervalMonth toUUID notEquals toInt16OrNull murmurHash2_64 hasAny toIntervalMinute isNull tupleElement replaceAll parseDateTimeBestEffortOrZero toFloat32OrZero lowerUTF8 notIn gcd like regionToPopulation MACStringToOUI notLike toStringCutToZero lcm parseDateTimeBestEffortOrNull not toInt32OrZero arrayFilter toInt16OrZero range equals now toTypeName toUInt32OrNull emptyArrayString dictGetDateTimeOrDefault bitRotateRight cutIPv6 toUInt32OrZero timezone reverse runningDifferenceStartingWithFirstValue toDateTime arrayPopFront toInt32 intHash64 extractURLParameters lowCardinalityIndices toStartOfMonth toYear hasAll rowNumberInAllBlocks bitTestAll arrayCount arraySort abs bitNot intDiv intDivOrZero firstSignificantSubdomain dictGetFloat32OrDefault reinterpretAsUInt16 toHour minus regionToArea unhex IPv4StringToNum toIntervalHour toInt8 dictGetFloat32 log IPv4NumToString modulo arrayEnumerate cutQueryString reinterpretAsFixedString countEqual bitTest toDecimal128 plus or reinterpretAsUInt64 toMonth visitParamExtractBool emptyArrayUInt64 replaceOne arrayReverseSort toFloat32 toRelativeMonthNum emptyArrayInt32 toRelativeYearNum arrayElement log2 array arrayReverse toUInt64OrZero emptyArrayFloat64 negate arrayPushBack subtractWeeks bitTestAny bitAnd toDecimal32 arrayPushFront lessOrEquals intExp2 toUInt16OrZero arrayConcat arrayCumSum arraySlice addDays dictGetUInt8 toUInt32 bitOr caseWithExpr toStartOfYear toIntervalDay MD5 emptyArrayUInt32 emptyArrayInt8 toMonday addMonths arrayUniq SHA256 arrayExists multiply toUInt16OrNull dictGetInt8 visitParamHas emptyArrayInt64 toIntervalSecond toDate sleep emptyArrayToSingle path toInt64OrZero SHA1 extractAll emptyArrayDate dumpColumnStructure toInt64 lengthUTF8 greatest arrayEnumerateUniq arrayDistinct arrayFirst toFixedString IPv4NumToStringClassC toFloat64OrNull IPv4ToIPv6 identity ceil toStartOfQuarter dictGetInt8OrDefault MACStringToNum emptyArrayUInt16 UUIDStringToNum dictGetUInt16 toStartOfFifteenMinutes toStartOfHour sumburConsistentHash toStartOfISOYear toRelativeQuarterNum toRelativeWeekNum toRelativeDayNum cbrt yesterday bitXor timeSlot timeSlots emptyArrayInt16 dictGetInt16 toYYYYMM toYYYYMMDDhhmmss toUInt16 addMinutes addHours addWeeks nullIf subtractSeconds subtractMinutes toIntervalWeek subtractHours isNaN subtractMonths toDateOrNull subtractYears toTimeZone formatDateTime has cityHash64 intHash32 fragment regionToCity indexOf regionToDistrict regionToCountry visibleWidth regionToContinent regionToTopContinent toColumnTypeName regionHierarchy CHAR_LENGTH least divide SEHierarchy dictGetDate OSToRoot SEToRoot OSIn SEIn regionToName dictGetStringOrDefault OSHierarchy exp floor dictGetUInt8OrDefault dictHas dictGetUInt64 cutToFirstSignificantSubdomain dictGetInt32 pointInPolygon dictGetInt64 blockNumber IPv6StringToNum dictGetString dictGetFloat64 dictGetUUID CHARACTER_LENGTH toQuarter dictGetHierarchy toFloat64 arraySum toInt64OrNull dictIsIn dictGetUInt16OrDefault dictGetUInt32OrDefault emptyArrayDateTime greater jumpConsistentHash dictGetUInt64OrDefault dictGetInt16OrDefault dictGetInt64OrDefault reinterpretAsInt32 dictGetUInt32 murmurHash3_32 bar dictGetUUIDOrDefault rand modelEvaluate arrayReduce farmHash64 bitmaskToList formatReadableSize halfMD5 SHA224 arrayMap sipHash64 dictGetFloat64OrDefault sipHash128 metroHash64 murmurHash3_128 yandexConsistentHash emptyArrayFloat32 arrayAll toYYYYMMDD today arrayFirstIndex greaterOrEquals arrayDifference visitParamExtractString toDateTimeOrZero globalNotIn throwIf and xor currentDatabase hostName URLHash getSizeOfEnumType defaultValueOfArgumentType blockSize tuple arrayCumSumNonNegative rowNumberInBlock arrayResize ignore toRelativeMinuteNum indexHint reinterpretAsInt16 addYears arrayJoin replicate hasColumnInTable version regionIn uptime runningAccumulate runningDifference assumeNotNull pi finalizeAggregation toLowCardinality exp2 lowCardinalityKeys in globalIn dictGetDateOrDefault rand64 CAST bitRotateLeft randConstant UUIDNumToString reinterpretAsUInt8 truncate ceiling retention maxIntersections groupBitXor groupBitOr uniqUpTo uniqCombined uniqExact uniq covarPop stddevPop varPop covarSamp varSamp sumMap corrStable corr quantileTiming quantileDeterministic quantilesExact uniqHLL12 quantilesTiming covarPopStable stddevSampStable quantilesExactWeighted quantileExactWeighted quantileTimingWeighted quantileExact quantilesDeterministic quantiles topK sumWithOverflow count groupArray stddevSamp groupArrayInsertAt quantile quantilesTimingWeighted quantileTDigest quantilesTDigest windowFunnel min argMax varSampStable maxIntersectionsPosition quantilesTDigestWeighted groupUniqArray sequenceCount sumKahan any anyHeavy histogram quantileTDigestWeighted max groupBitAnd argMin varPopStable avg sequenceMatch stddevPopStable sum anyLast covarSampStable BIT_XOR medianExactWeighted medianTiming medianExact median medianDeterministic VAR_SAMP STDDEV_POP medianTDigest VAR_POP medianTDigestWeighted BIT_OR STDDEV_SAMP medianTimingWeighted COVAR_SAMP COVAR_POP BIT_AND' ]; # $functions = [grep { not $_ ~~ [qw( )] } @$functions]; # will be removed # select name from system.table_functions format TSV; @@ -141,7 +140,6 @@ sub main { split /[\s;,]+/, $ENV{SQL_FUZZY_TABLE_FUNCTIONS} || file_read($ENV{SQL_FUZZY_FILE_TABLE_FUNCTIONS} || 'clickhouse-table-functions') - || 'mysql jdbc odbc remote catBoostPool merge file cluster shardByHash url numbers' ]; $table_functions = [grep { not $_ ~~ [qw(numbers)] } @$table_functions]; # too slow say one_of({}, $query), ';' for 1 .. ($ENV{SQL_FUZZY_LINES} || 100); diff --git a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh index 913963c7318..f417f0146c2 100755 --- a/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh +++ b/dbms/tests/queries/0_stateless/00746_sql_fuzzy.sh @@ -13,7 +13,7 @@ $CLICKHOUSE_CLIENT -q "select name from system.table_functions format TSV;" > $S # if you want long run use: env SQL_FUZZY_RUNS=100000 clickhouse-test sql_fuzzy for SQL_FUZZY_RUN in $(seq ${SQL_FUZZY_RUNS:=10}); do - env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT -n --ignore-error >/dev/null 2>&1 + env SQL_FUZZY_RUN=$SQL_FUZZY_RUN $CURDIR/00746_sql_fuzzy.pl | $CLICKHOUSE_CLIENT --max_execution_time 10 -n --ignore-error >/dev/null 2>&1 if [[ `$CLICKHOUSE_CLIENT -q "SELECT 'Still alive'"` != 'Still alive' ]]; then break fi diff --git a/dbms/tests/queries/0_stateless/00818_inner_join_bug_3567.reference b/dbms/tests/queries/0_stateless/00818_inner_join_bug_3567.reference index 4c5e10c19b0..7967cf7837e 100644 --- a/dbms/tests/queries/0_stateless/00818_inner_join_bug_3567.reference +++ b/dbms/tests/queries/0_stateless/00818_inner_join_bug_3567.reference @@ -1,5 +1,5 @@ -a 2018-01-01 00:00:00 0000-00-00 00:00:00 -b 2018-01-01 00:00:00 b 2018-01-01 00:00:00 -c 2018-01-01 00:00:00 c 2018-01-01 00:00:00 -b 2018-01-01 00:00:00 b 2018-01-01 00:00:00 -c 2018-01-01 00:00:00 c 2018-01-01 00:00:00 +a 2018-01-01 00:00:00 0000-00-00 00:00:00 +b 2018-01-01 00:00:00 b b 2018-01-01 00:00:00 +c 2018-01-01 00:00:00 c c 2018-01-01 00:00:00 +b 2018-01-01 00:00:00 b b 2018-01-01 00:00:00 +c 2018-01-01 00:00:00 c c 2018-01-01 00:00:00 diff --git a/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh b/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh index 92478119fe0..9d31234c877 100755 --- a/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh +++ b/dbms/tests/queries/0_stateless/00834_hints_for_type_function_typos.sh @@ -21,8 +21,8 @@ $CLICKHOUSE_CLIENT -q "select multisearchallposicionutf7casesensitive('abc');" 2 $CLICKHOUSE_CLIENT -q "select multiSearchAllposicionutf7sensitive('abc');" 2>&1 | grep "Maybe you meant: \['multiSearchAllPositionsCaseInsensitive','multiSearchAnyCaseInsensitive'\]." &>/dev/null; $CLICKHOUSE_CLIENT -q "select multiSearchAllPosicionSensitiveUTF8('abc');" 2>&1 | grep "Maybe you meant: \['multiSearchAnyCaseInsensitiveUTF8','multiSearchAllPositionsCaseInsensitiveUTF8'\]." &>/dev/null; -$CLICKHOUSE_CLIENT -q "select * FROM numberss(10);" 2>&1 | grep "Maybe you meant: \['numbers'\]." &>/dev/null -$CLICKHOUSE_CLIENT -q "select * FROM anothernumbers(10);" 2>&1 | grep -v "Maybe you meant: \['numbers'\]." &>/dev/null +$CLICKHOUSE_CLIENT -q "select * FROM numberss(10);" 2>&1 | grep "Maybe you meant: \['numbers'\,'numbers_mt'\]." &>/dev/null +$CLICKHOUSE_CLIENT -q "select * FROM anothernumbers(10);" 2>&1 | grep -v "Maybe you meant: \['numbers'\,'numbers_mt'\]." &>/dev/null $CLICKHOUSE_CLIENT -q "select * FROM mynumbers(10);" 2>&1 | grep "Maybe you meant: \['numbers'\]." &>/dev/null $CLICKHOUSE_CLIENT -q "CREATE TABLE stored_aggregates (d Date, Uniq AggregateFunction(uniq, UInt64)) ENGINE = MergeTre(d, d, 8192);" 2>&1 | grep "Maybe you meant: \['MergeTree'\]." &>/dev/null diff --git a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference index 4e65c4e0d23..e3038e03530 100644 --- a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference +++ b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.reference @@ -1,2 +1,2 @@ -10000000 -10000000 +5000000 +5000000 diff --git a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql index fc355bda462..0c8b8874293 100644 --- a/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql +++ b/dbms/tests/queries/0_stateless/00899_long_attach_memory_limit.sql @@ -1,9 +1,9 @@ DROP TABLE IF EXISTS index_memory; CREATE TABLE index_memory (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1; -INSERT INTO index_memory SELECT * FROM system.numbers LIMIT 10000000; +INSERT INTO index_memory SELECT * FROM system.numbers LIMIT 5000000; SELECT count() FROM index_memory; DETACH TABLE index_memory; -SET max_memory_usage = 79000000; +SET max_memory_usage = 39000000; ATTACH TABLE index_memory; SELECT count() FROM index_memory; DROP TABLE index_memory; diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py new file mode 100755 index 00000000000..b7fc3f4e3a6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv EVENTS') + client1.expect('1.*' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect('2.*' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect('3.*' + end_of_block) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference b/dbms/tests/queries/0_stateless/00960_live_view_watch_events_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference new file mode 100644 index 00000000000..6fbbedf1b21 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.reference @@ -0,0 +1,3 @@ +0 1 +6 2 +21 3 diff --git a/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql new file mode 100644 index 00000000000..c3e2ab8d102 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00961_temporary_live_view_watch.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py new file mode 100755 index 00000000000..f27b1213c70 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.py @@ -0,0 +1,42 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00962_temporary_live_view_watch_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled new file mode 100755 index 00000000000..df627c84e49 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.py.disabled @@ -0,0 +1,49 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send('DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET temporary_live_view_timeout=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client2.expect(prompt) + client1.expect(r'6.*2' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client2.expect(prompt) + client1.expect(r'21.*3' + end_of_block) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('SELECT sleep(1)') + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect('Table test.lv doesn\'t exist') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference b/dbms/tests/queries/0_stateless/00963_temporary_live_view_watch_live_timeout.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py new file mode 100755 index 00000000000..5664c0e6c6d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.py @@ -0,0 +1,44 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET live_view_heartbeat_interval=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv EVENTS') + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect('2.*' + end_of_block) + client1.expect('Progress: 2.00 rows.*\)') + # wait for heartbeat + client1.expect('Progress: 2.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00964_live_view_watch_events_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py new file mode 100755 index 00000000000..03e22175dff --- /dev/null +++ b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.py @@ -0,0 +1,45 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('SET live_view_heartbeat_interval=1') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client1.expect('Progress: 2.00 rows.*\)') + # wait for heartbeat + client1.expect('Progress: 2.00 rows.*\)') + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference b/dbms/tests/queries/0_stateless/00965_live_view_watch_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py new file mode 100755 index 00000000000..bb9d6152200 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.py @@ -0,0 +1,37 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + + with http_client({'method':'GET', 'url': '/?query=WATCH%20test.lv%20EVENTS'}, name='client2>', log=log) as client2: + client2.expect('.*1\n') + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + client2.expect('.*2\n') + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference b/dbms/tests/queries/0_stateless/00966_live_view_watch_events_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py new file mode 100755 index 00000000000..d3439431eb3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.py @@ -0,0 +1,37 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + + with http_client({'method':'GET', 'url':'/?query=WATCH%20test.lv'}, name='client2>', log=log) as client2: + client2.expect('.*0\t1\n') + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + client2.expect('.*6\t2\n') + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00967_live_view_watch_http.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference new file mode 100644 index 00000000000..5ae423d90d1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.reference @@ -0,0 +1,4 @@ +{"row":{"a":1}} +{"row":{"a":2}} +{"row":{"a":3}} +{"progress":{"read_rows":"3","read_bytes":"36","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql new file mode 100644 index 00000000000..8c6f4197d54 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00968_live_view_select_format_jsoneachrowwithprogress.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT * FROM test.lv FORMAT JSONEachRowWithProgress; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference new file mode 100644 index 00000000000..287a1ced92d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.reference @@ -0,0 +1,6 @@ +{"row":{"sum(a)":"0","_version":"1"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +{"row":{"sum(a)":"6","_version":"2"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} +{"row":{"sum(a)":"21","_version":"3"}} +{"progress":{"read_rows":"1","read_bytes":"16","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}} diff --git a/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql new file mode 100644 index 00000000000..725a4ad00ed --- /dev/null +++ b/dbms/tests/queries/0_stateless/00969_live_view_watch_format_jsoneachrowwithprogress.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0 FORMAT JSONEachRowWithProgress; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py new file mode 100755 index 00000000000..63628c4a76f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.py @@ -0,0 +1,43 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + + with http_client({'method':'GET', 'url': '/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20EVENTS%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}\n', escape=True) + client2.expect('{"row":{"version":"1"}', escape=True) + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + # heartbeat is provided by progress message + client2.expect('{"progress":{"read_rows":"1","read_bytes":"8","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"}}', escape=True) + + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + + client2.expect('{"row":{"version":"2"}}\n', escape=True) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference b/dbms/tests/queries/0_stateless/00970_live_view_watch_events_http_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py new file mode 100755 index 00000000000..7bdb47b7caa --- /dev/null +++ b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.py @@ -0,0 +1,43 @@ +#!/usr/bin/env python +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block +from httpclient import client as http_client + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1: + client1.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + + with http_client({'method':'GET', 'url':'/?live_view_heartbeat_interval=1&query=WATCH%20test.lv%20FORMAT%20JSONEachRowWithProgress'}, name='client2>', log=log) as client2: + client2.expect('"progress".*',) + client2.expect('{"row":{"sum(a)":"0","_version":"1"}}\n', escape=True) + client2.expect('"progress".*\n') + # heartbeat is provided by progress message + client2.expect('"progress".*\n') + + client1.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(prompt) + + client2.expect('"progress".*"read_rows":"2".*\n') + client2.expect('{"row":{"sum(a)":"6","_version":"2"}}\n', escape=True) + + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference b/dbms/tests/queries/0_stateless/00971_live_view_watch_http_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql new file mode 100644 index 00000000000..661080b577b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00972_live_view_select_1.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test.lv; + +CREATE LIVE VIEW test.lv AS SELECT 1; + +SELECT * FROM test.lv; + +DROP TABLE test.lv; diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.reference b/dbms/tests/queries/0_stateless/00973_live_view_select.reference new file mode 100644 index 00000000000..75236c0daf7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.reference @@ -0,0 +1,4 @@ +6 1 +6 1 +12 2 +12 2 diff --git a/dbms/tests/queries/0_stateless/00973_live_view_select.sql b/dbms/tests/queries/0_stateless/00973_live_view_select.sql new file mode 100644 index 00000000000..ff4a45ffcc1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00973_live_view_select.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; +SELECT *,_version FROM test.lv; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; +SELECT *,_version FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference new file mode 100644 index 00000000000..6d50f0e9c3a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.reference @@ -0,0 +1,2 @@ +6 +21 diff --git a/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql new file mode 100644 index 00000000000..3c11f855c9d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_live_view_select_with_aggregation.sql @@ -0,0 +1,16 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT sum(a) FROM test.lv; + +INSERT INTO test.mt VALUES (4),(5),(6); + +SELECT sum(a) FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.reference b/dbms/tests/queries/0_stateless/00975_live_view_create.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00975_live_view_create.sql b/dbms/tests/queries/0_stateless/00975_live_view_create.sql new file mode 100644 index 00000000000..1c929b15b00 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_live_view_create.sql @@ -0,0 +1,7 @@ +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference b/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference new file mode 100644 index 00000000000..453bd800469 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_live_view_select_version.reference @@ -0,0 +1,3 @@ +1 1 +2 1 +3 1 diff --git a/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql new file mode 100644 index 00000000000..5f3ab1f7546 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00976_live_view_select_version.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT * FROM test.mt; + +INSERT INTO test.mt VALUES (1),(2),(3); + +SELECT *,_version FROM test.lv; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference new file mode 100644 index 00000000000..01e79c32a8c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.reference @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql new file mode 100644 index 00000000000..a3b84e8d4c1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00977_live_view_watch_events.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv EVENTS LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv EVENTS LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv EVENTS LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.reference b/dbms/tests/queries/0_stateless/00978_live_view_watch.reference new file mode 100644 index 00000000000..6fbbedf1b21 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00978_live_view_watch.reference @@ -0,0 +1,3 @@ +0 1 +6 2 +21 3 diff --git a/dbms/tests/queries/0_stateless/00978_live_view_watch.sql b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql new file mode 100644 index 00000000000..abe4a6c32ae --- /dev/null +++ b/dbms/tests/queries/0_stateless/00978_live_view_watch.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (1),(2),(3); + +WATCH test.lv LIMIT 0; + +INSERT INTO test.mt VALUES (4),(5),(6); + +WATCH test.lv LIMIT 0; + +DROP TABLE test.lv; +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py new file mode 100755 index 00000000000..948e4c93662 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.py @@ -0,0 +1,48 @@ +#!/usr/bin/env python +import os +import sys +import signal + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from client import client, prompt, end_of_block + +log = None +# uncomment the line below for debugging +#log=sys.stdout + +with client(name='client1>', log=log) as client1, client(name='client2>', log=log) as client2: + client1.expect(prompt) + client2.expect(prompt) + + client1.send('DROP TABLE IF EXISTS test.lv') + client1.expect(prompt) + client1.send(' DROP TABLE IF EXISTS test.mt') + client1.expect(prompt) + client1.send('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()') + client1.expect(prompt) + client1.send('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt') + client1.expect(prompt) + client1.send('WATCH test.lv') + client1.expect(r'0.*1' + end_of_block) + client2.send('INSERT INTO test.mt VALUES (1),(2),(3)') + client1.expect(r'6.*2' + end_of_block) + client2.expect(prompt) + client2.send('INSERT INTO test.mt VALUES (4),(5),(6)') + client1.expect(r'21.*3' + end_of_block) + client2.expect(prompt) + for i in range(1,129): + client2.send('INSERT INTO test.mt VALUES (1)') + client1.expect(r'%d.*%d' % (21+i, 3+i) + end_of_block) + client2.expect(prompt) + # send Ctrl-C + client1.send('\x03', eol='') + match = client1.expect('(%s)|([#\$] )' % prompt) + if match.groups()[1]: + client1.send(client1.command) + client1.expect(prompt) + client1.send('DROP TABLE test.lv') + client1.expect(prompt) + client1.send('DROP TABLE test.mt') + client1.expect(prompt) diff --git a/dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00979_live_view_watch_live.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference new file mode 100644 index 00000000000..7f9fcbb2e9c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.reference @@ -0,0 +1,3 @@ +temporary_live_view_timeout 5 +live_view_heartbeat_interval 15 +0 diff --git a/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql new file mode 100644 index 00000000000..8cd6ee06ace --- /dev/null +++ b/dbms/tests/queries/0_stateless/00980_create_temporary_live_view.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test.lv; +DROP TABLE IF EXISTS test.mt; + +SELECT name, value from system.settings WHERE name = 'temporary_live_view_timeout'; +SELECT name, value from system.settings WHERE name = 'live_view_heartbeat_interval'; + +SET temporary_live_view_timeout=1; +CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple(); +CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt; + +SHOW TABLES LIKE 'lv'; +SELECT sleep(2); +SHOW TABLES LIKE 'lv'; + +DROP TABLE test.mt; diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python new file mode 100644 index 00000000000..782671cdfaf --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.python @@ -0,0 +1,81 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + line = q.get() + print(line) + assert (line == '21\t3') + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + print(line) + assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference new file mode 100644 index 00000000000..1e94cdade41 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.reference @@ -0,0 +1,7 @@ +0 1 +0 1 +6 2 +6 2 +21 3 +21 3 +None diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled new file mode 100755 index 00000000000..10e4e98b2e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_event_live.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_live_view_watch_event_live.python diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python new file mode 100755 index 00000000000..938547ca0cb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.python @@ -0,0 +1,63 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_http_query(query): + cmd = list(CLICKHOUSE_CURL.split()) # list(['curl', '-sSN', '--max-time', '10']) + cmd += ['-sSN', CLICKHOUSE_URL, '-d', query] + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def read_lines_and_push_to_queue(pipe, queue): + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + pipe = send_http_query('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(pipe, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference new file mode 100644 index 00000000000..489457d751b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.reference @@ -0,0 +1,4 @@ +0 1 +0 1 +6 2 +6 2 diff --git a/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled new file mode 100755 index 00000000000..88cce77f595 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_live_view_watch_http.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_live_view_watch_http.python diff --git a/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.reference b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh.disabled b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh.disabled new file mode 100755 index 00000000000..f8cd9fd7b36 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_system_parts_race_condition.sh.disabled @@ -0,0 +1,76 @@ +#!/usr/bin/env bash + +# This test is disabled because it triggers internal assert in Thread Sanitizer. +# Thread Sanitizer does not support for more than 64 mutexes to be locked in a single thread. +# https://github.com/google/sanitizers/issues/950 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table" +$CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = MergeTree ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1" + +function thread1() +{ + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done +} + +function thread2() +{ + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done +} + +function thread3() +{ + while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done +} + +function thread4() +{ + while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table FINAL"; done +} + +function thread5() +{ + while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table DELETE WHERE rand() % 2 = 1"; done +} + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; + +TIMEOUT=30 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + + +wait + +$CLICKHOUSE_CLIENT -q "DROP TABLE alter_table" diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python new file mode 100644 index 00000000000..70063adc6e3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.python @@ -0,0 +1,83 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query, '--live_view_heartbeat_interval=1', '--progress'] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + # print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + # print(line) + assert (line.endswith('0\t1')) + assert ('Progress: 0.00 rows' in line) + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + assert (line.endswith('6\t2')) + assert ('Progress: 1.00 rows' in line) + + # send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + # line = q.get() + # print(line) + # assert (line.endswith('6\t2')) + # assert ('Progress: 1.00 rows' in line) + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + # print(line) + # assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled new file mode 100755 index 00000000000..f7aa13d52b3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_events_heartbeat.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_temporary_live_view_watch_events_heartbeat.python diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python new file mode 100644 index 00000000000..d290018a02c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.python @@ -0,0 +1,81 @@ +#!/usr/bin/env python + +import subprocess +import threading +import Queue as queue +import os +import sys +import signal + + +CLICKHOUSE_CLIENT = os.environ.get('CLICKHOUSE_CLIENT') +CLICKHOUSE_CURL = os.environ.get('CLICKHOUSE_CURL') +CLICKHOUSE_URL = os.environ.get('CLICKHOUSE_URL') + + +def send_query(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT).stdout + + +def send_query_in_process_group(query): + cmd = list(CLICKHOUSE_CLIENT.split()) + cmd += ['--query', query] + # print(cmd) + return subprocess.Popen(cmd, stdout=subprocess.PIPE, stderr=subprocess.STDOUT, preexec_fn=os.setsid) + + +def read_lines_and_push_to_queue(pipe, queue): + try: + for line in iter(pipe.readline, ''): + line = line.strip() + print(line) + sys.stdout.flush() + queue.put(line) + except KeyboardInterrupt: + pass + + queue.put(None) + + +def test(): + send_query('DROP TABLE IF EXISTS test.lv').read() + send_query('DROP TABLE IF EXISTS test.mt').read() + send_query('CREATE TABLE test.mt (a Int32) Engine=MergeTree order by tuple()').read() + send_query('CREATE TEMPORARY LIVE VIEW test.lv AS SELECT sum(a) FROM test.mt').read() + + q = queue.Queue() + p = send_query_in_process_group('WATCH test.lv') + thread = threading.Thread(target=read_lines_and_push_to_queue, args=(p.stdout, q)) + thread.start() + + line = q.get() + print(line) + assert (line == '0\t1') + + send_query('INSERT INTO test.mt VALUES (1),(2),(3)').read() + line = q.get() + print(line) + assert (line == '6\t2') + + send_query('INSERT INTO test.mt VALUES (4),(5),(6)').read() + line = q.get() + print(line) + assert (line == '21\t3') + + # Send Ctrl+C to client. + os.killpg(os.getpgid(p.pid), signal.SIGINT) + # This insert shouldn't affect lv. + send_query('INSERT INTO test.mt VALUES (7),(8),(9)').read() + line = q.get() + print(line) + assert (line is None) + + send_query('DROP TABLE if exists test.lv').read() + send_query('DROP TABLE if exists test.lv').read() + + thread.join() + +test() diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference new file mode 100644 index 00000000000..1e94cdade41 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.reference @@ -0,0 +1,7 @@ +0 1 +0 1 +6 2 +6 2 +21 3 +21 3 +None diff --git a/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled new file mode 100755 index 00000000000..4d01d1c3a8e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00991_temporary_live_view_watch_live.sh.disabled @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +python $CURDIR/00991_temporary_live_view_watch_live.python diff --git a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.reference b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh new file mode 100755 index 00000000000..99c9e8774fe --- /dev/null +++ b/dbms/tests/queries/0_stateless/00992_system_parts_race_condition_zookeeper.sh @@ -0,0 +1,74 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table" +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table2" +$CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/alter_table', 'r1') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1" +$CLICKHOUSE_CLIENT -q "CREATE TABLE alter_table2 (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/alter_table', 'r2') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1" + +function thread1() +{ + while true; do $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; done +} + +function thread2() +{ + while true; do $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table ADD COLUMN h String; ALTER TABLE alter_table MODIFY COLUMN h UInt64; ALTER TABLE alter_table DROP COLUMN h;"; done +} + +function thread3() +{ + while true; do $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; done +} + +function thread4() +{ + while true; do $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table FINAL"; done +} + +function thread5() +{ + while true; do $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table DELETE WHERE rand() % 2 = 1"; done +} + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; + +TIMEOUT=10 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & + +wait + +$CLICKHOUSE_CLIENT -q "DROP TABLE alter_table" +$CLICKHOUSE_CLIENT -q "DROP TABLE alter_table2" diff --git a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.reference b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh new file mode 100755 index 00000000000..20f848155bc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00993_system_parts_race_condition_drop.sh @@ -0,0 +1,100 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +function thread1() +{ + while true; do + $CLICKHOUSE_CLIENT --query "SELECT * FROM system.parts FORMAT Null"; + done +} + +function thread2() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -n --query "ALTER TABLE alter_table_$REPLICA ADD COLUMN h String; ALTER TABLE alter_table_$REPLICA MODIFY COLUMN h UInt64; ALTER TABLE alter_table_$REPLICA DROP COLUMN h;"; + done +} + +function thread3() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "INSERT INTO alter_table_$REPLICA SELECT rand(1), rand(2), 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(100000)"; + done +} + +function thread4() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "OPTIMIZE TABLE alter_table_$REPLICA FINAL"; + sleep 0.$RANDOM; + done +} + +function thread5() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -q "ALTER TABLE alter_table_$REPLICA DELETE WHERE rand() % 2 = 1"; + sleep 0.$RANDOM; + done +} + +function thread6() +{ + while true; do + REPLICA=$(($RANDOM % 10)) + $CLICKHOUSE_CLIENT -n -q "DROP TABLE IF EXISTS alter_table_$REPLICA; + CREATE TABLE alter_table_$REPLICA (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) ENGINE = ReplicatedMergeTree('/clickhouse/tables/alter_table', 'r_$REPLICA') ORDER BY a PARTITION BY b % 10 SETTINGS old_parts_lifetime = 1;"; + sleep 0.$RANDOM; + done +} + + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; +export -f thread5; +export -f thread6; + +TIMEOUT=30 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & +timeout $TIMEOUT bash -c thread5 2> /dev/null & +timeout $TIMEOUT bash -c thread6 2> /dev/null & + +wait + +for i in {0..9}; do $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS alter_table_$i"; done diff --git a/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference new file mode 100644 index 00000000000..062dd5ddd57 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.reference @@ -0,0 +1,3 @@ +0 9999999 49999995000000 +10000000 14999999 62499997500000 +10000000 14999999 62499997500000 diff --git a/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql new file mode 100644 index 00000000000..f2bbbb438dd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00994_table_function_numbers_mt.sql @@ -0,0 +1,3 @@ +SELECT min(number), max(number), sum(number) FROM numbers_mt(10000000); +SELECT min(number), max(number), sum(number) FROM numbers(10000000, 5000000); +SELECT min(number), max(number), sum(number) FROM numbers_mt(10000000, 5000000); diff --git a/dbms/tests/queries/0_stateless/helpers/client.py b/dbms/tests/queries/0_stateless/helpers/client.py new file mode 100644 index 00000000000..f3938d3bf63 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/client.py @@ -0,0 +1,36 @@ +import os +import sys +import time + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +sys.path.insert(0, os.path.join(CURDIR)) + +import uexpect + +prompt = ':\) ' +end_of_block = r'.*\r\n.*\r\n' + +class client(object): + def __init__(self, command=None, name='', log=None): + self.client = uexpect.spawn(['/bin/bash','--noediting']) + if command is None: + command = os.environ.get('CLICKHOUSE_BINARY', 'clickhouse') + '-client' + self.client.command = command + self.client.eol('\r') + self.client.logger(log, prefix=name) + self.client.timeout(20) + self.client.expect('[#\$] ', timeout=2) + self.client.send(command) + + def __enter__(self): + return self.client.__enter__() + + def __exit__(self, type, value, traceback): + self.client.reader['kill_event'].set() + # send Ctrl-C + self.client.send('\x03', eol='') + time.sleep(0.3) + self.client.send('quit', eol='\r') + self.client.send('\x03', eol='') + return self.client.__exit__(type, value, traceback) diff --git a/dbms/tests/queries/0_stateless/helpers/httpclient.py b/dbms/tests/queries/0_stateless/helpers/httpclient.py new file mode 100644 index 00000000000..a42fad2cbc3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/httpclient.py @@ -0,0 +1,14 @@ +import os +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) + +sys.path.insert(0, os.path.join(CURDIR)) + +import httpexpect + +def client(request, name='', log=None): + client = httpexpect.spawn({'host':'localhost','port':8123}, request) + client.logger(log, prefix=name) + client.timeout(20) + return client diff --git a/dbms/tests/queries/0_stateless/helpers/httpexpect.py b/dbms/tests/queries/0_stateless/helpers/httpexpect.py new file mode 100644 index 00000000000..e440dafce4e --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/httpexpect.py @@ -0,0 +1,73 @@ +# Copyright (c) 2019 Vitaliy Zakaznikov +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os +import sys +import httplib + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, CURDIR) + +import uexpect + +from threading import Thread, Event +from Queue import Queue, Empty + +class IO(uexpect.IO): + def __init__(self, connection, response, queue, reader): + self.connection = connection + self.response = response + super(IO, self).__init__(None, None, queue, reader) + + def write(self, data): + raise NotImplementedError + + def close(self, force=True): + self.reader['kill_event'].set() + self.connection.close() + if self._logger: + self._logger.write('\n') + self._logger.flush() + + +def reader(response, queue, kill_event): + while True: + try: + if kill_event.is_set(): + break + data = response.read(1) + queue.put(data) + except Exception, e: + if kill_event.is_set(): + break + raise + +def spawn(connection, request): + connection = httplib.HTTPConnection(**connection) + connection.request(**request) + response = connection.getresponse() + + queue = Queue() + reader_kill_event = Event() + thread = Thread(target=reader, args=(response, queue, reader_kill_event)) + thread.daemon = True + thread.start() + + return IO(connection, response, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) + +if __name__ == '__main__': + with http({'host':'localhost','port':8123},{'method':'GET', 'url':'?query=SELECT%201'}) as client: + client.logger(sys.stdout) + client.timeout(2) + print client.response.status, client.response.reason + client.expect('1\n') diff --git a/dbms/tests/queries/0_stateless/helpers/uexpect.py b/dbms/tests/queries/0_stateless/helpers/uexpect.py new file mode 100644 index 00000000000..f71b32a53e1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/helpers/uexpect.py @@ -0,0 +1,206 @@ +# Copyright (c) 2019 Vitaliy Zakaznikov +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +import os +import pty +import time +import sys +import re + +from threading import Thread, Event +from subprocess import Popen +from Queue import Queue, Empty + +class TimeoutError(Exception): + def __init__(self, timeout): + self.timeout = timeout + + def __str__(self): + return 'Timeout %.3fs' % float(self.timeout) + +class ExpectTimeoutError(Exception): + def __init__(self, pattern, timeout, buffer): + self.pattern = pattern + self.timeout = timeout + self.buffer = buffer + + def __str__(self): + s = 'Timeout %.3fs ' % float(self.timeout) + if self.pattern: + s += 'for %s ' % repr(self.pattern.pattern) + if self.buffer: + s += 'buffer %s ' % repr(self.buffer[:]) + s += 'or \'%s\'' % ','.join(['%x' % ord(c) for c in self.buffer[:]]) + return s + +class IO(object): + class EOF(object): + pass + + class Timeout(object): + pass + + EOF = EOF + TIMEOUT = Timeout + + class Logger(object): + def __init__(self, logger, prefix=''): + self._logger = logger + self._prefix = prefix + + def write(self, data): + self._logger.write(('\n' + data).replace('\n','\n' + self._prefix)) + + def flush(self): + self._logger.flush() + + def __init__(self, process, master, queue, reader): + self.process = process + self.master = master + self.queue = queue + self.buffer = None + self.before = None + self.after = None + self.match = None + self.pattern = None + self.reader = reader + self._timeout = None + self._logger = None + self._eol = '' + + def __enter__(self): + return self + + def __exit__(self, type, value, traceback): + self.close() + + def logger(self, logger=None, prefix=''): + if logger: + self._logger = self.Logger(logger, prefix=prefix) + return self._logger + + def timeout(self, timeout=None): + if timeout: + self._timeout = timeout + return self._timeout + + def eol(self, eol=None): + if eol: + self._eol = eol + return self._eol + + def close(self, force=True): + self.reader['kill_event'].set() + os.system('pkill -TERM -P %d' % self.process.pid) + if force: + self.process.kill() + else: + self.process.terminate() + os.close(self.master) + if self._logger: + self._logger.write('\n') + self._logger.flush() + + def send(self, data, eol=None): + if eol is None: + eol = self._eol + return self.write(data + eol) + + def write(self, data): + return os.write(self.master, data) + + def expect(self, pattern, timeout=None, escape=False): + self.match = None + self.before = None + self.after = None + if escape: + pattern = re.escape(pattern) + pattern = re.compile(pattern) + if timeout is None: + timeout = self._timeout + timeleft = timeout + while True: + start_time = time.time() + if self.buffer is not None: + self.match = pattern.search(self.buffer, 0) + if self.match is not None: + self.after = self.buffer[self.match.start():self.match.end()] + self.before = self.buffer[:self.match.start()] + self.buffer = self.buffer[self.match.end():] + break + if timeleft < 0: + break + try: + data = self.read(timeout=timeleft, raise_exception=True) + except TimeoutError: + if self._logger: + self._logger.write((self.buffer or '') + '\n') + self._logger.flush() + exception = ExpectTimeoutError(pattern, timeout, self.buffer) + self.buffer = None + raise exception + timeleft -= (time.time() - start_time) + if data: + self.buffer = (self.buffer + data) if self.buffer else data + if self._logger: + self._logger.write((self.before or '') + (self.after or '')) + self._logger.flush() + if self.match is None: + exception = ExpectTimeoutError(pattern, timeout, self.buffer) + self.buffer = None + raise exception + return self.match + + def read(self, timeout=0, raise_exception=False): + data = '' + timeleft = timeout + try: + while timeleft >= 0 : + start_time = time.time() + data += self.queue.get(timeout=timeleft) + if data: + break + timeleft -= (time.time() - start_time) + except Empty: + if data: + return data + if raise_exception: + raise TimeoutError(timeout) + pass + if not data and raise_exception: + raise TimeoutError(timeout) + + return data + +def spawn(command): + master, slave = pty.openpty() + process = Popen(command, preexec_fn=os.setsid, stdout=slave, stdin=slave, stderr=slave, bufsize=1) + os.close(slave) + + queue = Queue() + reader_kill_event = Event() + thread = Thread(target=reader, args=(process, master, queue, reader_kill_event)) + thread.daemon = True + thread.start() + + return IO(process, master, queue, reader={'thread':thread, 'kill_event':reader_kill_event}) + +def reader(process, out, queue, kill_event): + while True: + try: + data = os.read(out, 65536) + queue.put(data) + except: + if kill_event.is_set(): + break + raise diff --git a/docs/en/data_types/float.md b/docs/en/data_types/float.md index 7c731d9fbc8..29ff604bb02 100644 --- a/docs/en/data_types/float.md +++ b/docs/en/data_types/float.md @@ -27,7 +27,7 @@ SELECT 1 - 0.9 - Floating-point calculations might result in numbers such as infinity (`Inf`) and "not-a-number" (`NaN`). This should be taken into account when processing the results of calculations. - When parsing floating point numbers from text, the result might not be the nearest machine-representable number. -## NaN and Inf +## NaN and Inf {#data_type-float-nan-inf} In contrast to standard SQL, ClickHouse supports the following categories of floating-point numbers: diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md index 8f6308cd0ab..8cdbae86e5e 100644 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -37,9 +37,9 @@ You can also download and install packages manually from here: :ﺪﯿﻨﮐ ﺐﺼﻧ ﻭ ﯼﺮﯿﮔﺭﺎﺑ ﺎﺠﻨ + + Docker Image ﺯﺍ ### + +.ﺪﻨﻨﮐ ﯽﻣ ﻩﺩﺎﻔﺘﺳﺍ ﻞﺧﺍﺩ ﺭﺩ "deb" ﯽﻤﺳﺭ ﯼﺎﻫ ﻪﺘﺴﺑ ﺯﺍ ﺮﯾﻭﺎﺼﺗ ﻦﯾﺍ .ﺪﯿﻨﮐ ﻝﺎﺒﻧﺩ ﺍﺭ (/ht + + ### نصب از طریق Source برای Compile، دستورالعمل های فایل build.md را دنبال کنید: diff --git a/docs/ru/getting_started/index.md b/docs/ru/getting_started/index.md index 8091f297019..e3fb2ab0985 100644 --- a/docs/ru/getting_started/index.md +++ b/docs/ru/getting_started/index.md @@ -37,13 +37,25 @@ sudo apt-get install clickhouse-client clickhouse-server ### Из RPM пакетов -Яндекс не использует ClickHouse на поддерживающих `rpm` дистрибутивах Linux, а также `rpm` пакеты менее тщательно тестируются. Таким образом, использовать их стоит на свой страх и риск, но, тем не менее, многие другие компании успешно работают на них в production без каких-либо серьезных проблем. +Команда ClickHouse в Яндексе рекомендует использовать официальные предкомпилированные `rpm` пакеты для CentOS, RedHad и всех остальных дистрибутивов Linux, основанных на rpm. -Для CentOS, RHEL и Fedora возможны следующие варианты: +Сначала нужно подключить официальный репозиторий: +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` -* Пакеты из генерируются на основе официальных `deb` пакетов от Яндекса и содержат в точности тот же исполняемый файл. -* Пакеты из собираются независимой компанией Altinity, но широко используются без каких-либо нареканий. -* Либо можно использовать Docker (см. ниже). +Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). + +Then run these commands to actually install packages: +Для, собственно, установки пакетов необходимо выполнить следующие команды: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +Также есть возможность установить пакеты вручную, скачав отсюда: . ### Из Docker образа diff --git a/docs/zh/data_types/float.md b/docs/zh/data_types/float.md index 113e640e611..2bada8c2410 100644 --- a/docs/zh/data_types/float.md +++ b/docs/zh/data_types/float.md @@ -29,7 +29,7 @@ SELECT 1 - 0.9 - 当一行行阅读浮点数的时候,浮点数的结果可能不是机器最近显示的数值。 -## NaN and Inf +## NaN and Inf {#data_type-float-nan-inf} 与标准SQL相比,ClickHouse 支持以下类别的浮点数: diff --git a/docs/zh/getting_started/index.md b/docs/zh/getting_started/index.md index 20d3c8ff9b1..f51323ce7e8 100644 --- a/docs/zh/getting_started/index.md +++ b/docs/zh/getting_started/index.md @@ -43,6 +43,32 @@ ClickHouse包含访问控制配置,它们位于`users.xml`文件中(与'config 默认情况下,允许从任何地方使用默认的‘default’用户无密码的访问ClickHouse。参考‘user/default/networks’。 有关更多信息,请参考"Configuration files"部分。 +###来自RPM包 + +Yandex ClickHouse团队建议使用官方预编译的`rpm`软件包,用于CentOS,RedHat和所有其他基于rpm的Linux发行版。 + +首先,您需要添加官方存储库: + +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` + +如果您想使用最新版本,请将`stable`替换为`testing`(建议您在测试环境中使用)。 + +然后运行这些命令以实际安装包: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +您也可以从此处手动下载和安装软件包:。 + +###来自Docker + +要在Docker中运行ClickHouse,请遵循[Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/)上的指南。那些图像使用官方的`deb`包。 + ### 使用源码安装 具体编译方式可以参考build.md。 @@ -67,14 +93,6 @@ Server: dbms/programs/clickhouse-server 日志的路径可以在server config (src/dbms/programs/server/config.xml)中配置。 -### 其他的安装方法 - -Docker image: - -CentOS或RHEL安装包: - -Gentoo:`emerge clickhouse` - ## 启动 可以运行如下命令在后台启动服务: diff --git a/docs/zh/query_language/select.md b/docs/zh/query_language/select.md index 53716c1cfac..4c819db1507 100644 --- a/docs/zh/query_language/select.md +++ b/docs/zh/query_language/select.md @@ -612,7 +612,7 @@ WHERE于HAVING不同之处在于WHERE在聚合前(GROUP BY)执行,HAVING在聚 如果不存在聚合,则不能使用HAVING。 -### ORDER BY 子句 +### ORDER BY 子句 {#select-order-by} 如果存在ORDER BY 子句,则该子句中必须存在一个表达式列表,表达式列表中每一个表达式都可以分配一个DESC或ASC(排序的方向)。如果没有指明排序的方向,将假定以ASC的方式进行排序。其中ASC表示按照升序排序,DESC按照降序排序。示例:`ORDER BY Visits DESC, SearchPhrase` diff --git a/libs/libmysqlxx/include/mysqlxx/Types.h b/libs/libmysqlxx/include/mysqlxx/Types.h index 30abdeb9fd0..b5ed70916fa 100644 --- a/libs/libmysqlxx/include/mysqlxx/Types.h +++ b/libs/libmysqlxx/include/mysqlxx/Types.h @@ -3,6 +3,8 @@ #include #include +#ifndef _mysql_h + struct st_mysql; using MYSQL = st_mysql; @@ -14,7 +16,7 @@ using MYSQL_ROW = char**; struct st_mysql_field; using MYSQL_FIELD = st_mysql_field; - +#endif namespace mysqlxx { diff --git a/utils/list_backports.sh b/utils/list_backports.sh index 552b560d5c6..6eba3410c95 100755 --- a/utils/list_backports.sh +++ b/utils/list_backports.sh @@ -1,5 +1,21 @@ #!/bin/sh +# sudo apt install python3-pip +# sudo pip3 install gitpython termcolor + +# Go to GitHub. +# In top-right corner, click to your profile icon. +# Choose "Settings". +# Choose "Developer settings". +# Choose "Personal access tokens". +# Choose "Generate new token". + +# Don't check any checkboxes. + +# Run as: +# ./list_backports.sh --token your-token + + set -e SCRIPTPATH=$(readlink -f "$0") SCRIPTDIR=$(dirname "$SCRIPTPATH")