diff --git a/README.md b/README.md index 08be0b9ed07..8cf62a7139e 100644 --- a/README.md +++ b/README.md @@ -13,7 +13,6 @@ ClickHouse is an open-source column-oriented database management system that all * You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person. ## Upcoming Events -* [ClickHouse Meetup in Tokyo](https://clickhouse.connpass.com/event/147001/) on November 14. * [ClickHouse Meetup in Istanbul](https://www.eventbrite.com/e/clickhouse-meetup-istanbul-create-blazing-fast-experiences-w-clickhouse-tickets-73101120419) on November 19. * [ClickHouse Meetup in Ankara](https://www.eventbrite.com/e/clickhouse-meetup-ankara-create-blazing-fast-experiences-w-clickhouse-tickets-73100530655) on November 21. * [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23. diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 1d5f4af645b..510faed187b 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -76,7 +76,7 @@ elseif (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") endif() if (USE_DEBUG_HELPERS) - set (INCLUDE_DEBUG_HELPERS "-include ${ClickHouse_SOURCE_DIR}/libs/libcommon/include/common/iostream_debug_helpers.h") + set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/libs/libcommon/include -include ${ClickHouse_SOURCE_DIR}/dbms/src/Core/iostream_debug_helpers.h") set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") endif () diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index 212a6500afd..0d2c2344bf5 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1226,7 +1226,7 @@ private: /// Returns true if one should continue receiving packets. bool receiveAndProcessPacket() { - Connection::Packet packet = connection->receivePacket(); + Packet packet = connection->receivePacket(); switch (packet.type) { @@ -1274,7 +1274,7 @@ private: { while (true) { - Connection::Packet packet = connection->receivePacket(); + Packet packet = connection->receivePacket(); switch (packet.type) { @@ -1308,7 +1308,7 @@ private: { while (true) { - Connection::Packet packet = connection->receivePacket(); + Packet packet = connection->receivePacket(); switch (packet.type) { diff --git a/dbms/programs/client/Suggest.h b/dbms/programs/client/Suggest.h index 57895b38764..78cc8d94db0 100644 --- a/dbms/programs/client/Suggest.h +++ b/dbms/programs/client/Suggest.h @@ -113,7 +113,7 @@ private: while (true) { - Connection::Packet packet = connection.receivePacket(); + Packet packet = connection.receivePacket(); switch (packet.type) { case Protocol::Server::Data: diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index ab55cd3d6cf..a138d6ab8f4 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -35,7 +35,7 @@ void waitQuery(Connection & connection) if (!connection.poll(1000000)) continue; - Connection::Packet packet = connection.receivePacket(); + Packet packet = connection.receivePacket(); switch (packet.type) { case Protocol::Server::EndOfStream: @@ -120,7 +120,7 @@ bool PerformanceTest::checkPreconditions() const while (true) { - Connection::Packet packet = connection.receivePacket(); + Packet packet = connection.receivePacket(); if (packet.type == Protocol::Server::Data) { diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 6d50a99cd93..76ea69cc737 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -924,7 +924,9 @@ void TCPHandler::receiveQuery() /// Per query settings. Settings & settings = query_context->getSettingsRef(); - settings.deserialize(*in); + auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS + : SettingsBinaryFormat::OLD; + settings.deserialize(*in, settings_format); /// Sync timeouts on client and server during current query to avoid dangling queries on server /// NOTE: We use settings.send_timeout for the receive timeout and vice versa (change arguments ordering in TimeoutSetter), @@ -953,7 +955,9 @@ void TCPHandler::receiveUnexpectedQuery() skip_client_info.read(*in, client_revision); Settings & skip_settings = query_context->getSettingsRef(); - skip_settings.deserialize(*in); + auto settings_format = (client_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS + : SettingsBinaryFormat::OLD; + skip_settings.deserialize(*in, settings_format); readVarUInt(skip_uint_64, *in); readVarUInt(skip_uint_64, *in); diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 95dff73f870..09d5de83f1e 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -409,7 +409,11 @@ void Connection::sendQuery( /// Per query settings. if (settings) - settings->serialize(*out); + { + auto settings_format = (server_revision >= DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS) ? SettingsBinaryFormat::STRINGS + : SettingsBinaryFormat::OLD; + settings->serialize(*out, settings_format); + } else writeStringBinary("" /* empty string is a marker of the end of settings */, *out); @@ -612,7 +616,7 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) } -Connection::Packet Connection::receivePacket() +Packet Connection::receivePacket() { try { diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index 8b507a4172a..bb639c6388b 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -42,6 +42,21 @@ using ConnectionPtr = std::shared_ptr; using Connections = std::vector; +/// Packet that could be received from server. +struct Packet +{ + UInt64 type; + + Block block; + std::unique_ptr exception; + std::vector multistring_message; + Progress progress; + BlockStreamProfileInfo profile_info; + + Packet() : type(Protocol::Server::Hello) {} +}; + + /** Connection with database server, to use by client. * How to use - see Core/Protocol.h * (Implementation of server end - see Server/TCPHandler.h) @@ -87,20 +102,6 @@ public: } - /// Packet that could be received from server. - struct Packet - { - UInt64 type; - - Block block; - std::unique_ptr exception; - std::vector multistring_message; - Progress progress; - BlockStreamProfileInfo profile_info; - - Packet() : type(Protocol::Server::Hello) {} - }; - /// Change default database. Changes will take effect on next reconnect. void setDefaultDatabase(const String & database); diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index d7934924242..c8d3fa4dcce 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -138,10 +138,10 @@ void MultiplexedConnections::sendQuery( sent_query = true; } -Connection::Packet MultiplexedConnections::receivePacket() +Packet MultiplexedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); - Connection::Packet packet = receivePacketUnlocked(); + Packet packet = receivePacketUnlocked(); return packet; } @@ -177,19 +177,19 @@ void MultiplexedConnections::sendCancel() cancelled = true; } -Connection::Packet MultiplexedConnections::drain() +Packet MultiplexedConnections::drain() { std::lock_guard lock(cancel_mutex); if (!cancelled) throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR); - Connection::Packet res; + Packet res; res.type = Protocol::Server::EndOfStream; while (hasActiveConnections()) { - Connection::Packet packet = receivePacketUnlocked(); + Packet packet = receivePacketUnlocked(); switch (packet.type) { @@ -235,7 +235,7 @@ std::string MultiplexedConnections::dumpAddressesUnlocked() const return os.str(); } -Connection::Packet MultiplexedConnections::receivePacketUnlocked() +Packet MultiplexedConnections::receivePacketUnlocked() { if (!sent_query) throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR); @@ -247,7 +247,7 @@ Connection::Packet MultiplexedConnections::receivePacketUnlocked() if (current_connection == nullptr) throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA); - Connection::Packet packet = current_connection->receivePacket(); + Packet packet = current_connection->receivePacket(); switch (packet.type) { diff --git a/dbms/src/Client/MultiplexedConnections.h b/dbms/src/Client/MultiplexedConnections.h index b26c9569422..9d825adb227 100644 --- a/dbms/src/Client/MultiplexedConnections.h +++ b/dbms/src/Client/MultiplexedConnections.h @@ -42,7 +42,7 @@ public: bool with_pending_data = false); /// Get packet from any replica. - Connection::Packet receivePacket(); + Packet receivePacket(); /// Break all active connections. void disconnect(); @@ -54,7 +54,7 @@ public: * Returns EndOfStream if no exception has been received. Otherwise * returns the last received packet of type Exception. */ - Connection::Packet drain(); + Packet drain(); /// Get the replica addresses as a string. std::string dumpAddresses() const; @@ -69,7 +69,7 @@ public: private: /// Internal version of `receivePacket` function without locking. - Connection::Packet receivePacketUnlocked(); + Packet receivePacketUnlocked(); /// Internal version of `dumpAddresses` function without locking. std::string dumpAddressesUnlocked() const; diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index def8f675c25..b875e174243 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -430,11 +430,11 @@ public: template void insert(iterator it, It1 from_begin, It2 from_end) { - insertPrepare(from_begin, from_end); - size_t bytes_to_copy = this->byte_size(from_end - from_begin); size_t bytes_to_move = (end() - it) * sizeof(T); + insertPrepare(from_begin, from_end); + if (unlikely(bytes_to_move)) memcpy(this->c_end + bytes_to_copy - bytes_to_move, this->c_end - bytes_to_move, bytes_to_move); diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 2ba55fa07d0..47c612c6ddf 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include diff --git a/dbms/src/Common/tests/gtest_pod_array.cpp b/dbms/src/Common/tests/gtest_pod_array.cpp new file mode 100644 index 00000000000..7962bf39f07 --- /dev/null +++ b/dbms/src/Common/tests/gtest_pod_array.cpp @@ -0,0 +1,34 @@ +#include + +#include + +using namespace DB; + +TEST(Common, PODArray_Insert) +{ + std::string str = "test_string_abacaba"; + PODArray chars; + chars.insert(chars.end(), str.begin(), str.end()); + EXPECT_EQ(str, std::string(chars.data(), chars.size())); + + std::string insert_in_the_middle = "insert_in_the_middle"; + auto pos = str.size() / 2; + str.insert(str.begin() + pos, insert_in_the_middle.begin(), insert_in_the_middle.end()); + chars.insert(chars.begin() + pos, insert_in_the_middle.begin(), insert_in_the_middle.end()); + EXPECT_EQ(str, std::string(chars.data(), chars.size())); + + std::string insert_with_resize; + insert_with_resize.reserve(chars.capacity() * 2); + char cur_char = 'a'; + while (insert_with_resize.size() < insert_with_resize.capacity()) + { + insert_with_resize += cur_char; + if (cur_char == 'z') + cur_char = 'a'; + else + ++cur_char; + } + str.insert(str.begin(), insert_with_resize.begin(), insert_with_resize.end()); + chars.insert(chars.begin(), insert_with_resize.begin(), insert_with_resize.end()); + EXPECT_EQ(str, std::string(chars.data(), chars.size())); +} diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 1fe8afe9966..0bf9c5b9544 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -59,9 +59,11 @@ #define DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA 54410 #define DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE 54405 - #define DBMS_MIN_REVISION_WITH_CLIENT_WRITE_INFO 54420 +/// Mininum revision supporting SettingsBinaryFormat::STRINGS. +#define DBMS_MIN_REVISION_WITH_SETTINGS_SERIALIZED_AS_STRINGS 54429 + /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. #define DBMS_TCP_PROTOCOL_VERSION 54226 @@ -148,9 +150,9 @@ #define OPTIMIZE(x) #endif -/// This number is only used for distributed version compatible. -/// It could be any magic number. -#define DBMS_DISTRIBUTED_SENDS_MAGIC_NUMBER 0xCAFECABE +/// Marks that extra information is sent to a shard. It could be any magic numbers. +#define DBMS_DISTRIBUTED_SIGNATURE_EXTRA_INFO 0xCAFEDACEull +#define DBMS_DISTRIBUTED_SIGNATURE_SETTINGS_OLD_FORMAT 0xCAFECABEull #if !__has_include() # define ASAN_UNPOISON_MEMORY_REGION(a, b) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index e0d97fe62fb..310860d3130 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -1,6 +1,6 @@ #pragma once -#include "SettingsCommon.h" +#include #include @@ -35,219 +35,222 @@ struct Settings : public SettingsCollection /// http://en.cppreference.com/w/cpp/language/aggregate_initialization Settings() {} - /** List of settings: type, name, default value. + /** List of settings: type, name, default value, description, flags * * This looks rather unconvenient. It is done that way to avoid repeating settings in different places. * Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field, * but we are not going to do it, because settings is used everywhere as static struct fields. + * + * `flags` can be either 0 or IGNORABLE. + * A setting is "IGNORABLE" if it doesn't affects the results of the queries and can be ignored without exception. */ #define LIST_OF_SETTINGS(M) \ - M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.") \ - M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.") \ - M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading") \ - M(SettingUInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.") \ - M(SettingUInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.") \ - M(SettingUInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.") \ - M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.") \ - M(SettingMaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.") \ - M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.") \ - M(SettingUInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).") \ - M(SettingUInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)") \ - M(SettingUInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.") \ - M(SettingSeconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.") \ - M(SettingMilliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.") \ - M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "") \ - M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "") \ - M(SettingSeconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes") \ - M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \ - M(SettingMilliseconds, connection_pool_max_wait_ms, 0, "The wait time when connection pool is full.") \ - M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.") \ - M(SettingMilliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.") \ - M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.") \ - M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.") \ - M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.") \ - M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.") \ - M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The mininum size of part to upload during multipart upload to S3.") \ - M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.") \ - M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.") \ - M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.") \ - M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.") \ - M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.") \ + M(SettingUInt64, min_compress_block_size, 65536, "The actual size of the block to compress, if the uncompressed data less than max_compress_block_size is no less than this value and no less than the volume of data for one mark.", 0) \ + M(SettingUInt64, max_compress_block_size, 1048576, "The maximum size of blocks of uncompressed data before compressing for writing to a table.", 0) \ + M(SettingUInt64, max_block_size, DEFAULT_BLOCK_SIZE, "Maximum block size for reading", 0) \ + M(SettingUInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE, "The maximum block size for insertion, if we control the creation of blocks for insertion.", 0) \ + M(SettingUInt64, min_insert_block_size_rows, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to INSERT query to specified size in rows, if blocks are not big enough.", 0) \ + M(SettingUInt64, min_insert_block_size_bytes, (DEFAULT_INSERT_BLOCK_SIZE * 256), "Squash blocks passed to INSERT query to specified size in bytes, if blocks are not big enough.", 0) \ + M(SettingMaxThreads, max_threads, 0, "The maximum number of threads to execute the request. By default, it is determined automatically.", 0) \ + M(SettingMaxThreads, max_alter_threads, 0, "The maximum number of threads to execute the ALTER requests. By default, it is determined automatically.", 0) \ + M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE, "The maximum size of the buffer to read from the filesystem.", 0) \ + M(SettingUInt64, max_distributed_connections, 1024, "The maximum number of connections for distributed processing of one query (should be greater than max_threads).", 0) \ + M(SettingUInt64, max_query_size, 262144, "Which part of the query can be read into RAM for parsing (the remaining data for INSERT, if any, is read later)", 0) \ + M(SettingUInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ + M(SettingSeconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ + M(SettingMilliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.", 0) \ + M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ + M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ + M(SettingSeconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ + M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ + M(SettingMilliseconds, connection_pool_max_wait_ms, 0, "The wait time when connection pool is full.", 0) \ + M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \ + M(SettingMilliseconds, kafka_max_wait_ms, 5000, "The wait time for reading from Kafka before retry.", 0) \ + M(SettingUInt64, poll_interval, DBMS_DEFAULT_POLL_INTERVAL, "Block at the query wait loop on the server for the specified number of seconds.", 0) \ + M(SettingUInt64, idle_connection_timeout, 3600, "Close idle TCP connections after specified number of seconds.", 0) \ + M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \ + M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ + M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The mininum size of part to upload during multipart upload to S3.", 0) \ + M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", 0) \ + M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ + M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ + M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ + M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.", 0) \ \ - M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.") \ - M(SettingMilliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.") \ + M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ + M(SettingMilliseconds, distributed_directory_monitor_max_sleep_time_ms, 30000, "Maximum sleep time for StorageDistributed DirectoryMonitors, it limits exponential growth too.", 0) \ \ - M(SettingBool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.") \ + M(SettingBool, distributed_directory_monitor_batch_inserts, false, "Should StorageDistributed DirectoryMonitors try to batch individual inserts into bigger ones.", 0) \ \ - M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.") \ + M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.", 0) \ \ - M(SettingUInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.") \ - M(SettingUInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.") \ + M(SettingUInt64, replication_alter_partitions_sync, 1, "Wait for actions to manipulate the partitions. 0 - do not wait, 1 - wait for execution only of itself, 2 - wait for everyone.", 0) \ + M(SettingUInt64, replication_alter_columns_timeout, 60, "Wait for actions to change the table structure within the specified number of seconds. 0 - wait unlimited time.", 0) \ \ - M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.") \ + M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \ \ - M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.") \ - M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.") \ + M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = ‘any’ are present.", 0) \ + M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \ \ - M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.") \ - M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.") \ - M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.") \ - M(SettingUInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled") \ - M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.") \ - M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.") \ - M(SettingBool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.") \ - M(SettingUInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.") \ + M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ + M(SettingBool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ + M(SettingUInt64, min_count_to_compile, 3, "The number of structurally identical queries before they are compiled.", 0) \ + M(SettingUInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ + M(SettingUInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ + M(SettingUInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ + M(SettingBool, distributed_aggregation_memory_efficient, false, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ + M(SettingUInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ \ - M(SettingUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.") \ - M(SettingUInt64, parallel_replicas_count, 0, "") \ - M(SettingUInt64, parallel_replica_offset, 0, "") \ + M(SettingUInt64, max_parallel_replicas, 1, "The maximum number of replicas of each shard used when the query is executed. For consistency (to get different parts of the same partition), this option only works for the specified sampling key. The lag of the replicas is not controlled.", 0) \ + M(SettingUInt64, parallel_replicas_count, 0, "", 0) \ + M(SettingUInt64, parallel_replica_offset, 0, "", 0) \ \ - M(SettingBool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.") \ + M(SettingBool, skip_unavailable_shards, false, "If 1, ClickHouse silently skips unavailable shards and nodes unresolvable through DNS. Shard is marked as unavailable when none of the replicas can be reached.", 0) \ \ - M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.") \ - M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.") \ + M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \ + M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ \ - M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.") \ - M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.") \ - M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.") \ - M(SettingUInt64, merge_tree_min_bytes_for_seek, 0, "You can skip reading more than that number of bytes at the price of one seek per file.") \ - M(SettingUInt64, merge_tree_coarse_index_granularity, 8, "If the index segment can contain the required keys, divide it into as many parts and recursively check them.") \ - M(SettingUInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)") \ - M(SettingUInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)") \ + M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ + M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ + M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.", 0) \ + M(SettingUInt64, merge_tree_min_bytes_for_seek, 0, "You can skip reading more than that number of bytes at the price of one seek per file.", 0) \ + M(SettingUInt64, merge_tree_coarse_index_granularity, 8, "If the index segment can contain the required keys, divide it into as many parts and recursively check them.", 0) \ + M(SettingUInt64, merge_tree_max_rows_to_use_cache, (128 * 8192), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ + M(SettingUInt64, merge_tree_max_bytes_to_use_cache, (192 * 10 * 1024 * 1024), "The maximum number of rows per request, to use the cache of uncompressed data. If the request is large, the cache is not used. (For large queries not to flush out the cache.)", 0) \ \ - M(SettingBool, merge_tree_uniform_read_distribution, true, "Distribute read from MergeTree over threads evenly, ensuring stable average execution time of each thread within one read operation.") \ + M(SettingBool, merge_tree_uniform_read_distribution, true, "Distribute read from MergeTree over threads evenly, ensuring stable average execution time of each thread within one read operation.", 0) \ \ - M(SettingUInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine") \ + M(SettingUInt64, mysql_max_rows_to_insert, 65536, "The maximum number of rows in MySQL batch insertion of the MySQL storage engine", 0) \ \ - M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ") \ + M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ - M(SettingUInt64, min_bytes_to_use_direct_io, 0, "The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.") \ + M(SettingUInt64, min_bytes_to_use_direct_io, 0, "The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.", 0) \ \ - M(SettingBool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.") \ - M(SettingBool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.") \ + M(SettingBool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ + M(SettingBool, force_primary_key, 0, "Throw an exception if there is primary key in a table, and it is not used.", 0) \ \ - M(SettingUInt64, mark_cache_min_lifetime, 10000, "If the maximum size of mark_cache is exceeded, delete only records older than mark_cache_min_lifetime seconds.") \ + M(SettingUInt64, mark_cache_min_lifetime, 10000, "If the maximum size of mark_cache is exceeded, delete only records older than mark_cache_min_lifetime seconds.", 0) \ \ - M(SettingFloat, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.") \ - M(SettingFloat, max_streams_multiplier_for_merge_tables, 5, "Ask more streams when reading from Merge table. Streams will be spread across tables that Merge table will use. This allows more even distribution of work across threads and especially helpful when merged tables differ in size.") \ + M(SettingFloat, max_streams_to_max_threads_ratio, 1, "Allows you to use more sources than the number of threads - to more evenly distribute work across threads. It is assumed that this is a temporary solution, since it will be possible in the future to make the number of sources equal to the number of threads, but for each source to dynamically select available work for itself.", 0) \ + M(SettingFloat, max_streams_multiplier_for_merge_tables, 5, "Ask more streams when reading from Merge table. Streams will be spread across tables that Merge table will use. This allows more even distribution of work across threads and especially helpful when merged tables differ in size.", 0) \ \ - M(SettingString, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.") \ + M(SettingString, network_compression_method, "LZ4", "Allows you to select the method of data compression when writing.", 0) \ \ - M(SettingInt64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.") \ + M(SettingInt64, network_zstd_compression_level, 1, "Allows you to select the level of ZSTD compression.", 0) \ \ - M(SettingUInt64, priority, 0, "Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.") \ - M(SettingInt64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.") \ + M(SettingUInt64, priority, 0, "Priority of the query. 1 - the highest, higher value - lower priority; 0 - do not use priorities.", 0) \ + M(SettingInt64, os_thread_priority, 0, "If non zero - set corresponding 'nice' value for query processing threads. Can be used to adjust query priority for OS scheduler.", 0) \ \ - M(SettingBool, log_queries, 0, "Log requests and write the log to the system table.") \ + M(SettingBool, log_queries, 0, "Log requests and write the log to the system table.", 0) \ \ - M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.") \ + M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ \ - M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?") \ + M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", 0) \ \ - M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.") \ + M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ \ - M(SettingBool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed") \ + M(SettingBool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed", 0) \ \ - M(SettingUInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.") \ - M(SettingMilliseconds, insert_quorum_timeout, 600000, "") \ - M(SettingUInt64, select_sequential_consistency, 0, "For SELECT queries from the replicated table, throw an exception if the replica does not have a chunk written with the quorum; do not read the parts that have not yet been written with the quorum.") \ - M(SettingUInt64, table_function_remote_max_addresses, 1000, "The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function.") \ - M(SettingMilliseconds, read_backoff_min_latency_ms, 1000, "Setting to reduce the number of threads in case of slow reads. Pay attention only to reads that took at least that much time.") \ - M(SettingUInt64, read_backoff_max_throughput, 1048576, "Settings to reduce the number of threads in case of slow reads. Count events when the read bandwidth is less than that many bytes per second.") \ - M(SettingMilliseconds, read_backoff_min_interval_between_events_ms, 1000, "Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time.") \ - M(SettingUInt64, read_backoff_min_events, 2, "Settings to reduce the number of threads in case of slow reads. The number of events after which the number of threads will be reduced.") \ + M(SettingUInt64, insert_quorum, 0, "For INSERT queries in the replicated table, wait writing for the specified number of replicas and linearize the addition of the data. 0 - disabled.", 0) \ + M(SettingMilliseconds, insert_quorum_timeout, 600000, "", 0) \ + M(SettingUInt64, select_sequential_consistency, 0, "For SELECT queries from the replicated table, throw an exception if the replica does not have a chunk written with the quorum; do not read the parts that have not yet been written with the quorum.", 0) \ + M(SettingUInt64, table_function_remote_max_addresses, 1000, "The maximum number of different shards and the maximum number of replicas of one shard in the `remote` function.", 0) \ + M(SettingMilliseconds, read_backoff_min_latency_ms, 1000, "Setting to reduce the number of threads in case of slow reads. Pay attention only to reads that took at least that much time.", 0) \ + M(SettingUInt64, read_backoff_max_throughput, 1048576, "Settings to reduce the number of threads in case of slow reads. Count events when the read bandwidth is less than that many bytes per second.", 0) \ + M(SettingMilliseconds, read_backoff_min_interval_between_events_ms, 1000, "Settings to reduce the number of threads in case of slow reads. Do not pay attention to the event, if the previous one has passed less than a certain amount of time.", 0) \ + M(SettingUInt64, read_backoff_min_events, 2, "Settings to reduce the number of threads in case of slow reads. The number of events after which the number of threads will be reduced.", 0) \ \ - M(SettingFloat, memory_tracker_fault_probability, 0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.") \ + M(SettingFloat, memory_tracker_fault_probability, 0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.", 0) \ \ - M(SettingBool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.") \ - M(SettingInt64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.") \ + M(SettingBool, enable_http_compression, 0, "Compress the result if the client over HTTP said that it understands data compressed by gzip or deflate.", 0) \ + M(SettingInt64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.", 0) \ \ - M(SettingBool, http_native_compression_disable_checksumming_on_decompress, 0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.") \ + M(SettingBool, http_native_compression_disable_checksumming_on_decompress, 0, "If you uncompress the POST data from the client compressed by the native format, do not check the checksum.", 0) \ \ - M(SettingString, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)") \ + M(SettingString, count_distinct_implementation, "uniqExact", "What aggregate function to use for implementation of count(DISTINCT ...)", 0) \ \ - M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.") \ + M(SettingBool, output_format_write_statistics, true, "Write statistics about read rows, bytes, time elapsed in suitable output formats.", 0) \ \ - M(SettingBool, add_http_cors_header, false, "Write add http CORS header.") \ + M(SettingBool, add_http_cors_header, false, "Write add http CORS header.", 0) \ \ - M(SettingUInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.") \ + M(SettingUInt64, max_http_get_redirects, 0, "Max number of http GET redirects hops allowed. Make sure additional security measures are in place to prevent a malicious server to redirect your requests to unexpected services.", 0) \ \ - M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).") \ - M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.") \ - M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).") \ - M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).") \ - M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.") \ - M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable") \ + M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ + M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ + M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ + M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", 0) \ + M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ + M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ \ - M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.") \ - M(SettingBool, input_format_values_deduce_templates_of_expressions, false, "For Values format: if field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.") \ - M(SettingBool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.") \ + M(SettingBool, input_format_values_interpret_expressions, true, "For Values format: if field could not be parsed by streaming parser, run SQL parser and try to interpret it as SQL expression.", 0) \ + M(SettingBool, input_format_values_deduce_templates_of_expressions, false, "For Values format: if field could not be parsed by streaming parser, run SQL parser, deduce template of the SQL expression, try to parse all rows using template and then interpret expression for all rows.", 0) \ + M(SettingBool, input_format_values_accurate_types_of_literals, true, "For Values format: when parsing and interpreting expressions using template, check actual type of literal to avoid possible overflow and precision issues.", 0) \ \ - M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.") \ + M(SettingBool, output_format_json_quote_64bit_integers, true, "Controls quoting of 64-bit integers in JSON output format.", 0) \ \ - M(SettingBool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.") \ + M(SettingBool, output_format_json_quote_denormals, false, "Enables '+nan', '-nan', '+inf', '-inf' outputs in JSON output format.", 0) \ \ - M(SettingBool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.") \ + M(SettingBool, output_format_json_escape_forward_slashes, true, "Controls escaping forward slashes for string outputs in JSON output format. This is intended for compatibility with JavaScript. Don't confuse with backslashes that are always escaped.", 0) \ \ - M(SettingUInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.") \ - M(SettingUInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.") \ - M(SettingBool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats") \ - M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.") \ + M(SettingUInt64, output_format_pretty_max_rows, 10000, "Rows limit for Pretty formats.", 0) \ + M(SettingUInt64, output_format_pretty_max_column_pad_width, 250, "Maximum width to pad all values in a column in Pretty formats.", 0) \ + M(SettingBool, output_format_pretty_color, true, "Use ANSI escape sequences to paint colors in Pretty formats", 0) \ + M(SettingUInt64, output_format_parquet_row_group_size, 1000000, "Row group size in rows.", 0) \ \ - M(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.") \ + M(SettingBool, use_client_time_zone, false, "Use client timezone for interpreting DateTime string values, instead of adopting server timezone.", 0) \ \ - M(SettingBool, send_progress_in_http_headers, false, "Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.") \ + M(SettingBool, send_progress_in_http_headers, false, "Send progress notifications using X-ClickHouse-Progress headers. Some clients do not support high amount of HTTP headers (Python requests in particular), so it is disabled by default.", 0) \ \ - M(SettingUInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.") \ + M(SettingUInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \ \ - M(SettingBool, fsync_metadata, 1, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.") \ + M(SettingBool, fsync_metadata, 1, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ \ - M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.") \ - M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.") \ + M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ + M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ \ - M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.") \ + M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", 0) \ \ - M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.") \ - M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.") \ + M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ + M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", 0) \ \ - M(SettingUInt64, preferred_block_size_bytes, 1000000, "") \ + M(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \ \ - M(SettingUInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.") \ - M(SettingBool, fallback_to_stale_replicas_for_distributed_queries, 1, "Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.") \ - M(SettingUInt64, preferred_max_column_in_block_size_bytes, 0, "Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.") \ + M(SettingUInt64, max_replica_delay_for_distributed_queries, 300, "If set, distributed queries of Replicated tables will choose servers with replication delay in seconds less than the specified value (not inclusive). Zero means do not take delay into account.", 0) \ + M(SettingBool, fallback_to_stale_replicas_for_distributed_queries, 1, "Suppose max_replica_delay_for_distributed_queries is set and all replicas for the queried table are stale. If this setting is enabled, the query will be performed anyway, otherwise the error will be reported.", 0) \ + M(SettingUInt64, preferred_max_column_in_block_size_bytes, 0, "Limit on max column size in block while reading. Helps to decrease cache misses count. Should be close to L2 cache size.", 0) \ \ - M(SettingBool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.") \ - M(SettingUInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.") \ - M(SettingInt64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.") \ - M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \ - M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.") \ + M(SettingBool, insert_distributed_sync, false, "If setting is enabled, insert query into distributed waits until data will be sent to all nodes in cluster.", 0) \ + M(SettingUInt64, insert_distributed_timeout, 0, "Timeout for insert query into distributed. Setting is used only with insert_distributed_sync enabled. Zero value means no timeout.", 0) \ + M(SettingInt64, distributed_ddl_task_timeout, 180, "Timeout for DDL query responses from all hosts in cluster. If a ddl request has not been performed on all hosts, a response will contain a timeout error and a request will be executed in an async mode. Negative value means infinite.", 0) \ + M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ + M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ - M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \ - M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)") \ - M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)") \ - M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)") \ + M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)", 0) \ + M(SettingString, format_template_resultset, "", "Path to file which contains format string for result set (for Template format)", 0) \ + M(SettingString, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \ + M(SettingString, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \ \ - M(SettingString, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)") \ - M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)") \ - M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)") \ - M(SettingString, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)") \ - M(SettingString, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)") \ - M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)") \ - M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)") \ + M(SettingString, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_row_between_delimiter, "", "Delimiter between rows (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_result_before_delimiter, "", "Prefix before result set (for CustomSeparated format)", 0) \ + M(SettingString, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ \ - M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \ - M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \ - M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout") \ - M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout") \ - M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown") \ - M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.") \ - M(SettingBool, joined_subquery_requires_alias, false, "Force joined subqueries to have aliases for correct name qualification.") \ - M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \ - M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \ - M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.") \ - M(SettingUInt64, query_profiler_real_time_period_ns, 1000000000, "Highly experimental. Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.") \ - M(SettingUInt64, query_profiler_cpu_time_period_ns, 1000000000, "Highly experimental. Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.") \ + M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ + M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ + M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ + M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout", 0) \ + M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown", 0) \ + M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.", 0) \ + M(SettingBool, joined_subquery_requires_alias, false, "Force joined subqueries to have aliases for correct name qualification.", 0) \ + M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.", 0) \ + M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.", 0) \ + M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.", 0) \ + M(SettingUInt64, query_profiler_real_time_period_ns, 1000000000, "Highly experimental. Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ + M(SettingUInt64, query_profiler_cpu_time_period_ns, 1000000000, "Highly experimental. Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.", 0) \ \ \ /** Limits during query execution are part of the settings. \ @@ -257,135 +260,135 @@ struct Settings : public SettingsCollection * Almost all limits apply to each stream individually. \ */ \ \ - M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.") \ - M(SettingUInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.") \ - M(SettingOverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \ + M(SettingUInt64, max_bytes_to_read, 0, "Limit on read bytes (after decompression) from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it is only checked on a remote server.", 0) \ + M(SettingOverflowMode, read_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(SettingUInt64, max_rows_to_group_by, 0, "") \ - M(SettingOverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ - M(SettingUInt64, max_bytes_before_external_group_by, 0, "") \ + M(SettingUInt64, max_rows_to_group_by, 0, "", 0) \ + M(SettingOverflowModeGroupBy, group_by_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(SettingUInt64, max_bytes_before_external_group_by, 0, "", 0) \ \ - M(SettingUInt64, max_rows_to_sort, 0, "") \ - M(SettingUInt64, max_bytes_to_sort, 0, "") \ - M(SettingOverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ - M(SettingUInt64, max_bytes_before_external_sort, 0, "") \ - M(SettingUInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.") \ + M(SettingUInt64, max_rows_to_sort, 0, "", 0) \ + M(SettingUInt64, max_bytes_to_sort, 0, "", 0) \ + M(SettingOverflowMode, sort_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(SettingUInt64, max_bytes_before_external_sort, 0, "", 0) \ + M(SettingUInt64, max_bytes_before_remerge_sort, 1000000000, "In case of ORDER BY with LIMIT, when memory usage is higher than specified threshold, perform additional steps of merging blocks before final merge to keep just top LIMIT rows.", 0) \ \ - M(SettingUInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.") \ - M(SettingUInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.") \ - M(SettingOverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingUInt64, max_result_rows, 0, "Limit on result size in rows. Also checked for intermediate data sent from remote servers.", 0) \ + M(SettingUInt64, max_result_bytes, 0, "Limit on result size in bytes (uncompressed). Also checked for intermediate data sent from remote servers.", 0) \ + M(SettingOverflowMode, result_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ /* TODO: Check also when merging and finalizing aggregate functions. */ \ - M(SettingSeconds, max_execution_time, 0, "") \ - M(SettingOverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingSeconds, max_execution_time, 0, "", 0) \ + M(SettingOverflowMode, timeout_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(SettingUInt64, min_execution_speed, 0, "Minimum number of execution rows per second.") \ - M(SettingUInt64, max_execution_speed, 0, "Maximum number of execution rows per second.") \ - M(SettingUInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.") \ - M(SettingUInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.") \ - M(SettingSeconds, timeout_before_checking_execution_speed, 0, "Check that the speed is not too low after the specified time has elapsed.") \ + M(SettingUInt64, min_execution_speed, 0, "Minimum number of execution rows per second.", 0) \ + M(SettingUInt64, max_execution_speed, 0, "Maximum number of execution rows per second.", 0) \ + M(SettingUInt64, min_execution_speed_bytes, 0, "Minimum number of execution bytes per second.", 0) \ + M(SettingUInt64, max_execution_speed_bytes, 0, "Maximum number of execution bytes per second.", 0) \ + M(SettingSeconds, timeout_before_checking_execution_speed, 0, "Check that the speed is not too low after the specified time has elapsed.", 0) \ \ - M(SettingUInt64, max_columns_to_read, 0, "") \ - M(SettingUInt64, max_temporary_columns, 0, "") \ - M(SettingUInt64, max_temporary_non_const_columns, 0, "") \ + M(SettingUInt64, max_columns_to_read, 0, "", 0) \ + M(SettingUInt64, max_temporary_columns, 0, "", 0) \ + M(SettingUInt64, max_temporary_non_const_columns, 0, "", 0) \ \ - M(SettingUInt64, max_subquery_depth, 100, "") \ - M(SettingUInt64, max_pipeline_depth, 1000, "") \ - M(SettingUInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.") \ - M(SettingUInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.") \ - M(SettingUInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.") \ + M(SettingUInt64, max_subquery_depth, 100, "", 0) \ + M(SettingUInt64, max_pipeline_depth, 1000, "", 0) \ + M(SettingUInt64, max_ast_depth, 1000, "Maximum depth of query syntax tree. Checked after parsing.", 0) \ + M(SettingUInt64, max_ast_elements, 50000, "Maximum size of query syntax tree in number of nodes. Checked after parsing.", 0) \ + M(SettingUInt64, max_expanded_ast_elements, 500000, "Maximum size of query syntax tree in number of nodes after expansion of aliases and the asterisk.", 0) \ \ - M(SettingUInt64, readonly, 0, "0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.") \ + M(SettingUInt64, readonly, 0, "0 - everything is allowed. 1 - only read requests. 2 - only read requests, as well as changing settings, except for the 'readonly' setting.", 0) \ \ - M(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.") \ - M(SettingUInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.") \ - M(SettingOverflowMode, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingUInt64, max_rows_in_set, 0, "Maximum size of the set (in number of elements) resulting from the execution of the IN section.", 0) \ + M(SettingUInt64, max_bytes_in_set, 0, "Maximum size of the set (in bytes in memory) resulting from the execution of the IN section.", 0) \ + M(SettingOverflowMode, set_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).") \ - M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \ - M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ - M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.") \ - M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.") \ - M(SettingBool, partial_merge_join_optimizations, false, "Enable optimizations in partial merge join") \ - M(SettingUInt64, default_max_bytes_in_join, 100000000, "Maximum size of right-side table if limit's required but max_bytes_in_join is not set.") \ - M(SettingUInt64, partial_merge_join_rows_in_right_blocks, 10000, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.") \ - M(SettingUInt64, partial_merge_join_rows_in_left_blocks, 10000, "Group left-hand joining data in bigger blocks. Setting it to a bigger value increase JOIN performance and memory usage.") \ + M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \ + M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ + M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ + M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", 0) \ + M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.", 0) \ + M(SettingBool, partial_merge_join_optimizations, false, "Enable optimizations in partial merge join", 0) \ + M(SettingUInt64, default_max_bytes_in_join, 100000000, "Maximum size of right-side table if limit's required but max_bytes_in_join is not set.", 0) \ + M(SettingUInt64, partial_merge_join_rows_in_right_blocks, 10000, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ + M(SettingUInt64, partial_merge_join_rows_in_left_blocks, 10000, "Group left-hand joining data in bigger blocks. Setting it to a bigger value increase JOIN performance and memory usage.", 0) \ \ - M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ - M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ - M(SettingOverflowMode, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \ + M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.", 0) \ + M(SettingOverflowMode, transfer_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(SettingUInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.") \ - M(SettingUInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.") \ - M(SettingOverflowMode, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ + M(SettingUInt64, max_rows_in_distinct, 0, "Maximum number of elements during execution of DISTINCT.", 0) \ + M(SettingUInt64, max_bytes_in_distinct, 0, "Maximum total size of state (in uncompressed bytes) in memory for the execution of DISTINCT.", 0) \ + M(SettingOverflowMode, distinct_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ \ - M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.") \ - M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.") \ - M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.") \ + M(SettingUInt64, max_memory_usage, 0, "Maximum memory usage for processing of single query. Zero means unlimited.", 0) \ + M(SettingUInt64, max_memory_usage_for_user, 0, "Maximum memory usage for processing all concurrently running queries for the user. Zero means unlimited.", 0) \ + M(SettingUInt64, max_memory_usage_for_all_queries, 0, "Maximum memory usage for processing all concurrently running queries on the server. Zero means unlimited.", 0) \ \ - M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.") \ - M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.") \ - M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.")\ - M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.") \ - M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.") \ - M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.") \ - M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.") \ - M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N") \ + M(SettingUInt64, max_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for a query. Zero means unlimited.", 0) \ + M(SettingUInt64, max_network_bytes, 0, "The maximum number of bytes (compressed) to receive or transmit over the network for execution of the query.", 0) \ + M(SettingUInt64, max_network_bandwidth_for_user, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running user queries. Zero means unlimited.", 0)\ + M(SettingUInt64, max_network_bandwidth_for_all_users, 0, "The maximum speed of data exchange over the network in bytes per second for all concurrently running queries. Zero means unlimited.", 0) \ + M(SettingChar, format_csv_delimiter, ',', "The character to be considered as a delimiter in CSV data. If setting with a string, a string has to have a length of 1.", 0) \ + M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.", 0) \ + M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.", 0) \ + M(SettingBool, input_format_csv_unquoted_null_literal_as_null, false, "Consider unquoted NULL literal as \\N", 0) \ \ - M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ - M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ - M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ - M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \ - M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ - M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.") \ - M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.") \ + M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \ + M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \ + M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \ + M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \ + M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'", 0) \ + M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \ + M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \ \ - M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \ - M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \ - M(SettingBool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations") \ + M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \ + M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \ + M(SettingBool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \ \ - M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing") \ - M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.") \ - M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).") \ - M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.") \ - M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.") \ - M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \ - M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.") \ - M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.") \ - M(SettingBool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.") \ - M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.") \ - M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Emulate multiple joins using subselects") \ - M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Convert CROSS JOIN to INNER JOIN if possible") \ - M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.") \ - M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.") \ - M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ + M(SettingBool, prefer_localhost_replica, 1, "1 - always send query to local replica, if it exists. 0 - choose replica to send query between local and remote ones according to load_balancing", 0) \ + M(SettingUInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \ + M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).", 0) \ + M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.", 0) \ + M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.", 0) \ + M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.", 0) \ + M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.", 0) \ + M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \ + M(SettingBool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ + M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \ + M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Emulate multiple joins using subselects", 0) \ + M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Convert CROSS JOIN to INNER JOIN if possible", 0) \ + M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \ + M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \ + M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.", 0) \ \ - M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ + M(SettingBool, experimental_use_processors, false, "Use processors pipeline.", 0) \ \ - M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ - M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ - M(SettingBool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.") \ + M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ + M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ + M(SettingBool, allow_introspection_functions, false, "Allow functions for introspection of ELF and DWARF for query profiling. These functions are slow and may impose security considerations.", 0) \ \ - M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.") \ - M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value") \ - M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries") \ + M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.", 0) \ + M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value", 0) \ + M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries", 0) \ \ - M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.") \ - M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up increadible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.") \ + M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \ + M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up increadible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \ \ - M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.") \ - 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(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.") \ - M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.") \ + M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \ + M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \ + M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.", 0) \ + 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.", 0) \ + M(SettingUInt64, min_free_disk_space_for_temporary_data, 0, "The minimum disk space to keep while writing temporary data used in external sorting and aggregation.", 0) \ \ - M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.") \ - M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.") \ + M(SettingBool, enable_scalar_subquery_optimization, true, "If it is set to true, prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once.", 0) \ + M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \ \ /** 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(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13") \ + M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13", 0) \ + M(SettingBool, compile, false, "Whether query compilation is enabled. Will be removed after 2020-03-13", 0) \ DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) diff --git a/dbms/src/Core/SettingsCommon.cpp b/dbms/src/Core/SettingsCollection.cpp similarity index 78% rename from dbms/src/Core/SettingsCommon.cpp rename to dbms/src/Core/SettingsCollection.cpp index ddaa294eeb6..fa9d0df57da 100644 --- a/dbms/src/Core/SettingsCommon.cpp +++ b/dbms/src/Core/SettingsCollection.cpp @@ -1,17 +1,17 @@ -#include "SettingsCommon.h" +#include +#include #include #include #include +#include #include #include #include - namespace DB { - namespace ErrorCodes { extern const int TYPE_MISMATCH; @@ -90,8 +90,14 @@ void SettingNumber::set(const String & x) } template -void SettingNumber::serialize(WriteBuffer & buf) const +void SettingNumber::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { + if (format >= SettingsBinaryFormat::STRINGS) + { + writeStringBinary(toString(), buf); + return; + } + if constexpr (is_integral_v && is_unsigned_v) writeVarUInt(static_cast(value), buf); else if constexpr (is_integral_v && is_signed_v) @@ -99,13 +105,21 @@ void SettingNumber::serialize(WriteBuffer & buf) const else { static_assert(std::is_floating_point_v); - writeBinary(toString(), buf); + writeStringBinary(toString(), buf); } } template -void SettingNumber::deserialize(ReadBuffer & buf) +void SettingNumber::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) { + if (format >= SettingsBinaryFormat::STRINGS) + { + String x; + readStringBinary(x, buf); + set(x); + return; + } + if constexpr (is_integral_v && is_unsigned_v) { UInt64 x; @@ -122,7 +136,7 @@ void SettingNumber::deserialize(ReadBuffer & buf) { static_assert(std::is_floating_point_v); String x; - readBinary(x, buf); + readStringBinary(x, buf); set(x); } } @@ -167,13 +181,27 @@ void SettingMaxThreads::set(const String & x) set(parse(x)); } -void SettingMaxThreads::serialize(WriteBuffer & buf) const +void SettingMaxThreads::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { + if (format >= SettingsBinaryFormat::STRINGS) + { + writeStringBinary(is_auto ? "auto" : DB::toString(value), buf); + return; + } + writeVarUInt(is_auto ? 0 : value, buf); } -void SettingMaxThreads::deserialize(ReadBuffer & buf) +void SettingMaxThreads::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) { + if (format >= SettingsBinaryFormat::STRINGS) + { + String x; + readStringBinary(x, buf); + set(x); + return; + } + UInt64 x = 0; readVarUInt(x, buf); set(x); @@ -233,14 +261,28 @@ void SettingTimespan::set(const String & x) } template -void SettingTimespan::serialize(WriteBuffer & buf) const +void SettingTimespan::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { + if (format >= SettingsBinaryFormat::STRINGS) + { + writeStringBinary(toString(), buf); + return; + } + writeVarUInt(value.totalMicroseconds() / microseconds_per_io_unit, buf); } template -void SettingTimespan::deserialize(ReadBuffer & buf) +void SettingTimespan::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) { + if (format >= SettingsBinaryFormat::STRINGS) + { + String x; + readStringBinary(x, buf); + set(x); + return; + } + UInt64 x = 0; readVarUInt(x, buf); set(x); @@ -271,15 +313,15 @@ void SettingString::set(const Field & x) set(safeGet(x)); } -void SettingString::serialize(WriteBuffer & buf) const +void SettingString::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { - writeBinary(value, buf); + writeStringBinary(value, buf); } -void SettingString::deserialize(ReadBuffer & buf) +void SettingString::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; - readBinary(s, buf); + readStringBinary(s, buf); set(s); } @@ -314,30 +356,30 @@ void SettingChar::set(const Field & x) set(s); } -void SettingChar::serialize(WriteBuffer & buf) const +void SettingChar::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { - writeBinary(toString(), buf); + writeStringBinary(toString(), buf); } -void SettingChar::deserialize(ReadBuffer & buf) +void SettingChar::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; - readBinary(s, buf); + readStringBinary(s, buf); set(s); } template -void SettingEnum::serialize(WriteBuffer & buf) const +void SettingEnum::serialize(WriteBuffer & buf, SettingsBinaryFormat) const { - writeBinary(toString(), buf); + writeStringBinary(toString(), buf); } template -void SettingEnum::deserialize(ReadBuffer & buf) +void SettingEnum::deserialize(ReadBuffer & buf, SettingsBinaryFormat) { String s; - readBinary(s, buf); + readStringBinary(s, buf); set(s); } @@ -462,14 +504,43 @@ IMPLEMENT_SETTING_ENUM(LogsLevel, LOGS_LEVEL_LIST_OF_NAMES, ErrorCodes::BAD_ARGU namespace details { + void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf) + { + writeStringBinary(name, buf); + } + String SettingsCollectionUtils::deserializeName(ReadBuffer & buf) { String name; - readBinary(name, buf); + readStringBinary(name, buf); return name; } - void SettingsCollectionUtils::serializeName(const StringRef & name, WriteBuffer & buf) { writeBinary(name, buf); } + void SettingsCollectionUtils::serializeFlag(bool flag, WriteBuffer & buf) + { + buf.write(flag); + } + + bool SettingsCollectionUtils::deserializeFlag(ReadBuffer & buf) + { + char c; + buf.readStrict(c); + return c; + } + + void SettingsCollectionUtils::skipValue(ReadBuffer & buf) + { + /// Ignore a string written by the function writeStringBinary(). + UInt64 size; + readVarUInt(size, buf); + buf.ignore(size); + } + + void SettingsCollectionUtils::warningNameNotFound(const StringRef & name) + { + static auto * log = &Logger::get("Settings"); + LOG_WARNING(log, "Unknown setting " << name << ", skipping"); + } void SettingsCollectionUtils::throwNameNotFound(const StringRef & name) { diff --git a/dbms/src/Core/SettingsCommon.h b/dbms/src/Core/SettingsCollection.h similarity index 67% rename from dbms/src/Core/SettingsCommon.h rename to dbms/src/Core/SettingsCollection.h index 20da4d77685..57bd42bc7b3 100644 --- a/dbms/src/Core/SettingsCommon.h +++ b/dbms/src/Core/SettingsCollection.h @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -17,6 +16,8 @@ struct SettingChange; using SettingsChanges = std::vector; class ReadBuffer; class WriteBuffer; +enum class SettingsBinaryFormat; + /** One setting for any type. * Stores a value within itself, as well as a flag - whether the value was changed. @@ -51,10 +52,10 @@ struct SettingNumber void set(const String & x); /// Serialize to binary stream suitable for transfer over network. - void serialize(WriteBuffer & buf) const; + void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; /// Read from binary stream. - void deserialize(ReadBuffer & buf); + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); }; using SettingUInt64 = SettingNumber; @@ -85,8 +86,8 @@ struct SettingMaxThreads void set(const Field & x); void set(const String & x); - void serialize(WriteBuffer & buf) const; - void deserialize(ReadBuffer & buf); + void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); void setAuto(); UInt64 getAutoValue() const; @@ -118,8 +119,8 @@ struct SettingTimespan void set(const Field & x); void set(const String & x); - void serialize(WriteBuffer & buf) const; - void deserialize(ReadBuffer & buf); + void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); static constexpr UInt64 microseconds_per_io_unit = (io_unit == SettingTimespanIO::MILLISECOND) ? 1000 : 1000000; }; @@ -144,8 +145,8 @@ struct SettingString void set(const String & x); void set(const Field & x); - void serialize(WriteBuffer & buf) const; - void deserialize(ReadBuffer & buf); + void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); }; @@ -167,8 +168,8 @@ public: void set(const String & x); void set(const Field & x); - void serialize(WriteBuffer & buf) const; - void deserialize(ReadBuffer & buf); + void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); }; @@ -191,8 +192,8 @@ struct SettingEnum void set(const Field & x); void set(const String & x); - void serialize(WriteBuffer & buf) const; - void deserialize(ReadBuffer & buf); + void serialize(WriteBuffer & buf, SettingsBinaryFormat format) const; + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format); }; @@ -269,15 +270,12 @@ enum class LogsLevel using SettingLogsLevel = SettingEnum; -namespace details +enum class SettingsBinaryFormat { - struct SettingsCollectionUtils - { - static void serializeName(const StringRef & name, WriteBuffer & buf); - static String deserializeName(ReadBuffer & buf); - [[noreturn]] static void throwNameNotFound(const StringRef & name); - }; -} + OLD, /// Part of the settings are serialized as strings, and other part as varints. This is the old behaviour. + STRINGS, /// All settings are serialized as strings. Before each value the flag `is_ignorable` is serialized. + DEFAULT = STRINGS, +}; /** Template class to define collections of settings. @@ -287,9 +285,9 @@ namespace details * struct MySettings : public SettingsCollection * { * # define APPLY_FOR_MYSETTINGS(M) \ - * M(SettingUInt64, a, 100, "Description of a") \ - * M(SettingFloat, f, 3.11, "Description of f") \ - * M(SettingString, s, "default", "Description of s") + * M(SettingUInt64, a, 100, "Description of a", 0) \ + * M(SettingFloat, f, 3.11, "Description of f", IGNORABLE) // IGNORABLE - means the setting can be ignored by older versions) \ + * M(SettingString, s, "default", "Description of s", 0) * * DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS) * }; @@ -304,21 +302,22 @@ private: Derived & castToDerived() { return *static_cast(this); } const Derived & castToDerived() const { return *static_cast(this); } - using IsChangedFunction = bool (*)(const Derived &); - using GetStringFunction = String (*)(const Derived &); - using GetFieldFunction = Field (*)(const Derived &); - using SetStringFunction = void (*)(Derived &, const String &); - using SetFieldFunction = void (*)(Derived &, const Field &); - using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf); - using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf); - using ValueToStringFunction = String (*)(const Field &); - using ValueToCorrespondingTypeFunction = Field (*)(const Field &); - struct MemberInfo { - IsChangedFunction is_changed; + using IsChangedFunction = bool (*)(const Derived &); + using GetStringFunction = String (*)(const Derived &); + using GetFieldFunction = Field (*)(const Derived &); + using SetStringFunction = void (*)(Derived &, const String &); + using SetFieldFunction = void (*)(Derived &, const Field &); + using SerializeFunction = void (*)(const Derived &, WriteBuffer & buf, SettingsBinaryFormat); + using DeserializeFunction = void (*)(Derived &, ReadBuffer & buf, SettingsBinaryFormat); + using ValueToStringFunction = String (*)(const Field &); + using ValueToCorrespondingTypeFunction = Field (*)(const Field &); + StringRef name; StringRef description; + bool is_ignorable; + IsChangedFunction is_changed; GetStringFunction get_string; GetFieldFunction get_field; SetStringFunction set_string; @@ -329,52 +328,22 @@ private: ValueToCorrespondingTypeFunction value_to_corresponding_type; }; - class MemberInfos : private boost::noncopyable + class MemberInfos { public: - static const MemberInfos & instance(); - - size_t size() const { return infos.size(); } - const MemberInfo & operator[](size_t index) const { return infos[index]; } - const MemberInfo * begin() const { return infos.data(); } - const MemberInfo * end() const { return infos.data() + infos.size(); } - - size_t findIndex(const StringRef & name) const - { - auto it = by_name_map.find(name); - if (it == by_name_map.end()) - return static_cast(-1); // npos - return it->second; - } - - size_t findIndexStrict(const StringRef & name) const - { - auto it = by_name_map.find(name); - if (it == by_name_map.end()) - details::SettingsCollectionUtils::throwNameNotFound(name); - return it->second; - } - - const MemberInfo * find(const StringRef & name) const - { - auto it = by_name_map.find(name); - if (it == by_name_map.end()) - return end(); - else - return &infos[it->second]; - } - - const MemberInfo * findStrict(const StringRef & name) const { return &infos[findIndexStrict(name)]; } - - private: MemberInfos(); - void add(MemberInfo && member) - { - size_t index = infos.size(); - infos.emplace_back(member); - by_name_map.emplace(infos.back().name, index); - } + size_t size() const { return infos.size(); } + const MemberInfo * data() const { return infos.data(); } + const MemberInfo & operator[](size_t index) const { return infos[index]; } + + const MemberInfo * find(const StringRef & name) const; + const MemberInfo & findStrict(const StringRef & name) const; + size_t findIndex(const StringRef & name) const; + size_t findIndexStrict(const StringRef & name) const; + + private: + void add(MemberInfo && member); std::vector infos; std::unordered_map by_name_map; @@ -396,6 +365,7 @@ public: bool isChanged() const { return member->is_changed(*collection); } Field getValue() const; String getValueAsString() const { return member->get_string(*collection); } + protected: friend class SettingsCollection::const_iterator; const_reference() : collection(nullptr), member(nullptr) {} @@ -410,7 +380,7 @@ public: public: reference(Derived & collection_, const MemberInfo & member_) : const_reference(collection_, member_) {} reference(const const_reference & src) : const_reference(src) {} - void setValue(const Field & value); + void setValue(const Field & value) { this->member->set_field(*const_cast(this->collection), value); } void setValue(const String & value) { this->member->set_string(*const_cast(this->collection), value); } }; @@ -453,7 +423,7 @@ public: /// Returns description of a setting. static StringRef getDescription(size_t index) { return members()[index].description; } - static StringRef getDescription(const String & name) { return members().findStrict(name)->description; } + static StringRef getDescription(const String & name) { return members().findStrict(name).description; } /// Searches a setting by its name; returns `npos` if not found. static size_t findIndex(const StringRef & name) { return members().findIndex(name); } @@ -463,36 +433,36 @@ public: static size_t findIndexStrict(const StringRef & name) { return members().findIndexStrict(name); } /// Casts a value to a string according to a specified setting without actual changing this settings. - static String valueToString(size_t index, const Field & value); - static String valueToString(const StringRef & name, const Field & value); + static String valueToString(size_t index, const Field & value) { return members()[index].value_to_string(value); } + static String valueToString(const StringRef & name, const Field & value) { return members().findStrict(name).value_to_string(value); } /// Casts a value to a type according to a specified setting without actual changing this settings. /// E.g. for SettingInt64 it casts Field to Field::Types::Int64. static Field valueToCorrespondingType(size_t index, const Field & value); static Field valueToCorrespondingType(const StringRef & name, const Field & value); - iterator begin() { return iterator(castToDerived(), members().begin()); } - const_iterator begin() const { return const_iterator(castToDerived(), members().begin()); } - iterator end() { return iterator(castToDerived(), members().end()); } - const_iterator end() const { return const_iterator(castToDerived(), members().end()); } + iterator begin() { return iterator(castToDerived(), members().data()); } + const_iterator begin() const { return const_iterator(castToDerived(), members().data()); } + iterator end() { const auto & the_members = members(); return iterator(castToDerived(), the_members.data() + the_members.size()); } + const_iterator end() const { const auto & the_members = members(); return const_iterator(castToDerived(), the_members.data() + the_members.size()); } /// Returns a proxy object for accessing to a setting. Throws an exception if there is not setting with such name. reference operator[](size_t index) { return reference(castToDerived(), members()[index]); } - reference operator[](const StringRef & name) { return reference(castToDerived(), *(members().findStrict(name))); } + reference operator[](const StringRef & name) { return reference(castToDerived(), members().findStrict(name)); } const_reference operator[](size_t index) const { return const_reference(castToDerived(), members()[index]); } - const_reference operator[](const StringRef & name) const { return const_reference(castToDerived(), *(members().findStrict(name))); } + const_reference operator[](const StringRef & name) const { return const_reference(castToDerived(), members().findStrict(name)); } /// Searches a setting by its name; returns end() if not found. - iterator find(const StringRef & name) { return iterator(castToDerived(), members().find(name)); } - const_iterator find(const StringRef & name) const { return const_iterator(castToDerived(), members().find(name)); } + iterator find(const StringRef & name); + const_iterator find(const StringRef & name) const; /// Searches a setting by its name; throws an exception if not found. - iterator findStrict(const StringRef & name) { return iterator(castToDerived(), members().findStrict(name)); } - const_iterator findStrict(const StringRef & name) const { return const_iterator(castToDerived(), members().findStrict(name)); } + iterator findStrict(const StringRef & name); + const_iterator findStrict(const StringRef & name) const; /// Sets setting's value. - void set(size_t index, const Field & value); - void set(const StringRef & name, const Field & value); + void set(size_t index, const Field & value) { (*this)[index].setValue(value); } + void set(const StringRef & name, const Field & value) { (*this)[name].setValue(value); } /// Sets setting's value. Read value in text form from string (for example, from configuration file or from URL parameter). void set(size_t index, const String & value) { (*this)[index].setValue(value); } @@ -514,11 +484,7 @@ public: /// Compares two collections of settings. bool operator ==(const Derived & rhs) const; - - bool operator !=(const Derived & rhs) const - { - return !(*this == rhs); - } + bool operator!=(const Derived & rhs) const { return !(*this == rhs); } /// Gathers all changed values (e.g. for applying them later to another collection of settings). SettingsChanges changes() const; @@ -536,82 +502,16 @@ public: /// Writes the settings to buffer (e.g. to be sent to remote server). /// Only changed settings are written. They are written as list of contiguous name-value pairs, /// finished with empty name. - void serialize(WriteBuffer & buf) const - { - for (const auto & member : members()) - { - if (member.is_changed(castToDerived())) - { - details::SettingsCollectionUtils::serializeName(member.name, buf); - member.serialize(castToDerived(), buf); - } - } - details::SettingsCollectionUtils::serializeName(StringRef{} /* empty string is a marker of the end of settings */, buf); - } + void serialize(WriteBuffer & buf, SettingsBinaryFormat format = SettingsBinaryFormat::DEFAULT) const; /// Reads the settings from buffer. - void deserialize(ReadBuffer & buf) - { - const auto & the_members = members(); - while (true) - { - String name = details::SettingsCollectionUtils::deserializeName(buf); - if (name.empty() /* empty string is a marker of the end of settings */) - break; - the_members.findStrict(name)->deserialize(castToDerived(), buf); - } - } + void deserialize(ReadBuffer & buf, SettingsBinaryFormat format = SettingsBinaryFormat::DEFAULT); }; + #define DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS_MACRO) \ LIST_OF_SETTINGS_MACRO(DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_) - -#define IMPLEMENT_SETTINGS_COLLECTION(DERIVED_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ - template<> \ - SettingsCollection::MemberInfos::MemberInfos() \ - { \ - using Derived = DERIVED_CLASS_NAME; \ - struct Functions \ - { \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ - }; \ - LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \ - } \ - template <> \ - const SettingsCollection::MemberInfos & SettingsCollection::MemberInfos::instance() \ - { \ - static const SettingsCollection::MemberInfos single_instance; \ - return single_instance; \ - } \ - /** \ - * Instantiation should happen when all method definitions from SettingsCollectionImpl.h \ - * are accessible, so we instantiate explicitly. \ - */ \ - template class SettingsCollection; - - - -#define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ +#define DECLARE_SETTINGS_COLLECTION_DECLARE_VARIABLES_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ TYPE NAME {DEFAULT}; - - -#define IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - static String NAME##_getString(const Derived & collection) { return collection.NAME.toString(); } \ - static Field NAME##_getField(const Derived & collection) { return collection.NAME.toField(); } \ - static void NAME##_setString(Derived & collection, const String & value) { collection.NAME.set(value); } \ - static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \ - static void NAME##_serialize(const Derived & collection, WriteBuffer & buf) { collection.NAME.serialize(buf); } \ - static void NAME##_deserialize(Derived & collection, ReadBuffer & buf) { collection.NAME.deserialize(buf); } \ - static String NAME##_valueToString(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ - static Field NAME##_valueToCorrespondingType(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ - - -#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION) \ - add({[](const Derived & d) { return d.NAME.changed; }, \ - StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ - &Functions::NAME##_getString, &Functions::NAME##_getField, \ - &Functions::NAME##_setString, &Functions::NAME##_setField, \ - &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ - &Functions::NAME##_valueToString, &Functions::NAME##_valueToCorrespondingType}); } diff --git a/dbms/src/Core/SettingsCollectionImpl.h b/dbms/src/Core/SettingsCollectionImpl.h index 195dffe609d..d6ff9573a4c 100644 --- a/dbms/src/Core/SettingsCollectionImpl.h +++ b/dbms/src/Core/SettingsCollectionImpl.h @@ -2,15 +2,84 @@ /** * This file implements some functions that are dependent on Field type. - * Unlinke SettingsCommon.h, we only have to include it once for each - * instantiation of SettingsCollection<>. This allows to work on Field without - * always recompiling the entire project. + * Unlike SettingsCollection.h, we only have to include it once for each + * instantiation of SettingsCollection<>. */ #include namespace DB { +namespace details +{ + struct SettingsCollectionUtils + { + static void serializeName(const StringRef & name, WriteBuffer & buf); + static String deserializeName(ReadBuffer & buf); + static void serializeFlag(bool flag, WriteBuffer & buf); + static bool deserializeFlag(ReadBuffer & buf); + static void skipValue(ReadBuffer & buf); + static void warningNameNotFound(const StringRef & name); + [[noreturn]] static void throwNameNotFound(const StringRef & name); + }; +} + + +template +size_t SettingsCollection::MemberInfos::findIndex(const StringRef & name) const +{ + auto it = by_name_map.find(name); + if (it == by_name_map.end()) + return static_cast(-1); // npos + return it->second; +} + + +template +size_t SettingsCollection::MemberInfos::findIndexStrict(const StringRef & name) const +{ + auto it = by_name_map.find(name); + if (it == by_name_map.end()) + details::SettingsCollectionUtils::throwNameNotFound(name); + return it->second; +} + + +template +const typename SettingsCollection::MemberInfo * SettingsCollection::MemberInfos::find(const StringRef & name) const +{ + auto it = by_name_map.find(name); + if (it == by_name_map.end()) + return nullptr; + else + return &infos[it->second]; +} + + +template +const typename SettingsCollection::MemberInfo & SettingsCollection::MemberInfos::findStrict(const StringRef & name) const +{ + return infos[findIndexStrict(name)]; +} + + +template +void SettingsCollection::MemberInfos::add(MemberInfo && member) +{ + size_t index = infos.size(); + infos.emplace_back(member); + by_name_map.emplace(infos.back().name, index); +} + + +template +const typename SettingsCollection::MemberInfos & +SettingsCollection::members() +{ + static const MemberInfos the_instance; + return the_instance; +} + template Field SettingsCollection::const_reference::getValue() const @@ -18,23 +87,6 @@ Field SettingsCollection::const_reference::getValue() const return member->get_field(*collection); } -template -void SettingsCollection::reference::setValue(const Field & value) -{ - this->member->set_field(*const_cast(this->collection), value); -} - -template -String SettingsCollection::valueToString(size_t index, const Field & value) -{ - return members()[index].value_to_string(value); -} - -template -String SettingsCollection::valueToString(const StringRef & name, const Field & value) -{ - return members().findStrict(name)->value_to_string(value); -} template Field SettingsCollection::valueToCorrespondingType(size_t index, const Field & value) @@ -42,36 +94,62 @@ Field SettingsCollection::valueToCorrespondingType(size_t index, const return members()[index].value_to_corresponding_type(value); } + template Field SettingsCollection::valueToCorrespondingType(const StringRef & name, const Field & value) { - return members().findStrict(name)->value_to_corresponding_type(value); + return members().findStrict(name).value_to_corresponding_type(value); } -template -void SettingsCollection::set(size_t index, const Field & value) -{ - (*this)[index].setValue(value); -} template -void SettingsCollection::set(const StringRef & name, const Field & value) +typename SettingsCollection::iterator SettingsCollection::find(const StringRef & name) { - (*this)[name].setValue(value); + const auto * member = members().find(name); + if (member) + return iterator(castToDerived(), member); + return end(); } + +template +typename SettingsCollection::const_iterator SettingsCollection::find(const StringRef & name) const +{ + const auto * member = members().find(name); + if (member) + return const_iterator(castToDerived(), member); + return end(); +} + + +template +typename SettingsCollection::iterator SettingsCollection::findStrict(const StringRef & name) +{ + return iterator(castToDerived(), &members().findStrict(name)); +} + + +template +typename SettingsCollection::const_iterator SettingsCollection::findStrict(const StringRef & name) const +{ + return const_iterator(castToDerived(), &members().findStrict(name)); +} + + template Field SettingsCollection::get(size_t index) const { return (*this)[index].getValue(); } + template Field SettingsCollection::get(const StringRef & name) const { return (*this)[name].getValue(); } + template bool SettingsCollection::tryGet(const StringRef & name, Field & value) const { @@ -82,6 +160,7 @@ bool SettingsCollection::tryGet(const StringRef & name, Field & value) return true; } + template bool SettingsCollection::tryGet(const StringRef & name, String & value) const { @@ -92,11 +171,14 @@ bool SettingsCollection::tryGet(const StringRef & name, String & value) return true; } + template bool SettingsCollection::operator ==(const Derived & rhs) const { - for (const auto & member : members()) + const auto & the_members = members(); + for (size_t i = 0; i != the_members.size(); ++i) { + const auto & member = the_members[i]; bool left_changed = member.is_changed(castToDerived()); bool right_changed = member.is_changed(rhs); if (left_changed || right_changed) @@ -110,27 +192,29 @@ bool SettingsCollection::operator ==(const Derived & rhs) const return true; } -/// Gathers all changed values (e.g. for applying them later to another collection of settings). + template SettingsChanges SettingsCollection::changes() const { SettingsChanges found_changes; - for (const auto & member : members()) + const auto & the_members = members(); + for (size_t i = 0; i != the_members.size(); ++i) { + const auto & member = the_members[i]; if (member.is_changed(castToDerived())) found_changes.push_back({member.name.toString(), member.get_field(castToDerived())}); } return found_changes; } -/// Applies change to concrete setting. + template void SettingsCollection::applyChange(const SettingChange & change) { set(change.name, change.value); } -/// Applies changes to the settings. + template void SettingsCollection::applyChanges(const SettingsChanges & changes) { @@ -138,25 +222,110 @@ void SettingsCollection::applyChanges(const SettingsChanges & changes) applyChange(change); } + template void SettingsCollection::copyChangesFrom(const Derived & src) { - for (const auto & member : members()) + const auto & the_members = members(); + for (size_t i = 0; i != the_members.size(); ++i) + { + const auto & member = the_members[i]; if (member.is_changed(src)) member.set_field(castToDerived(), member.get_field(src)); + } } + template void SettingsCollection::copyChangesTo(Derived & dest) const { dest.copyChangesFrom(castToDerived()); } + template -const typename SettingsCollection::MemberInfos & -SettingsCollection::members() +void SettingsCollection::serialize(WriteBuffer & buf, SettingsBinaryFormat format) const { - return MemberInfos::instance(); + const auto & the_members = members(); + for (size_t i = 0; i != the_members.size(); ++i) + { + const auto & member = the_members[i]; + if (member.is_changed(castToDerived())) + { + details::SettingsCollectionUtils::serializeName(member.name, buf); + if (format >= SettingsBinaryFormat::STRINGS) + details::SettingsCollectionUtils::serializeFlag(member.is_ignorable, buf); + member.serialize(castToDerived(), buf, format); + } + } + details::SettingsCollectionUtils::serializeName(StringRef{} /* empty string is a marker of the end of settings */, buf); } -} /* namespace DB */ + +template +void SettingsCollection::deserialize(ReadBuffer & buf, SettingsBinaryFormat format) +{ + const auto & the_members = members(); + while (true) + { + String name = details::SettingsCollectionUtils::deserializeName(buf); + if (name.empty() /* empty string is a marker of the end of settings */) + break; + auto * member = the_members.find(name); + bool is_ignorable = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : false; + if (member) + { + member->deserialize(castToDerived(), buf, format); + } + else if (is_ignorable) + { + details::SettingsCollectionUtils::warningNameNotFound(name); + details::SettingsCollectionUtils::skipValue(buf); + } + else + details::SettingsCollectionUtils::throwNameNotFound(name); + } +} + + +//-V:IMPLEMENT_SETTINGS_COLLECTION:501 +#define IMPLEMENT_SETTINGS_COLLECTION(DERIVED_CLASS_NAME, LIST_OF_SETTINGS_MACRO) \ + template<> \ + SettingsCollection::MemberInfos::MemberInfos() \ + { \ + using Derived = DERIVED_CLASS_NAME; \ + struct Functions \ + { \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ + }; \ + constexpr int IGNORABLE = 1; \ + UNUSED(IGNORABLE); \ + LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \ + } \ + /** \ + * Instantiation should happen when all method definitions from SettingsCollectionImpl.h \ + * are accessible, so we instantiate explicitly. \ + */ \ + template class SettingsCollection; + + +#define IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + static String NAME##_getString(const Derived & collection) { return collection.NAME.toString(); } \ + static Field NAME##_getField(const Derived & collection) { return collection.NAME.toField(); } \ + static void NAME##_setString(Derived & collection, const String & value) { collection.NAME.set(value); } \ + static void NAME##_setField(Derived & collection, const Field & value) { collection.NAME.set(value); } \ + static void NAME##_serialize(const Derived & collection, WriteBuffer & buf, SettingsBinaryFormat format) { collection.NAME.serialize(buf, format); } \ + static void NAME##_deserialize(Derived & collection, ReadBuffer & buf, SettingsBinaryFormat format) { collection.NAME.deserialize(buf, format); } \ + static String NAME##_valueToString(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toString(); } \ + static Field NAME##_valueToCorrespondingType(const Field & value) { TYPE temp{DEFAULT}; temp.set(value); return temp.toField(); } \ + + +#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ + add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ + FLAGS & IGNORABLE, \ + [](const Derived & d) { return d.NAME.changed; }, \ + &Functions::NAME##_getString, &Functions::NAME##_getField, \ + &Functions::NAME##_setString, &Functions::NAME##_setField, \ + &Functions::NAME##_serialize, &Functions::NAME##_deserialize, \ + &Functions::NAME##_valueToString, &Functions::NAME##_valueToCorrespondingType}); +} diff --git a/dbms/src/Core/iostream_debug_helpers.cpp b/dbms/src/Core/iostream_debug_helpers.cpp index 8e673d1c547..eea8694dfb0 100644 --- a/dbms/src/Core/iostream_debug_helpers.cpp +++ b/dbms/src/Core/iostream_debug_helpers.cpp @@ -1,6 +1,7 @@ #include "iostream_debug_helpers.h" #include +#include #include #include #include @@ -92,9 +93,9 @@ std::ostream & operator<<(std::ostream & stream, const IColumn & what) return stream; } -std::ostream & operator<<(std::ostream & stream, const Connection::Packet & what) +std::ostream & operator<<(std::ostream & stream, const Packet & what) { - stream << "Connection::Packet(" + stream << "Packet(" << "type = " << what.type; // types description: Core/Protocol.h if (what.exception) diff --git a/dbms/src/Core/iostream_debug_helpers.h b/dbms/src/Core/iostream_debug_helpers.h index 35fc05faf1d..dc48da931f0 100644 --- a/dbms/src/Core/iostream_debug_helpers.h +++ b/dbms/src/Core/iostream_debug_helpers.h @@ -1,9 +1,6 @@ #pragma once #include -#include - - namespace DB { @@ -40,7 +37,8 @@ std::ostream & operator<<(std::ostream & stream, const ColumnWithTypeAndName & w class IColumn; std::ostream & operator<<(std::ostream & stream, const IColumn & what); -std::ostream & operator<<(std::ostream & stream, const Connection::Packet & what); +struct Packet; +std::ostream & operator<<(std::ostream & stream, const Packet & what); struct ExpressionAction; std::ostream & operator<<(std::ostream & stream, const ExpressionAction & what); diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 4f945001686..dfa9194a6f9 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index aedbe676688..f6dc30d6e8c 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -222,7 +222,7 @@ Block RemoteBlockInputStream::readImpl() if (isCancelledOrThrowIfKilled()) return Block(); - Connection::Packet packet = multiplexed_connections->receivePacket(); + Packet packet = multiplexed_connections->receivePacket(); switch (packet.type) { @@ -301,7 +301,7 @@ void RemoteBlockInputStream::readSuffixImpl() tryCancel("Cancelling query because enough data has been read"); /// Get the remaining packets so that there is no out of sync in the connections to the replicas. - Connection::Packet packet = multiplexed_connections->drain(); + Packet packet = multiplexed_connections->drain(); switch (packet.type) { case Protocol::Server::EndOfStream: diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index a95ea174541..3446af8b840 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -32,7 +32,7 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, while (true) { - Connection::Packet packet = connection.receivePacket(); + Packet packet = connection.receivePacket(); if (Protocol::Server::Data == packet.type) { @@ -77,7 +77,7 @@ void RemoteBlockOutputStream::write(const Block & block) auto packet_type = connection.checkPacket(); if (packet_type && *packet_type == Protocol::Server::Exception) { - Connection::Packet packet = connection.receivePacket(); + Packet packet = connection.receivePacket(); packet.exception->rethrow(); } @@ -101,7 +101,7 @@ void RemoteBlockOutputStream::writeSuffix() /// Wait for EndOfStream or Exception packet, skip Log packets. while (true) { - Connection::Packet packet = connection.receivePacket(); + Packet packet = connection.receivePacket(); if (Protocol::Server::EndOfStream == packet.type) break; diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h index 70a47049c2c..0114f62d77c 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h @@ -10,7 +10,7 @@ class Arena; using ArenaPtr = std::shared_ptr; class ExpressionActions; - +enum class TotalsMode; /** Takes blocks after grouping, with non-finalized aggregate functions. * Calculates total values according to totals_mode. diff --git a/dbms/src/Dictionaries/Embedded/RegionsNames.cpp b/dbms/src/Dictionaries/Embedded/RegionsNames.cpp index 9b19ab71373..62c1db071ec 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsNames.cpp +++ b/dbms/src/Dictionaries/Embedded/RegionsNames.cpp @@ -19,7 +19,7 @@ RegionsNames::RegionsNames(IRegionsNamesDataProviderPtr data_provider) { for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id) { - const std::string & language = getSupportedLanguages()[language_id]; + const std::string & language = supported_languages[language_id]; names_sources[language_id] = data_provider->getLanguageRegionsNamesSource(language); } @@ -34,7 +34,7 @@ std::string RegionsNames::dumpSupportedLanguagesNames() if (i > 0) res += ", "; res += '\''; - res += getLanguageAliases()[i].name; + res += language_aliases[i].first; res += '\''; } return res; @@ -48,7 +48,7 @@ void RegionsNames::reload() RegionID max_region_id = 0; for (size_t language_id = 0; language_id < SUPPORTED_LANGUAGES_COUNT; ++language_id) { - const std::string & language = getSupportedLanguages()[language_id]; + const std::string & language = supported_languages[language_id]; auto names_source = names_sources[language_id]; diff --git a/dbms/src/Dictionaries/Embedded/RegionsNames.h b/dbms/src/Dictionaries/Embedded/RegionsNames.h index 7acb23d001e..6aede7fb464 100644 --- a/dbms/src/Dictionaries/Embedded/RegionsNames.h +++ b/dbms/src/Dictionaries/Embedded/RegionsNames.h @@ -20,7 +20,7 @@ class RegionsNames { public: - enum class Language + enum class Language : size_t { RU = 0, EN, @@ -28,36 +28,35 @@ public: BY, KZ, TR, + + END }; private: - static const size_t ROOT_LANGUAGE = 0; - static const size_t SUPPORTED_LANGUAGES_COUNT = 6; - static const size_t LANGUAGE_ALIASES_COUNT = 7; - - static const char ** getSupportedLanguages() + static inline constexpr const char * supported_languages[] = { - static const char * res[]{"ru", "en", "ua", "by", "kz", "tr"}; - return res; - } - - struct language_alias - { - const char * const name; - const Language lang; + "ru", + "en", + "ua", + "by", + "kz", + "tr" }; - static const language_alias * getLanguageAliases() - { - static constexpr const language_alias language_aliases[]{{"ru", Language::RU}, - {"en", Language::EN}, - {"ua", Language::UA}, - {"uk", Language::UA}, - {"by", Language::BY}, - {"kz", Language::KZ}, - {"tr", Language::TR}}; - return language_aliases; - } + static inline constexpr std::pair language_aliases[] = + { + {"ru", Language::RU}, + {"en", Language::EN}, + {"ua", Language::UA}, + {"uk", Language::UA}, + {"by", Language::BY}, + {"kz", Language::KZ}, + {"tr", Language::TR} + }; + + static constexpr size_t ROOT_LANGUAGE = 0; + static constexpr size_t SUPPORTED_LANGUAGES_COUNT = size_t(Language::END); + static constexpr size_t LANGUAGE_ALIASES_COUNT = sizeof(language_aliases); using NamesSources = std::vector>; @@ -94,9 +93,9 @@ public: { for (size_t i = 0; i < LANGUAGE_ALIASES_COUNT; ++i) { - const auto & alias = getLanguageAliases()[i]; - if (language[0] == alias.name[0] && language[1] == alias.name[1]) - return alias.lang; + const auto & alias = language_aliases[i]; + if (language[0] == alias.first[0] && language[1] == alias.first[1]) + return alias.second; } } throw Poco::Exception("Unsupported language for region name. Supported languages are: " + dumpSupportedLanguagesNames() + "."); diff --git a/dbms/src/Dictionaries/readInvalidateQuery.cpp b/dbms/src/Dictionaries/readInvalidateQuery.cpp index 26f543af0b9..9b5b34133b2 100644 --- a/dbms/src/Dictionaries/readInvalidateQuery.cpp +++ b/dbms/src/Dictionaries/readInvalidateQuery.cpp @@ -1,6 +1,7 @@ #include "readInvalidateQuery.h" #include #include +#include namespace DB diff --git a/dbms/src/Functions/array/arrayIntersect.cpp b/dbms/src/Functions/array/arrayIntersect.cpp index 8881abb1552..2da0667f577 100644 --- a/dbms/src/Functions/array/arrayIntersect.cpp +++ b/dbms/src/Functions/array/arrayIntersect.cpp @@ -58,10 +58,19 @@ private: struct UnpackedArrays { size_t base_rows = 0; - std::vector is_const; - std::vector null_maps; - std::vector offsets; - ColumnRawPtrs nested_columns; + + struct UnpackedArray + { + bool is_const = false; + const NullMap * null_map = nullptr; + const NullMap * overflow_mask = nullptr; + const ColumnArray::ColumnOffsets::Container * offsets = nullptr; + const IColumn * nested_column = nullptr; + + }; + + std::vector args; + Columns column_holders; UnpackedArrays() = default; }; @@ -69,9 +78,16 @@ private: /// Cast column to data_type removing nullable if data_type hasn't. /// It's expected that column can represent data_type after removing some NullMap's. ColumnPtr castRemoveNullable(const ColumnPtr & column, const DataTypePtr & data_type) const; - Columns castColumns(Block & block, const ColumnNumbers & arguments, + + struct CastArgumentsResult + { + ColumnsWithTypeAndName initial; + ColumnsWithTypeAndName casted; + }; + + CastArgumentsResult castColumns(Block & block, const ColumnNumbers & arguments, const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls) const; - UnpackedArrays prepareArrays(const Columns & columns) const; + UnpackedArrays prepareArrays(const ColumnsWithTypeAndName & columns, ColumnsWithTypeAndName & initial_columns) const; template static ColumnPtr execute(const UnpackedArrays & arrays, MutableColumnPtr result_data); @@ -173,12 +189,13 @@ ColumnPtr FunctionArrayIntersect::castRemoveNullable(const ColumnPtr & column, c return column; } -Columns FunctionArrayIntersect::castColumns( +FunctionArrayIntersect::CastArgumentsResult FunctionArrayIntersect::castColumns( Block & block, const ColumnNumbers & arguments, const DataTypePtr & return_type, const DataTypePtr & return_type_with_nulls) const { size_t num_args = arguments.size(); - Columns columns(num_args); + ColumnsWithTypeAndName initial_columns(num_args); + ColumnsWithTypeAndName columns(num_args); auto type_array = checkAndGetDataType(return_type.get()); auto & type_nested = type_array->getNestedType(); @@ -201,6 +218,8 @@ Columns FunctionArrayIntersect::castColumns( for (size_t i = 0; i < num_args; ++i) { const ColumnWithTypeAndName & arg = block.getByPosition(arguments[i]); + initial_columns[i] = arg; + columns[i] = arg; auto & column = columns[i]; if (is_numeric_or_string) @@ -208,68 +227,120 @@ Columns FunctionArrayIntersect::castColumns( /// Cast to Array(T) or Array(Nullable(T)). if (nested_is_nullable) { - if (arg.type->equals(*return_type)) - column = arg.column; - else - column = castColumn(arg, return_type, context); + if (!arg.type->equals(*return_type)) + { + column.column = castColumn(arg, return_type, context); + column.type = return_type; + } } else { - /// If result has array type Array(T) still cast Array(Nullable(U)) to Array(Nullable(T)) - /// because cannot cast Nullable(T) to T. - if (arg.type->equals(*return_type) || arg.type->equals(*nullable_return_type)) - column = arg.column; - else if (static_cast(*arg.type).getNestedType()->isNullable()) - column = castColumn(arg, nullable_return_type, context); - else - column = castColumn(arg, return_type, context); + + if (!arg.type->equals(*return_type) && !arg.type->equals(*nullable_return_type)) + { + /// If result has array type Array(T) still cast Array(Nullable(U)) to Array(Nullable(T)) + /// because cannot cast Nullable(T) to T. + if (static_cast(*arg.type).getNestedType()->isNullable()) + { + column.column = castColumn(arg, nullable_return_type, context); + column.type = nullable_return_type; + } + else + { + column.column = castColumn(arg, return_type, context); + column.type = return_type; + } + } } } else { /// return_type_with_nulls is the most common subtype with possible nullable parts. - if (arg.type->equals(*return_type_with_nulls)) - column = arg.column; - else - column = castColumn(arg, return_type_with_nulls, context); + if (!arg.type->equals(*return_type_with_nulls)) + { + column.column = castColumn(arg, return_type_with_nulls, context); + column.type = return_type_with_nulls; + } } } - return columns; + return {.initial = initial_columns, .casted = columns}; } -FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(const Columns & columns) const +static ColumnPtr callFunctionNotEquals(ColumnWithTypeAndName first, ColumnWithTypeAndName second, const Context & context) +{ + ColumnsWithTypeAndName args; + args.reserve(2); + args.emplace_back(std::move(first)); + args.emplace_back(std::move(second)); + + auto eq_func = FunctionFactory::instance().get("notEquals", context)->build(args); + + Block block = args; + block.insert({nullptr, eq_func->getReturnType(), ""}); + + eq_func->execute(block, {0, 1}, 2, args.front().column->size()); + + return block.getByPosition(2).column; +} + +FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays( + const ColumnsWithTypeAndName & columns, ColumnsWithTypeAndName & initial_columns) const { UnpackedArrays arrays; size_t columns_number = columns.size(); - arrays.is_const.assign(columns_number, false); - arrays.null_maps.resize(columns_number); - arrays.offsets.resize(columns_number); - arrays.nested_columns.resize(columns_number); + arrays.args.resize(columns_number); bool all_const = true; for (auto i : ext::range(0, columns_number)) { - auto argument_column = columns[i].get(); + auto & arg = arrays.args[i]; + auto argument_column = columns[i].column.get(); + auto initial_column = initial_columns[i].column.get(); + if (auto argument_column_const = typeid_cast(argument_column)) { - arrays.is_const[i] = true; + arg.is_const = true; argument_column = argument_column_const->getDataColumnPtr().get(); + initial_column = typeid_cast(initial_column)->getDataColumnPtr().get(); } if (auto argument_column_array = typeid_cast(argument_column)) { - if (!arrays.is_const[i]) + if (!arg.is_const) all_const = false; - arrays.offsets[i] = &argument_column_array->getOffsets(); - arrays.nested_columns[i] = &argument_column_array->getData(); - if (auto column_nullable = typeid_cast(arrays.nested_columns[i])) + arg.offsets = &argument_column_array->getOffsets(); + arg.nested_column = &argument_column_array->getData(); + + initial_column = &typeid_cast(initial_column)->getData(); + + if (auto column_nullable = typeid_cast(arg.nested_column)) { - arrays.null_maps[i] = &column_nullable->getNullMapData(); - arrays.nested_columns[i] = &column_nullable->getNestedColumn(); + arg.null_map = &column_nullable->getNullMapData(); + arg.nested_column = &column_nullable->getNestedColumn(); + initial_column = &typeid_cast(initial_column)->getNestedColumn(); + } + + /// In case column was casted need to create overflow mask for integer types. + if (arg.nested_column != initial_column) + { + auto & nested_init_type = typeid_cast(removeNullable(initial_columns[i].type).get())->getNestedType(); + auto & nested_cast_type = typeid_cast(removeNullable(columns[i].type).get())->getNestedType(); + + if (isInteger(nested_init_type) || isDateOrDateTime(nested_init_type)) + { + /// Compare original and casted columns. It seem to be the easiest way. + auto overflow_mask = callFunctionNotEquals( + {arg.nested_column->getPtr(), nested_init_type, ""}, + {initial_column->getPtr(), nested_cast_type, ""}, + context); + + arg.overflow_mask = &typeid_cast(overflow_mask.get())->getData(); + arrays.column_holders.emplace_back(std::move(overflow_mask)); + } } } else @@ -278,16 +349,16 @@ FunctionArrayIntersect::UnpackedArrays FunctionArrayIntersect::prepareArrays(con if (all_const) { - arrays.base_rows = arrays.offsets.front()->size(); + arrays.base_rows = arrays.args.front().offsets->size(); } else { for (auto i : ext::range(0, columns_number)) { - if (arrays.is_const[i]) + if (arrays.args[i].is_const) continue; - size_t rows = arrays.offsets[i]->size(); + size_t rows = arrays.args[i].offsets->size(); if (arrays.base_rows == 0 && rows > 0) arrays.base_rows = rows; else if (arrays.base_rows != rows) @@ -322,9 +393,9 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar auto return_type_with_nulls = getMostSubtype(data_types, true, true); - Columns columns = castColumns(block, arguments, return_type, return_type_with_nulls); + auto columns = castColumns(block, arguments, return_type, return_type_with_nulls); - UnpackedArrays arrays = prepareArrays(columns); + UnpackedArrays arrays = prepareArrays(columns.casted, columns.initial); ColumnPtr result_column; auto not_nullable_nested_return_type = removeNullable(nested_return_type); @@ -356,7 +427,7 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar result_column = execute(arrays, std::move(column)); else { - column = static_cast(*return_type_with_nulls).getNestedType()->createColumn(); + column = assert_cast(*return_type_with_nulls).getNestedType()->createColumn(); result_column = castRemoveNullable(execute(arrays, std::move(column)), return_type); } } @@ -377,24 +448,24 @@ void FunctionArrayIntersect::NumberExecutor::operator()() template ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr) { - auto args = arrays.nested_columns.size(); + auto args = arrays.args.size(); auto rows = arrays.base_rows; bool all_nullable = true; std::vector columns; columns.reserve(args); - for (auto arg : ext::range(0, args)) + for (auto & arg : arrays.args) { if constexpr (std::is_same::value) - columns.push_back(arrays.nested_columns[arg]); + columns.push_back(arg.nested_column); else - columns.push_back(checkAndGetColumn(arrays.nested_columns[arg])); + columns.push_back(checkAndGetColumn(arg.nested_column)); if (!columns.back()) throw Exception("Unexpected array type for function arrayIntersect", ErrorCodes::LOGICAL_ERROR); - if (!arrays.null_maps[arg]) + if (!arg.null_map) all_nullable = false; } @@ -415,44 +486,45 @@ ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, Mutable bool all_has_nullable = all_nullable; - for (auto arg : ext::range(0, args)) + for (auto arg_num : ext::range(0, args)) { + auto & arg = arrays.args[arg_num]; bool current_has_nullable = false; size_t off; // const array has only one row - bool const_arg = arrays.is_const[arg]; - if (const_arg) - off = (*arrays.offsets[arg])[0]; + if (arg.is_const) + off = (*arg.offsets)[0]; else - off = (*arrays.offsets[arg])[row]; + off = (*arg.offsets)[row]; - for (auto i : ext::range(prev_off[arg], off)) + for (auto i : ext::range(prev_off[arg_num], off)) { - if (arrays.null_maps[arg] && (*arrays.null_maps[arg])[i]) + if (arg.null_map && (*arg.null_map)[i]) current_has_nullable = true; - else + else if (!arg.overflow_mask || (*arg.overflow_mask)[i] == 0) { typename Map::mapped_type * value = nullptr; if constexpr (is_numeric_column) - value = &map[columns[arg]->getElement(i)]; + value = &map[columns[arg_num]->getElement(i)]; else if constexpr (std::is_same::value || std::is_same::value) - value = &map[columns[arg]->getDataAt(i)]; + value = &map[columns[arg_num]->getDataAt(i)]; else { const char * data = nullptr; - value = &map[columns[arg]->serializeValueIntoArena(i, arena, data)]; + value = &map[columns[arg_num]->serializeValueIntoArena(i, arena, data)]; } - if (*value == arg) + /// Here we count the number of element appearances, but no more than once per array. + if (*value == arg_num) ++(*value); } } - prev_off[arg] = off; - if (const_arg) - prev_off[arg] = 0; + prev_off[arg_num] = off; + if (arg.is_const) + prev_off[arg_num] = 0; if (!current_has_nullable) all_has_nullable = false; diff --git a/dbms/src/Functions/modulo.cpp b/dbms/src/Functions/modulo.cpp index b308a820421..35eeedfc9ce 100644 --- a/dbms/src/Functions/modulo.cpp +++ b/dbms/src/Functions/modulo.cpp @@ -61,8 +61,23 @@ struct ModuloByConstantImpl /// Here we failed to make the SSE variant from libdivide give an advantage. size_t size = a.size(); - for (size_t i = 0; i < size; ++i) - c[i] = a[i] - (a[i] / divider) * b; /// NOTE: perhaps, the division semantics with the remainder of negative numbers is not preserved. + + /// strict aliasing optimization for char like arrays + auto * __restrict src = a.data(); + auto * __restrict dst = c.data(); + + if (b & (b - 1)) + { + for (size_t i = 0; i < size; ++i) + dst[i] = src[i] - (src[i] / divider) * b; /// NOTE: perhaps, the division semantics with the remainder of negative numbers is not preserved. + } + else + { + // gcc libdivide doesn't work well for pow2 division + auto mask = b - 1; + for (size_t i = 0; i < size; ++i) + dst[i] = src[i] & mask; + } } }; diff --git a/dbms/src/IO/BufferWithOwnMemory.h b/dbms/src/IO/BufferWithOwnMemory.h index 4849a52344b..713c9b12cd1 100644 --- a/dbms/src/IO/BufferWithOwnMemory.h +++ b/dbms/src/IO/BufferWithOwnMemory.h @@ -77,7 +77,7 @@ struct Memory : boost::noncopyable, Allocator m_capacity = new_size; alloc(); } - else if (new_size <= m_size) + else if (new_size <= m_capacity - pad_right) { m_size = new_size; return; diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index f0b6a8d434c..a7052c90a1d 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include diff --git a/dbms/src/Interpreters/IdentifierSemantic.cpp b/dbms/src/Interpreters/IdentifierSemantic.cpp index 34910ef039f..45b58c1293a 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.cpp +++ b/dbms/src/Interpreters/IdentifierSemantic.cpp @@ -132,6 +132,15 @@ std::pair IdentifierSemantic::extractDatabaseAndTable(const ASTI return { "", identifier.name }; } +std::optional IdentifierSemantic::extractNestedName(const ASTIdentifier & identifier, const String & table_name) +{ + if (identifier.name_parts.size() == 3 && table_name == identifier.name_parts[0]) + return identifier.name_parts[1] + '.' + identifier.name_parts[2]; + else if (identifier.name_parts.size() == 2) + return identifier.name_parts[0] + '.' + identifier.name_parts[1]; + return {}; +} + bool IdentifierSemantic::doesIdentifierBelongTo(const ASTIdentifier & identifier, const String & database, const String & table) { size_t num_components = identifier.name_parts.size(); diff --git a/dbms/src/Interpreters/IdentifierSemantic.h b/dbms/src/Interpreters/IdentifierSemantic.h index 82b5ff31dde..dc5859dc05e 100644 --- a/dbms/src/Interpreters/IdentifierSemantic.h +++ b/dbms/src/Interpreters/IdentifierSemantic.h @@ -36,6 +36,7 @@ struct IdentifierSemantic static std::optional getTableName(const ASTIdentifier & node); static std::optional getTableName(const ASTPtr & ast); static std::pair extractDatabaseAndTable(const ASTIdentifier & identifier); + static std::optional extractNestedName(const ASTIdentifier & identifier, const String & table_name); static ColumnMatch canReferColumnToTable(const ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); static void setColumnShortName(ASTIdentifier & identifier, const DatabaseAndTableWithAlias & db_and_table); diff --git a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h index 00984832ad6..34d49888528 100644 --- a/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h +++ b/dbms/src/Interpreters/InJoinSubqueriesPreprocessor.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 09afaad90fa..c1c41b5a66e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -925,14 +925,14 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c } -static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query, +static InputSortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query, const Context & context, const SyntaxAnalyzerResultPtr & global_syntax_result) { if (!merge_tree.hasSortingKey()) return {}; auto order_descr = getSortDescription(query, context); - SortDescription prefix_order_descr; + SortDescription order_key_prefix_descr; int read_direction = order_descr.at(0).direction; const auto & sorting_key_columns = merge_tree.getSortingKeyColumns(); @@ -947,7 +947,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons /// or in some simple cases when order key element is wrapped into monotonic function. int current_direction = order_descr[i].direction; if (order_descr[i].column_name == sorting_key_columns[i] && current_direction == read_direction) - prefix_order_descr.push_back(order_descr[i]); + order_key_prefix_descr.push_back(order_descr[i]); else { auto ast = query.orderBy()->children[i]->children.at(0); @@ -995,14 +995,14 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons if (i == 0) read_direction = current_direction; - prefix_order_descr.push_back(order_descr[i]); + order_key_prefix_descr.push_back(order_descr[i]); } } - if (prefix_order_descr.empty()) + if (order_key_prefix_descr.empty()) return {}; - return std::make_shared(std::move(prefix_order_descr), read_direction); + return std::make_shared(std::move(order_key_prefix_descr), read_direction); } @@ -1026,11 +1026,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS const Settings & settings = context.getSettingsRef(); auto & expressions = analysis_result; - SortingInfoPtr sorting_info; + InputSortingInfoPtr input_sorting_info; if (settings.optimize_read_in_order && storage && query.orderBy() && !query_analyzer->hasAggregation() && !query.final() && !query.join()) { if (const auto * merge_tree_data = dynamic_cast(storage.get())) - sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); + input_sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); } if (options.only_analyze) @@ -1090,7 +1090,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ - executeFetchColumns(from_stage, pipeline, sorting_info, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); + executeFetchColumns(from_stage, pipeline, input_sorting_info, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(options.to_stage)); } @@ -1216,7 +1216,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (!expressions.second_stage && !expressions.need_aggregate && !expressions.has_having) { if (expressions.has_order_by) - executeOrder(pipeline, query_info.sorting_info); + executeOrder(pipeline, query_info.input_sorting_info); if (expressions.has_order_by && query.limitLength()) executeDistinct(pipeline, false, expressions.selected_columns); @@ -1289,7 +1289,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (!expressions.first_stage && !expressions.need_aggregate && !(query.group_by_with_totals && !aggregate_final)) executeMergeSorted(pipeline); else /// Otherwise, just sort. - executeOrder(pipeline, query_info.sorting_info); + executeOrder(pipeline, query_info.input_sorting_info); } /** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT, @@ -1349,7 +1349,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS template void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, - const SortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) + const InputSortingInfoPtr & input_sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) { constexpr bool pipeline_with_processors = std::is_same::value; @@ -1666,7 +1666,7 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); query_info.prewhere_info = prewhere_info; - query_info.sorting_info = sorting_info; + query_info.input_sorting_info = input_sorting_info; BlockInputStreams streams; Pipes pipes; @@ -2248,46 +2248,46 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E }); } -void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info) +void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); + SortDescription output_order_descr = getSortDescription(query, context); const Settings & settings = context.getSettingsRef(); UInt64 limit = getLimitForSorting(query, context); - if (sorting_info) + if (input_sorting_info) { /* Case of sorting with optimization using sorting key. * We have several threads, each of them reads batch of parts in direct * or reverse order of sorting key using one input stream per part * and then merge them into one sorted stream. * At this stage we merge per-thread streams into one. + * If the input is sorted by some prefix of the sorting key required for output, + * we have to finish sorting after the merge. */ - bool need_finish_sorting = (sorting_info->prefix_order_descr.size() < order_descr.size()); + bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); + + UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); + executeMergeSorted(pipeline, input_sorting_info->order_key_prefix_descr, limit_for_merging); + if (need_finish_sorting) { pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, order_descr, limit); + stream = std::make_shared(stream, output_order_descr, limit); }); - } - UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - executeMergeSorted(pipeline, sorting_info->prefix_order_descr, limit_for_merging); - - if (need_finish_sorting) - { pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), sorting_info->prefix_order_descr, - order_descr, settings.max_block_size, limit); + pipeline.firstStream(), input_sorting_info->order_key_prefix_descr, + output_order_descr, settings.max_block_size, limit); } } else { pipeline.transform([&](auto & stream) { - auto sorting_stream = std::make_shared(stream, order_descr, limit); + auto sorting_stream = std::make_shared(stream, output_order_descr, limit); /// Limits on sorting IBlockInputStream::LocalLimits limits; @@ -2303,16 +2303,16 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so /// Merge the sorted blocks. pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), order_descr, settings.max_block_size, limit, + pipeline.firstStream(), output_order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); } } -void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info) +void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query, context); + SortDescription output_order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); const Settings & settings = context.getSettingsRef(); @@ -2322,7 +2322,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP // limits.mode = IBlockInputStream::LIMITS_TOTAL; // limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - if (sorting_info) + if (input_sorting_info) { /* Case of sorting with optimization using sorting key. * We have several threads, each of them reads batch of parts in direct @@ -2331,16 +2331,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP * At this stage we merge per-thread streams into one. */ - bool need_finish_sorting = (sorting_info->prefix_order_descr.size() < order_descr.size()); - - if (need_finish_sorting) - { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) - { - bool do_count_rows = stream_type == QueryPipeline::StreamType::Main; - return std::make_shared(header, order_descr, limit, do_count_rows); - }); - } + bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); if (pipeline.getNumStreams() > 1) { @@ -2348,7 +2339,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP auto transform = std::make_shared( pipeline.getHeader(), pipeline.getNumStreams(), - sorting_info->prefix_order_descr, + input_sorting_info->order_key_prefix_descr, settings.max_block_size, limit_for_merging); pipeline.addPipe({ std::move(transform) }); @@ -2356,11 +2347,17 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP if (need_finish_sorting) { + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) + { + bool do_count_rows = stream_type == QueryPipeline::StreamType::Main; + return std::make_shared(header, output_order_descr, limit, do_count_rows); + }); + pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr { return std::make_shared( - header, sorting_info->prefix_order_descr, - order_descr, settings.max_block_size, limit); + header, input_sorting_info->order_key_prefix_descr, + output_order_descr, settings.max_block_size, limit); }); } @@ -2370,7 +2367,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) { bool do_count_rows = stream_type == QueryPipeline::StreamType::Main; - return std::make_shared(header, order_descr, limit, do_count_rows); + return std::make_shared(header, output_order_descr, limit, do_count_rows); }); /// If there are several streams, we merge them into one @@ -2383,7 +2380,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP return nullptr; return std::make_shared( - header, order_descr, settings.max_block_size, limit, + header, output_order_descr, settings.max_block_size, limit, settings.max_bytes_before_remerge_sort, settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); }); @@ -2808,11 +2805,11 @@ void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { /// Merge streams to one. Use MergeSorting if data was read in sorted order, Union otherwise. - if (query_info.sorting_info) + if (query_info.input_sorting_info) { if (pipeline.stream_with_non_joined_data) throw Exception("Using read in order optimization, but has stream with non-joined data in pipeline", ErrorCodes::LOGICAL_ERROR); - executeMergeSorted(pipeline, query_info.sorting_info->prefix_order_descr, 0); + executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); } else executeUnion(pipeline, {}); @@ -2823,11 +2820,11 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { - if (query_info.sorting_info) + if (query_info.input_sorting_info) { if (pipeline.hasDelayedStream()) throw Exception("Using read in order optimization, but has delayed stream in pipeline", ErrorCodes::LOGICAL_ERROR); - executeMergeSorted(pipeline, query_info.sorting_info->prefix_order_descr, 0); + executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); } const Settings & settings = context.getSettingsRef(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 6d922c6b079..8e548adeaff 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -198,7 +198,7 @@ private: template void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, - const SortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, + const InputSortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); @@ -207,7 +207,7 @@ private: void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); - void executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info); + void executeOrder(Pipeline & pipeline, InputSortingInfoPtr sorting_info); void executeWithFill(Pipeline & pipeline); void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); @@ -226,7 +226,7 @@ private: void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); - void executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info); + void executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info); void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline); diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 3553680b879..eec52788ae0 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -10,7 +10,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index 27772b8fc94..f68ac0bc708 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -142,7 +142,7 @@ bool PredicateExpressionsOptimizer::allowPushDown( if (!subquery || (!settings.enable_optimize_predicate_expression_to_final_subquery && subquery->final()) || subquery->limitBy() || subquery->limitLength() - || subquery->with()) + || subquery->with() || subquery->withFill()) return false; else { diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp index df0946f098a..e3b8a1bc293 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.cpp @@ -29,6 +29,26 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +bool TranslateQualifiedNamesMatcher::Data::unknownColumn(size_t table_pos, const ASTIdentifier & identifier) const +{ + const auto & table = tables[table_pos].first; + auto nested1 = IdentifierSemantic::extractNestedName(identifier, table.table); + auto nested2 = IdentifierSemantic::extractNestedName(identifier, table.alias); + + String short_name = identifier.shortName(); + const Names & column_names = tables[table_pos].second; + for (auto & known_name : column_names) + { + if (short_name == known_name) + return false; + if (nested1 && *nested1 == known_name) + return false; + if (nested2 && *nested2 == known_name) + return false; + } + return !column_names.empty(); +} + bool TranslateQualifiedNamesMatcher::needChildVisit(ASTPtr & node, const ASTPtr & child) { /// Do not go to FROM, JOIN, subqueries. @@ -66,6 +86,13 @@ void TranslateQualifiedNamesMatcher::visit(ASTIdentifier & identifier, ASTPtr &, bool allow_ambiguous = data.join_using_columns.count(short_name); if (IdentifierSemantic::chooseTable(identifier, data.tables, table_pos, allow_ambiguous)) { + if (data.unknownColumn(table_pos, identifier)) + { + String table_name = data.tables[table_pos].first.getQualifiedNamePrefix(false); + throw Exception("There's no column '" + identifier.name + "' in table '" + table_name + "'", + ErrorCodes::UNKNOWN_IDENTIFIER); + } + IdentifierSemantic::setMembership(identifier, table_pos); /// In case if column from the joined table are in source columns, change it's name to qualified. diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index b3718170dda..ddb26b1557c 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -38,6 +38,7 @@ public: bool hasColumn(const String & name) const { return source_columns.count(name); } bool hasTable() const { return !tables.empty(); } bool processAsterisks() const { return hasTable() && has_columns; } + bool unknownColumn(size_t table_pos, const ASTIdentifier & node) const; static std::vector tablesOnly(const std::vector & tables) { diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index b06d786a5f9..6faf27e96e1 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -276,6 +277,18 @@ bool ASTSelectQuery::final() const return table_expression->final; } +bool ASTSelectQuery::withFill() const +{ + if (!orderBy()) + return false; + + for (const auto & order_expression_element : orderBy()->children) + if (order_expression_element->as().with_fill) + return true; + + return false; +} + ASTPtr ASTSelectQuery::array_join_expression_list(bool & is_left) const { diff --git a/dbms/src/Parsers/ASTSelectQuery.h b/dbms/src/Parsers/ASTSelectQuery.h index 38ba12b88a2..81203585462 100644 --- a/dbms/src/Parsers/ASTSelectQuery.h +++ b/dbms/src/Parsers/ASTSelectQuery.h @@ -83,6 +83,7 @@ public: ASTPtr array_join_expression_list() const; const ASTTablesInSelectQueryElement * join() const; bool final() const; + bool withFill() const; void replaceDatabaseAndTable(const String & database_name, const String & table_name); void addTableFunction(ASTPtr & table_function_ptr); diff --git a/dbms/src/Processors/Transforms/TotalsHavingTransform.h b/dbms/src/Processors/Transforms/TotalsHavingTransform.h index e7f28a2ada4..c52aa6c3323 100644 --- a/dbms/src/Processors/Transforms/TotalsHavingTransform.h +++ b/dbms/src/Processors/Transforms/TotalsHavingTransform.h @@ -1,7 +1,6 @@ #include #include -#include namespace DB { @@ -12,6 +11,8 @@ using ArenaPtr = std::shared_ptr; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +enum class TotalsMode; + /** Takes blocks after grouping, with non-finalized aggregate functions. * Calculates total values according to totals_mode. * If necessary, evaluates the expression from HAVING and filters rows. Returns the finalized and filtered blocks. diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index a9b51aebdd4..c6748601c83 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -269,17 +270,41 @@ void StorageDistributedDirectoryMonitor::processFile(const std::string & file_pa void StorageDistributedDirectoryMonitor::readQueryAndSettings( ReadBuffer & in, Settings & insert_settings, std::string & insert_query) const { - UInt64 magic_number_or_query_size; + UInt64 query_size; + readVarUInt(query_size, in); - readVarUInt(magic_number_or_query_size, in); - - if (magic_number_or_query_size == UInt64(DBMS_DISTRIBUTED_SENDS_MAGIC_NUMBER)) + if (query_size == DBMS_DISTRIBUTED_SIGNATURE_EXTRA_INFO) { - insert_settings.deserialize(in); - readVarUInt(magic_number_or_query_size, in); + /// Read extra information. + String extra_info_as_string; + readStringBinary(extra_info_as_string, in); + readVarUInt(query_size, in); + ReadBufferFromString extra_info(extra_info_as_string); + + UInt64 initiator_revision; + readVarUInt(initiator_revision, extra_info); + if (ClickHouseRevision::get() < initiator_revision) + { + LOG_WARNING( + log, + "ClickHouse shard version is older than ClickHouse initiator version. " + << "It may lack support for new features."); + } + + insert_settings.deserialize(extra_info); + + /// Add handling new data here, for example: + /// if (initiator_revision >= DBMS_MIN_REVISION_WITH_MY_NEW_DATA) + /// readVarUInt(my_new_data, extra_info); } - insert_query.resize(magic_number_or_query_size); - in.readStrict(insert_query.data(), magic_number_or_query_size); + else if (query_size == DBMS_DISTRIBUTED_SIGNATURE_SETTINGS_OLD_FORMAT) + { + insert_settings.deserialize(in, SettingsBinaryFormat::OLD); + readVarUInt(query_size, in); + } + + insert_query.resize(query_size); + in.readStrict(insert_query.data(), query_size); } struct StorageDistributedDirectoryMonitor::BatchHeader diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index ee3ebfd9964..61cb10cc38e 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -588,8 +588,19 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: CompressedWriteBuffer compress{out}; NativeBlockOutputStream stream{compress, ClickHouseRevision::get(), block.cloneEmpty()}; - writeVarUInt(UInt64(DBMS_DISTRIBUTED_SENDS_MAGIC_NUMBER), out); - context.getSettingsRef().serialize(out); + /// We wrap the extra information into a string for compatibility with older versions: + /// a shard will able to read this information partly and ignore other parts + /// based on its version. + WriteBufferFromOwnString extra_info; + writeVarUInt(ClickHouseRevision::get(), extra_info); + context.getSettingsRef().serialize(extra_info); + + /// Add new fields here, for example: + /// writeVarUInt(my_new_data, extra_info); + + writeVarUInt(DBMS_DISTRIBUTED_SIGNATURE_EXTRA_INFO, out); + writeStringBinary(extra_info.str(), out); + writeStringBinary(query_string, out); stream.writePrefix(); diff --git a/dbms/src/Storages/Kafka/KafkaSettings.h b/dbms/src/Storages/Kafka/KafkaSettings.h index 6ff62f30411..fc823848ab4 100644 --- a/dbms/src/Storages/Kafka/KafkaSettings.h +++ b/dbms/src/Storages/Kafka/KafkaSettings.h @@ -1,6 +1,6 @@ #pragma once -#include +#include namespace DB @@ -16,16 +16,16 @@ struct KafkaSettings : public SettingsCollection #define LIST_OF_KAFKA_SETTINGS(M) \ - M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.") \ - M(SettingString, kafka_topic_list, "", "A list of Kafka topics.") \ - M(SettingString, kafka_group_name, "", "A group of Kafka consumers.") \ - M(SettingString, kafka_format, "", "The message format for Kafka engine.") \ - M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.") \ - M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine") \ - M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.") \ - M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.") \ - M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block") \ - M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block") + M(SettingString, kafka_broker_list, "", "A comma-separated list of brokers for Kafka engine.", 0) \ + M(SettingString, kafka_topic_list, "", "A list of Kafka topics.", 0) \ + M(SettingString, kafka_group_name, "", "A group of Kafka consumers.", 0) \ + M(SettingString, kafka_format, "", "The message format for Kafka engine.", 0) \ + M(SettingChar, kafka_row_delimiter, '\0', "The character to be considered as a delimiter in Kafka message.", 0) \ + M(SettingString, kafka_schema, "", "Schema identifier (used by schema-based formats) for Kafka engine", 0) \ + M(SettingUInt64, kafka_num_consumers, 1, "The number of consumers per table for Kafka engine.", 0) \ + M(SettingUInt64, kafka_max_block_size, 0, "The maximum block size per table for Kafka engine.", 0) \ + M(SettingUInt64, kafka_skip_broken_messages, 0, "Skip at least this number of broken messages from Kafka topic per block", 0) \ + M(SettingUInt64, kafka_commit_every_batch, 0, "Commit every consumed and handled batch instead of a single commit after writing a whole block", 0) DECLARE_SETTINGS_COLLECTION(LIST_OF_KAFKA_SETTINGS) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d7dc209c707..25d2c0ac2dc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -604,9 +604,9 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( virt_column_names, settings); } - else if (settings.optimize_read_in_order && query_info.sorting_info) + else if (settings.optimize_read_in_order && query_info.input_sorting_info) { - size_t prefix_size = query_info.sorting_info->prefix_order_descr.size(); + size_t prefix_size = query_info.input_sorting_info->order_key_prefix_descr.size(); auto order_key_prefix_ast = data.sorting_key_expr_ast->clone(); order_key_prefix_ast->children.resize(prefix_size); @@ -853,7 +853,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const Settings & settings) const { size_t sum_marks = 0; - SortingInfoPtr sorting_info = query_info.sorting_info; + const InputSortingInfoPtr & input_sorting_info = query_info.input_sorting_info; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); const auto data_settings = data.getSettings(); @@ -1004,9 +1004,9 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( parts.emplace_back(part); } - ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, sorting_info->direction); + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_sorting_info->direction); - if (sorting_info->direction == 1) + if (input_sorting_info->direction == 1) { pipes.emplace_back(std::make_shared( data, part.data_part, max_block_size, settings.preferred_block_size_bytes, @@ -1029,9 +1029,9 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (pipes.size() > 1) { SortDescription sort_description; - for (size_t j = 0; j < query_info.sorting_info->prefix_order_descr.size(); ++j) + for (size_t j = 0; j < input_sorting_info->order_key_prefix_descr.size(); ++j) sort_description.emplace_back(data.sorting_key_columns[j], - sorting_info->direction, 1); + input_sorting_info->direction, 1); for (auto & pipe : pipes) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), sorting_key_prefix_expr)); diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 6db22063841..67e58e6083f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -26,70 +26,70 @@ struct MergeTreeSettings : public SettingsCollection { #define LIST_OF_MERGE_TREE_SETTINGS(M) \ - M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.") \ + M(SettingUInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ \ /** Merge settings. */ \ - M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.") \ - M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).") \ - M(SettingUInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).") \ - M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \ - M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.") \ - M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.") \ - M(SettingUInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"") \ - M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.") \ - M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.") \ + M(SettingUInt64, merge_max_block_size, DEFAULT_MERGE_BLOCK_SIZE, "How many rows in blocks should be formed for merge operations.", 0) \ + M(SettingUInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \ + M(SettingUInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ + M(SettingUInt64, max_replicated_merges_in_queue, 16, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(SettingUInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + M(SettingUInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ + M(SettingUInt64, number_of_free_entries_in_pool_to_execute_mutation, 10, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + M(SettingSeconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ + M(SettingSeconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories.", 0) \ \ /** Inserts settings. */ \ - M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.") \ - M(SettingUInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.") \ - M(SettingUInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.") \ - M(SettingUInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.") \ + M(SettingUInt64, parts_to_delay_insert, 150, "If table contains at least that many active parts in single partition, artificially slow down insert into table.", 0) \ + M(SettingUInt64, parts_to_throw_insert, 300, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ + M(SettingUInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ + M(SettingUInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ \ /** Replication settings. */ \ - M(SettingUInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).") \ - M(SettingUInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.") \ - M(SettingUInt64, max_replicated_logs_to_keep, 10000, "How many records may be in log, if there is inactive replica.") \ - M(SettingUInt64, min_replicated_logs_to_keep, 100, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.") \ - M(SettingSeconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.") \ - M(SettingUInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.") \ - M(SettingUInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.") \ - M(SettingUInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.") \ - M(SettingUInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.") \ - M(SettingFloat, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.") \ - M(SettingUInt64, replicated_max_parallel_fetches, 0, "Limit parallel fetches.") \ - M(SettingUInt64, replicated_max_parallel_fetches_for_table, 0, "Limit parallel fetches for one table.") \ - M(SettingUInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).") \ - M(SettingUInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.") \ - M(SettingUInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.") \ - M(SettingBool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.") \ - M(SettingSeconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.") \ + M(SettingUInt64, replicated_deduplication_window, 100, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ + M(SettingUInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ + M(SettingUInt64, max_replicated_logs_to_keep, 10000, "How many records may be in log, if there is inactive replica.", 0) \ + M(SettingUInt64, min_replicated_logs_to_keep, 100, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ + M(SettingSeconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(SettingUInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + M(SettingUInt64, max_suspicious_broken_parts, 10, "Max broken parts, if more - deny automatic deletion.", 0) \ + M(SettingUInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ + M(SettingUInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ + M(SettingFloat, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \ + M(SettingUInt64, replicated_max_parallel_fetches, 0, "Limit parallel fetches.", 0) \ + M(SettingUInt64, replicated_max_parallel_fetches_for_table, 0, "Limit parallel fetches for one table.", 0) \ + M(SettingUInt64, replicated_max_parallel_fetches_for_host, DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT, "Limit parallel fetches from endpoint (actually pool size).", 0) \ + M(SettingUInt64, replicated_max_parallel_sends, 0, "Limit parallel sends.", 0) \ + M(SettingUInt64, replicated_max_parallel_sends_for_table, 0, "Limit parallel sends for one table.", 0) \ + M(SettingBool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ + M(SettingSeconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ \ /** Check delay of replicas settings. */ \ - M(SettingUInt64, check_delay_period, 60, "Period to check replication delay and compare with other replicas.") \ - M(SettingUInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.") \ - M(SettingUInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.") \ - M(SettingUInt64, min_relative_delay_to_yield_leadership, 120, "Minimal delay from other replicas to yield leadership. Here and further 0 means unlimited.") \ - M(SettingUInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.") \ - M(SettingUInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.") \ - M(SettingUInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.") \ - M(SettingUInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.") \ - M(SettingUInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.") \ + M(SettingUInt64, check_delay_period, 60, "Period to check replication delay and compare with other replicas.", 0) \ + M(SettingUInt64, cleanup_delay_period, 30, "Period to clean old queue logs, blocks hashes and parts.", 0) \ + M(SettingUInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ + M(SettingUInt64, min_relative_delay_to_yield_leadership, 120, "Minimal delay from other replicas to yield leadership. Here and further 0 means unlimited.", 0) \ + M(SettingUInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ + M(SettingUInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ + M(SettingUInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ + M(SettingUInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * DEFAULT_MERGE_BLOCK_SIZE, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \ + M(SettingUInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \ \ /** Compatibility settings */ \ - M(SettingBool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.") \ - M(SettingBool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.") \ - M(SettingBool, use_minimalistic_part_header_in_zookeeper, false, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.") \ - M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.") \ - M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).") \ - M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).") \ - M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.") \ - M(SettingBool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.") \ - M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)") \ - M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity") \ - M(SettingMaxThreads, max_part_loading_threads, 0, "The number of theads to load data parts at startup.") \ - M(SettingMaxThreads, max_part_removal_threads, 0, "The number of theads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).") \ - M(SettingUInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.") \ - M(SettingString, storage_policy, "default", "Name of storage disk policy") + M(SettingBool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \ + M(SettingBool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \ + M(SettingBool, use_minimalistic_part_header_in_zookeeper, false, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \ + M(SettingUInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \ + M(SettingUInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ + M(SettingUInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ + M(SettingInt64, merge_with_ttl_timeout, 3600 * 24, "Minimal time in seconds, when merge with TTL can be repeated.", 0) \ + M(SettingBool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ + M(SettingBool, write_final_mark, 1, "Write final mark after end of column (0 - disabled, do nothing if index_granularity_bytes=0)", 0) \ + M(SettingBool, enable_mixed_granularity_parts, 0, "Enable parts with adaptive and non adaptive granularity", 0) \ + M(SettingMaxThreads, max_part_loading_threads, 0, "The number of theads to load data parts at startup.", 0) \ + M(SettingMaxThreads, max_part_removal_threads, 0, "The number of theads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).", 0) \ + M(SettingUInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ + M(SettingString, storage_policy, "default", "Name of storage disk policy", 0) DECLARE_SETTINGS_COLLECTION(LIST_OF_MERGE_TREE_SETTINGS) diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 74e28ede679..23dcf9e3ddf 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -34,18 +34,18 @@ struct FilterInfo bool do_remove_column = false; }; -struct SortingInfo +struct InputSortingInfo { - SortDescription prefix_order_descr; + SortDescription order_key_prefix_descr; int direction; - SortingInfo(const SortDescription & prefix_order_descr_, int direction_) - : prefix_order_descr(prefix_order_descr_), direction(direction_) {} + InputSortingInfo(const SortDescription & order_key_prefix_descr_, int direction_) + : order_key_prefix_descr(order_key_prefix_descr_), direction(direction_) {} }; using PrewhereInfoPtr = std::shared_ptr; using FilterInfoPtr = std::shared_ptr; -using SortingInfoPtr = std::shared_ptr; +using InputSortingInfoPtr = std::shared_ptr; struct SyntaxAnalyzerResult; using SyntaxAnalyzerResultPtr = std::shared_ptr; @@ -62,7 +62,7 @@ struct SelectQueryInfo PrewhereInfoPtr prewhere_info; - SortingInfoPtr sorting_info; + InputSortingInfoPtr input_sorting_info; /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index deaea288e7b..f74c81750a1 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/tests/integration/test_old_versions_client/__init__.py b/dbms/tests/integration/test_old_versions/__init__.py similarity index 100% rename from dbms/tests/integration/test_old_versions_client/__init__.py rename to dbms/tests/integration/test_old_versions/__init__.py diff --git a/dbms/tests/integration/test_old_versions/configs/config.d/test_cluster.xml b/dbms/tests/integration/test_old_versions/configs/config.d/test_cluster.xml new file mode 100644 index 00000000000..56d53d63d78 --- /dev/null +++ b/dbms/tests/integration/test_old_versions/configs/config.d/test_cluster.xml @@ -0,0 +1,13 @@ + + + + + 1 + + node_new + 9000 + + + + + diff --git a/dbms/tests/integration/test_old_versions/test.py b/dbms/tests/integration/test_old_versions/test.py new file mode 100644 index 00000000000..d77b4af016a --- /dev/null +++ b/dbms/tests/integration/test_old_versions/test.py @@ -0,0 +1,73 @@ +import time +import os +import pytest + +from helpers.cluster import ClickHouseCluster +from multiprocessing.dummy import Pool +from helpers.client import QueryRuntimeException, QueryTimeoutExceedException +from helpers.test_tools import assert_eq_with_retry + + +cluster = ClickHouseCluster(__file__) +node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True, config_dir="configs") +node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True, config_dir="configs") +node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True, config_dir="configs") +node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True, config_dir="configs") +node19_11 = cluster.add_instance('node19_11', image='yandex/clickhouse-server:19.11.13.74', with_installed_binary=True, config_dir="configs") +node19_13 = cluster.add_instance('node19_13', image='yandex/clickhouse-server:19.13.7.57', with_installed_binary=True, config_dir="configs") +node19_16 = cluster.add_instance('node19_16', image='yandex/clickhouse-server:19.16.2.2', with_installed_binary=True, config_dir="configs") +old_nodes = [node18_14, node19_1, node19_4, node19_8, node19_11, node19_13, node19_16] +new_node = cluster.add_instance('node_new') + + +def query_from_one_node_to_another(client_node, server_node, query): + client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query {!r}".format(server_node.name, query)]) + + +@pytest.fixture(scope="module") +def setup_nodes(): + try: + cluster.start() + + for n in old_nodes + [new_node]: + n.query('''CREATE TABLE test_table (id UInt32, value UInt64) ENGINE = MergeTree() ORDER BY tuple()''') + + for n in old_nodes: + n.query('''CREATE TABLE dist_table AS test_table ENGINE = Distributed('test_cluster', 'default', 'test_table')''') + + yield cluster + finally: + cluster.shutdown() + + +def test_client_is_older_than_server(setup_nodes): + server = new_node + for i, client in enumerate(old_nodes): + query_from_one_node_to_another(client, server, "INSERT INTO test_table VALUES (1, {})".format(i)) + + for client in old_nodes: + query_from_one_node_to_another(client, server, "SELECT COUNT() FROM test_table") + + assert server.query("SELECT COUNT() FROM test_table WHERE id=1") == str(len(old_nodes)) + "\n" + + +def test_server_is_older_than_client(setup_nodes): + client = new_node + for i, server in enumerate(old_nodes): + query_from_one_node_to_another(client, server, "INSERT INTO test_table VALUES (2, {})".format(i)) + + for server in old_nodes: + query_from_one_node_to_another(client, server, "SELECT COUNT() FROM test_table") + + for server in old_nodes: + assert server.query("SELECT COUNT() FROM test_table WHERE id=2") == "1\n" + + +def test_distributed_query_initiator_is_older_than_shard(setup_nodes): + distributed_query_initiator_old_nodes = [node18_14, node19_13, node19_16] + shard = new_node + for i, initiator in enumerate(distributed_query_initiator_old_nodes): + initiator.query("INSERT INTO dist_table VALUES (3, {})".format(i)) + + assert_eq_with_retry(shard, "SELECT COUNT() FROM test_table WHERE id=3", str(len(distributed_query_initiator_old_nodes))) + assert_eq_with_retry(initiator, "SELECT COUNT() FROM dist_table WHERE id=3", str(len(distributed_query_initiator_old_nodes))) diff --git a/dbms/tests/integration/test_old_versions_client/test.py b/dbms/tests/integration/test_old_versions_client/test.py deleted file mode 100644 index 59a4aaca31c..00000000000 --- a/dbms/tests/integration/test_old_versions_client/test.py +++ /dev/null @@ -1,51 +0,0 @@ -import time -import pytest - -from helpers.cluster import ClickHouseCluster -from multiprocessing.dummy import Pool -from helpers.client import QueryRuntimeException, QueryTimeoutExceedException - -from helpers.test_tools import assert_eq_with_retry -cluster = ClickHouseCluster(__file__) -node18_14 = cluster.add_instance('node18_14', image='yandex/clickhouse-server:18.14.19', with_installed_binary=True) -node19_1 = cluster.add_instance('node19_1', image='yandex/clickhouse-server:19.1.16', with_installed_binary=True) -node19_4 = cluster.add_instance('node19_4', image='yandex/clickhouse-server:19.4.5.35', with_installed_binary=True) -node19_6 = cluster.add_instance('node19_6', image='yandex/clickhouse-server:19.6.3.18', with_installed_binary=True) -node19_8 = cluster.add_instance('node19_8', image='yandex/clickhouse-server:19.8.3.8', with_installed_binary=True) -node_new = cluster.add_instance('node_new') - -@pytest.fixture(scope="module") -def setup_nodes(): - try: - cluster.start() - for n in (node18_14, node19_1, node19_4, node19_6, node19_8, node_new): - n.query('''CREATE TABLE test_table (id UInt32, value UInt64) ENGINE = MergeTree() ORDER BY tuple()''') - - yield cluster - finally: - cluster.shutdown() - - -def query_from_one_node_to_another(client_node, server_node, query): - client_node.exec_in_container(["bash", "-c", "/usr/bin/clickhouse client --host {} --query '{}'".format(server_node.name, query)]) - -def test_client_from_different_versions(setup_nodes): - old_nodes = (node18_14, node19_1, node19_4, node19_6, node19_8) - # from new to old - for n in old_nodes: - query_from_one_node_to_another(node_new, n, "INSERT INTO test_table VALUES (1, 1)") - - for n in old_nodes: - query_from_one_node_to_another(node_new, n, "SELECT COUNT() FROM test_table") - - for n in old_nodes: - assert n.query("SELECT COUNT() FROM test_table") == "1\n" - - # from old to new - for i, n in enumerate(old_nodes): - query_from_one_node_to_another(n, node_new, "INSERT INTO test_table VALUES ({i}, {i})".format(i=i)) - - for n in old_nodes: - query_from_one_node_to_another(n, node_new, "SELECT COUNT() FROM test_table") - - assert node_new.query("SELECT COUNT() FROM test_table") == str(len(old_nodes)) + "\n" diff --git a/dbms/tests/performance/modulo.xml b/dbms/tests/performance/modulo.xml new file mode 100644 index 00000000000..931b160ea00 --- /dev/null +++ b/dbms/tests/performance/modulo.xml @@ -0,0 +1,17 @@ + + loop + + + + 10 + + + + + + + + SELECT number % 128 FROM numbers(300000000) FORMAT Null + SELECT number % 255 FROM numbers(300000000) FORMAT Null + SELECT number % 256 FROM numbers(300000000) FORMAT Null + diff --git a/dbms/tests/queries/0_stateless/00930_arrayIntersect.reference b/dbms/tests/queries/0_stateless/00930_arrayIntersect.reference index 31d8d92cd89..15cd44cbc21 100644 --- a/dbms/tests/queries/0_stateless/00930_arrayIntersect.reference +++ b/dbms/tests/queries/0_stateless/00930_arrayIntersect.reference @@ -46,3 +46,6 @@ [] [] [] +- +[] +[] diff --git a/dbms/tests/queries/0_stateless/00930_arrayIntersect.sql b/dbms/tests/queries/0_stateless/00930_arrayIntersect.sql index e5eee83b337..64505fe4180 100644 --- a/dbms/tests/queries/0_stateless/00930_arrayIntersect.sql +++ b/dbms/tests/queries/0_stateless/00930_arrayIntersect.sql @@ -25,3 +25,6 @@ select arrayIntersect([], []) from array_intersect order by arr; drop table if exists array_intersect; +select '-'; +select arrayIntersect([-100], [156]); +select arrayIntersect([1], [257]); \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00974_full_outer_join.sql b/dbms/tests/queries/0_stateless/00974_full_outer_join.sql index fda9d70e444..c30d933dc41 100644 --- a/dbms/tests/queries/0_stateless/00974_full_outer_join.sql +++ b/dbms/tests/queries/0_stateless/00974_full_outer_join.sql @@ -1,7 +1,7 @@ SELECT q0.dt, q0.cnt, - q0.cnt2 + q1.cnt2 FROM ( SELECT diff --git a/dbms/tests/queries/0_stateless/01034_order_by_pk_prefix.reference b/dbms/tests/queries/0_stateless/01034_order_by_pk_prefix.reference new file mode 100644 index 00000000000..81410243ffb --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_order_by_pk_prefix.reference @@ -0,0 +1,30 @@ +0 0 +0 0 +0 0 +1 1 +1 1 +1 1 +2 2 +2 4 +2 4 +3 3 +3 9 +3 9 +4 16 +4 16 +4 4 +5 25 +5 25 +5 5 +6 36 +6 36 +6 6 +7 49 +7 49 +7 7 +8 64 +8 64 +8 8 +9 81 +9 81 +9 9 diff --git a/dbms/tests/queries/0_stateless/01034_order_by_pk_prefix.sql b/dbms/tests/queries/0_stateless/01034_order_by_pk_prefix.sql new file mode 100644 index 00000000000..cffac819f10 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_order_by_pk_prefix.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test_table; + +CREATE TABLE test_table (n Int32, s String) +ENGINE = MergeTree() PARTITION BY n % 10 ORDER BY n; + +INSERT INTO test_table SELECT number, toString(number) FROM system.numbers LIMIT 100; +INSERT INTO test_table SELECT number, toString(number * number) FROM system.numbers LIMIT 100; +INSERT INTO test_table SELECT number, toString(number * number) FROM system.numbers LIMIT 100; + +SELECT * FROM test_table ORDER BY n, s LIMIT 30; + +DROP TABLE test_table; diff --git a/dbms/tests/queries/0_stateless/01034_unknown_qualified_column_in_join.reference b/dbms/tests/queries/0_stateless/01034_unknown_qualified_column_in_join.reference new file mode 100644 index 00000000000..7c897f126ee --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_unknown_qualified_column_in_join.reference @@ -0,0 +1 @@ +1 3 diff --git a/dbms/tests/queries/0_stateless/01034_unknown_qualified_column_in_join.sql b/dbms/tests/queries/0_stateless/01034_unknown_qualified_column_in_join.sql new file mode 100644 index 00000000000..35f6d07d9b6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_unknown_qualified_column_in_join.sql @@ -0,0 +1,3 @@ +SELECT l.c FROM (SELECT 1 AS a, 2 AS b) AS l join (SELECT 2 AS b, 3 AS c) AS r USING b; -- { serverError 47 } +SELECT r.a FROM (SELECT 1 AS a, 2 AS b) AS l join (SELECT 2 AS b, 3 AS c) AS r USING b; -- { serverError 47 } +SELECT l.a, r.c FROM (SELECT 1 AS a, 2 AS b) AS l join (SELECT 2 AS b, 3 AS c) AS r USING b; diff --git a/dbms/tests/queries/0_stateless/01034_with_fill_and_push_down_predicate.reference b/dbms/tests/queries/0_stateless/01034_with_fill_and_push_down_predicate.reference new file mode 100644 index 00000000000..3455f766bfb --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_with_fill_and_push_down_predicate.reference @@ -0,0 +1,8 @@ +2019-11-14 22:15:00 +2019-11-14 22:30:00 +2019-11-14 22:45:00 +2019-11-14 23:00:00 +2019-11-14 23:15:00 +2019-11-14 23:30:00 +2019-11-14 23:45:00 +2019-11-15 00:00:00 diff --git a/dbms/tests/queries/0_stateless/01034_with_fill_and_push_down_predicate.sql b/dbms/tests/queries/0_stateless/01034_with_fill_and_push_down_predicate.sql new file mode 100644 index 00000000000..718e8f292b2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_with_fill_and_push_down_predicate.sql @@ -0,0 +1 @@ +SELECT * FROM ( SELECT date_time FROM ( SELECT toDateTime('2019-11-14 22:15:00') AS date_time UNION ALL SELECT toDateTime('2019-11-15 01:15:00') AS date_time ) ORDER BY date_time WITH fill step 900 ) WHERE date_time < toDateTime('2019-11-15 00:15:00') diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index b582ab447d2..b046cfd6c6e 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -1,17 +1,23 @@ # Command-line Client -To work from the command line, you can use `clickhouse-client`: +ClickHouse provides the native command line client `clickhouse-client`. The client supports command-line options and configuration files. For more information, see "[Configuring](#interfaces_cli_configuration)". + +[Install](../getting_started/index.md) it by the `clickhouse-client` package and run it by the command `clickhouse-client`. ```bash $ clickhouse-client -ClickHouse client version 0.0.26176. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.26176. +ClickHouse client version 19.17.1.1579 (official build). +Connecting to localhost:9000 as user default. +Connected to ClickHouse server version 19.17.1 revision 54428. :) ``` -The client supports command-line options and configuration files. For more information, see "[Configuring](#interfaces_cli_configuration)". +Different versions of client and server are compatible, but some features may be disabled for older clients. We don't recommend using different versions of the client and the server app. When you try to use the client of the older version, then the server, `clickhouse-client` displays the message: + +``` +ClickHouse client version is older than ClickHouse server. It may lack support for new features. +``` ## Usage {#cli_usage} diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index a98c112a1c0..c2418263fef 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -76,6 +76,7 @@ Features: [clickhouse-cli](https://github.com/hatarist/clickhouse-cli) is an alternative command line client for ClickHouse, written in Python 3. Features: + - Autocompletion. - Syntax highlighting for the queries and data output. - Pager support for the data output. @@ -87,6 +88,18 @@ Features: ## Commercial +### Holistics Software + +[Holistics](https://www.holistics.io/) was listed by Gartner's Frontrunners in 2019 as one of the top 2 highest ranked business intelligence tools globally for usability. Holistics is a full-stack data platform and business intelligence tool for setting up your analytics processes, built on SQL. + +Features: + +- Automated email, Slack and Google Sheet schedules of reports. +- Powerful SQL editor with visualizations, version control, auto-completion, reusable query components and dynamic filters. +- Embedded analytics of reports and dashboards via iframe. +- Data preparation and ETL capabilities. +- SQL data modeling support for relational mapping of data. + ### DataGrip [DataGrip](https://www.jetbrains.com/datagrip/) is a database IDE from JetBrains with dedicated support for ClickHouse. It is also embedded into other IntelliJ-based tools: PyCharm, IntelliJ IDEA, GoLand, PhpStorm and others. diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md new file mode 100644 index 00000000000..22442cab76c --- /dev/null +++ b/docs/ru/extended_roadmap.md @@ -0,0 +1,1324 @@ +# Планы разработки ClickHouse 2020. + +Здесь собраны важные задачи на 2020 год. Также многие из них присутствуют в GitHub Issues или (редко) в Яндекс.Трекере, а также (очень редко) в Яндекс.Goals. Мы предпочитаем использовать Яндекс.Трекер только для секретных задач, а все публичные задачи размещать в GitHub Issues для того, чтобы сделать разработку более открытой наружу. В любом случае, для работы над задачами, таск-трекеры уступают в удобстве plaintext файлу. + +Классификация задач условная и за её основу взята известная [классификация животных](https://ru.wikipedia.org/wiki/%D0%9A%D0%BB%D0%B0%D1%81%D1%81%D0%B8%D1%84%D0%B8%D0%BA%D0%B0%D1%86%D0%B8%D1%8F_%D0%B6%D0%B8%D0%B2%D0%BE%D1%82%D0%BD%D1%8B%D1%85_(%D0%91%D0%BE%D1%80%D1%85%D0%B5%D1%81)). + +Примечение по терминологии. В тексте иногда используется понятие "просранная задача". Это - технический термин, который лишён оскорбительной коннотации и обозначает просранную задачу. + + +## 1. Хранение данных, индексация. + +### 1.1. Индексы по z-Order curve, normalized z-Order curve. + +Задача также относится к категории "17. Работа с географическими данными", так как geoHash - это частный случай z-Order curve. +Также связана с задачей 24.27 для нечёткого поиска полудубликатов строк, так как позволит индексировать min-hash. +Задача "normalized z-Order curve" в перспективе может быть полезна для БК и Метрики, так как позволяет смешивать OrderID и PageID и избежать дублирования данных. +В задаче также вводится способ индексации путём обращения функции нескольких аргументов на интервале, что имеет смысл для дальнейшего развития. + +Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но задача прокрастинирована из-за задачи 25.9. Сейчас оказалось, что задача 25.9 тоже не готова. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро выйдет на работу стажёром на пол ставки и сможет помочь её доделать. + +### 1.2. Wait-free каталог баз данных. + +Делает [Александр Токмаков](https://github.com/tavplubix), первый рабочий вариант в декабре 2019. Нужно для DataLens и Яндекс.Метрики. + +### 1.3. Неблокирующие ALTER. + +И полностью immutable куски. Делает [Александр Сапин](https://github.com/alesapin). Готов приступить к задаче в конце ноября 2019. Нужно для Яндекс.Метрики. + +### 1.4. Нетранзитивные ALTER столбцов. + +Требует 1.3. Будет делать [Александр Сапин](https://github.com/alesapin). + +### 1.5. ALTER RENAME COLUMN. + +Требует 1.3. Будет делать [Александр Сапин](https://github.com/alesapin). + +### 1.6. Полиморфные куски данных. + +Делает [Антон Попов](https://github.com/CurtizJ), первый рабочий вариант в декабре. Пререквизит чтобы снизить сложность мелких INSERT, что в свою очередь нужно для 1.12, иначе задача 1.12 не сможет нормально работать. Особенно нужно для Яндекс.Облака. + +### 1.7. Буферизация и WAL в MergeTree. + +Требует 1.6. + +### 1.8. Перенос между разделами по TTL. + +Делает [Владимир Чеботарёв](https://github.com/excitoon), Altinity. + +### 1.9. Использование TTL для прореживания данных. + +В очереди. + +### 1.10. Пережатие старых данных в фоне. + +Будет делать Кирилл Барухов, ВШЭ, экспериментальная реализация к весне 2020. Нужно для Яндекс.Метрики. + +### 1.11. Виртуальная файловая система. + +Нужно для Яндекс.Облака. Делает Александр Бурмак, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс. + +### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS. + +Нужно для Яндекс.Облака. Требует 1.11. Желательно 1.6 и 1.18. +Делает Александр Бурмак, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс. + +### 1.13. Ускорение запросов с FINAL. + +Требует 2.1. Делает [Николай Кочетов](https://github.com/KochetovNicolai). Нужно для Яндекс.Метрики. + +### 1.14. Не писать столбцы, полностью состоящие из нулей. + +В очереди. Простая задача, является небольшим пререквизитом для потенциальной поддержки полуструктурированных данных. + +### 1.15. Возможность иметь разный первичный ключ в разных кусках. + +Сложная задача, только после 1.3. + +### 1.16. Несколько физических представлений для одного куска данных. + +Сложная задача, только после 1.3 и 1.6. Позволяет компенсировать 21.20. + +### 1.17. Несколько сортировок для одной таблицы. + +Сложная задача, только после 1.3 и 1.6. + +### 1.18. Отдельное хранение файлов кусков. + +Требует 1.3 и 1.6. Полная замена hard links на sym links, что будет лучше для 1.12. + + +## 2. Крупные рефакторинги. + +Для обоснования необходимости смотрите ссылки в описании других задач. + +### 2.1. Переделка конвейера выполнения запросов на Processors. + +Делает [Николай Кочетов](https://github.com/KochetovNicolai). Финальная стадия разработки. Включение по-умолчанию в конце декабря 2019. Удаление старого кода в начале 2020. + +### 2.2. Инфраструктура событий/метрик/ограничений/квот/трассировки. + +В очереди. https://gist.github.com/alexey-milovidov/d62d73222d83b9319dc519cbb13aeff6 + +### 2.3. Перенос столбцового ser/de из DataType в Column. + +В очереди. + +### 2.4. Перевод LowCardinality из DataType в Column. Добавление ColumnSparse. + +Требует 2.3. + +### 2.5. Версионирование состояний агрегатных функций. + +В очереди. + +### 2.6. Правая часть IN как тип данных. Выполнение IN в виде скалярного подзапроса. + +Требует 2.1. + +### 2.7. Нормализация Context. + +В очереди. + +### 2.8. Декларативный парсер запросов. + +Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но за последний год никаких поползновений не видно. + +### 2.9. Логгировние в format-стиле. + +В задаче заинтересован [Александр Кузьменков](https://github.com/akuzm). Нет прогресса. + +### 2.10. Запрашивать у таблиц не столбцы, а срезы. + +В очереди. + +### 2.11. Разбирательство и нормализация функциональности для bitmap. + +В очереди. + +### 2.12. Декларативные сигнатуры функций. + +Задачу делает Алексей Миловидов. Прогресс 50% и разработка временно приостановлена. + +### 2.13. Каждая функция в отдельном файле. + +Задачу делает Алексей Миловидов. Прогресс 80%. Потребуется помощь других разработчиков. + +### 2.14. Все функции с состоянием переделать на FunctionBuilder. + +Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлимо. + +### 2.15. Функция subscribe для IStorage. + +Для нормализации работы materialized views поверх Merge, Distributed, Kafka. + + +## 3. Документация. + +Здесь задачи только по инфраструктуре документации. + +### 3.1. Перенос документации по функциям в код. + +Требует 2.12 и 2.13. + +### 3.2. Перенос однородных частей документации в код. + +Требует 3.1. + +### 3.3. Исправить идиотский поиск по документации. + +Не делает [Иван Блинков](https://github.com/blinkov/), и есть подозрения, что он не в состоянии выполнить эту задачу. Сам сайт документации основан на треш-технологиях, которые трудно исправить. + +### 3.4. Добавить японский язык в документацию. + +Эту задачу сделает [Иван Блинков](https://github.com/blinkov/), до конца ноября 2019. + + +## 4. Сетевое взаимодействие. + +### 4.1. Уменьшение числа потоков при распределённых запросах. + +[Никита Лапков](https://github.com/laplab), весна 2020. + +### 4.2. Спекулятивное выполнение запросов на нескольких репликах. + +[Никита Лапков](https://github.com/laplab), весна 2020. Нужно для Яндекс.Метрики. Требует 4.1. + +### 4.3. Ограничение числа одновременных скачиваний с реплик. + +Изначально делал Олег Алексеенков, но решение оказалось неудачным, хотя там не так уж много доделывать. + +### 4.4. Ограничение сетевой полосы при репликации. + +### 4.5. Возможность продолжить передачу куска данных при репликации после сбоя. + +### 4.6. p2p передача для GLOBAL подзапросов. + +### 4.7. Ленивая загрузка множеств для IN и JOIN с помощью k/v запросов. + +### 4.8. Разделить background pool для fetch и merge. + +В очереди. Исправить проблему, что восстанавливающаяся реплика перестаёт мержить. Частично компенсируется 4.3. + + +## 5. Операции. + +### 5.1. Разделение задач на более мелкие куски в clickhouse-copier. + +Нужно для Метрики, в очереди, но исполнитель не назначен, есть шанс успеть в 2019. + +### 5.2. Автонастройка лимита на оперативку и размера кэшей. + +### 5.3. Встроенная ручка для Prometheus и, возможно, Solomon. + +Простая задача. + +### 5.4. Opt-in сообщать в клиенте, если вышла новая версия. + +### 5.5. LTS релизы. + +Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Задача полностью обречена на провал с практической точки зрения, но пользователям важен сам факт того, что она будет, а мы уважаем чувства пользователей. Первой LTS версией уже стала версия 19.14. + + +## 6. Инструментирование. + +### 6.1. Исправления сэмплирующего профайлера запросов. + +[Никита Лапков](https://github.com/laplab), ноябрь 2019. Может не сделать, тогда будет делать Алексей Миловидов. + +### 6.2. Добавление memory profiler. + +Сравнительно простая задача, но только для опытных разработчиков. Нужна всем. + +### 6.3. Учёт оперативки total расширить не только на запросы. + +Исправление долгоживущей проблемы с дрифтом учёта оперативки. Нужна для Метрики и БК. + +### 6.4. Поддержка perf events как метрик запроса. + +Делает Андрей Скобцов, ВШЭ. + +### 6.5. Эксперименты с LLVM X-Ray. + +Требует 2.2. + +### 6.6. Стек трейс для любых исключений. + +Сейчас есть стек трейс для почти всех, но не всех исключений. Требует 7.4. + +### 6.7. Таблица system.stack_trace. + +Сравнительно простая задача, но только для опытных разработчиков. + +### 6.8. Таблица system.crashes. + +Сравнительно простая задача, но только для опытных разработчиков. + +### 6.9. Отправлять информацию клиенту, если сервер падает по сигналу. + +### 6.10. Сбор общих системных метрик. + + +## 7. Сопровождение разработки. + +### 7.1. ICU в submodules. + +[Иван Лежанкин](https://github.com/abyss7). +Добавление в submodules также нужно для Аркадии (7.26). + +### 7.2. LLVM в submodules. + +Уже добавлено, но старой версии, и поэтому не используется. Надо обновить. +Георгий Кондратьев. Возможно, он не сможет сделать эту задачу, тогда будет делать Алексей Миловидов. + +### 7.3. Обновление Poco. + +Алексанр Кузьменков. + +### 7.4. Включить libc++, libc++-abi при сборке с gcc. + +Сейчас включено только при сборке с clang, но продакшен сборка использует gcc. +Требует 7.2 и, возможно, 7.1 (только в случае новой версии ICU). + +### 7.5. Начать публиковать LTS релизы. + +[Александр Сапин](https://github.com/alesapin). + +### 7.6. Правильный статистический тест для comparison mode в clickhouse-performance-test. + +Задачу начал делать Дмитрий Рубашкин. Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm). + +### 7.7. Доделать тесты под MSan. + +Уже есть ASan, TSan, UBSan. Не хватает тестов под MSan. Они уже добавлены в CI, но не проходят. +[Александр Кузьменков](https://github.com/akuzm). + +### 7.8. Добавить clang-tidy. + +Уже есть PVS-Studio, но этого недостаточно. + +### 7.9. Проверки на стиль имён с помощью clang-tidy. + +### 7.10. Включение UBSan и MSan в интеграционных тестах. + +UBSan включен в функциональных тестах, но не включен в интеграционных тестах. Требует 7.7. + +### 7.11. Включение *San в unit тестах. + +У нас мало unit тестов по сравнению с функциональными тестами и их использование не обязательно. Но они всё-равно важны и нет причин не запускать их под всеми видами sanitizers. + +### 7.12. Показывать тестовое покрытие нового кода в PR. + +Пока есть просто показ тестового покрытия всего кода. + +### 7.13. Включение аналога -Weverything в gcc. + +Мы используем -Wall -Wextra -Weverything -Werror. +При сборке с clang, -Weverything уже включено. Но в gcc есть уникальные warning-и, отсутствующие в clang. +Wolf Kreuzerkrieg. Возможно, его уже не интересует эта задача. + +### 7.14. Альтернатива для readline и libedit. + +Тагир Кускаров, ВШЭ. Посмотрим на https://github.com/AmokHuginnsson/replxx + +### 7.15. Замена libressl обратно на openssl. + +Поводом использования libressl послужило желание Константина podshumok Игнатова из QRator и то, что тогда openssl был опозорен и libressl считалась адекватной альтернативой. Но сейчас ситуация изменилась - openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно. + +### 7.16. tzdata внутри бинарника. + +Как в Аркадии, но только в качестве fallback. + +### 7.17. Доделать tgz пакеты. + +Уже давно собираются универсальные tgz пакеты, но по нелепой случайности из них исчез install скрипт. +[Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. + +### 7.18.1. Доделать бинарники под Mac. + +Уже есть автосборка бинарников под Mac на каждый коммит и PR, но с недостатками. +[Иван Лежанкин](https://github.com/abyss7). Требует 7.1, 7.2. Рекомендуется 7.14. Сейчас не хватает по крайней мере SSL и ICU. Нужно для Яндекс.Облака. + +### 7.18. Поместить ссылку на собранные бинарники под Mac на сайт. + +Сейчас людям приходится делать несколько кликов, чтобы их скачать. +[Иван Лежанкин](https://github.com/abyss7) или [Александр Сапин](https://github.com/alesapin). + +### 7.19. Доделать (проверить) автосборку под AArch64. + +Проверяем, что работает на Cavium и на Raspberry Pi. +[Иван Лежанкин](https://github.com/abyss7). + +### 7.20. Автосборка для FreeBSD x86_64. + +[Иван Лежанкин](https://github.com/abyss7). + +### 7.21. Автосборка для Linux ppc64. + +[Иван Лежанкин](https://github.com/abyss7). + +### 7.22. Дэшборд для pull requests. + +Дарья Петрова, УрФУ. + +### 7.23. Функции для fuzzing. + +Андрей Некрашевич, ВШЭ. + +### 7.24. Fuzzing лексера и парсера запросов; кодеков и форматов. + +Андрей Некрашевич, ВШЭ. + +### 7.25. Синхронизация релизов в Аркадию. + +Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в Яндекс.Морду, но обещает продолжать синхронизацию. +Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим Ахмедов должен исправить регрессию производительности в анализе индекса. + +### 7.26. Побайтовая идентичность репозитория с Аркадией. + +Дмитрий Копылов, Анастасия Сидоровская, Александр Артамонов, Илья Зубков. Фактически никто ничего не делает. + +### 7.27. Запуск автотестов в Аркадии. + +Требует 7.26. + +### 7.29. Опции clickhouse install, stop, start вместо postinst, init.d, systemd скриптов. + +Низкий приоритет. + +### 7.30. Возможность переключения бинарных файлов на продакшене без выкладки пакетов. + +Низкий приоритет. + +### 7.31. Зеркалирование нагрузки между серверами. + +В очереди. Нужно для Яндекс.Метрики. + +### 7.32. Обфускация продакшен запросов. + +Роман Ильговский. Нужно для Яндекс.Метрики. + +### 7.33. Выкладывать патч релизы в репозиторий автоматически. + +[Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. + +### 7.34. Бэкпортировать bugfix автоматически. + +[Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. + +### 7.35. Начальные правила для авто-merge. + +Зелёные проверки и два ревью. +[Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. + +### 7.36. Понятие доверенных контрибьюторов. + +Контрибьюторы, у которых есть 5 померженных PR. Для их новых PR автотесты запускаются сразу. +[Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. + +### 7.37. Разобраться с repo.yandex.ru. + +Есть жалобы на скорость загрузки. Подозрение, что repo.yandex.ru не является нормальным CDN. Отсутствует простой доступ к мониторингу и логам. +Очень редко бывает нужно удалить пакет, но сделать это можно только через Аркадия Шейна. + + +## 8. Интеграция с внешними системами. + +### 8.1. Поддержка ALTER MODIFY SETTING для Kafka. + +[Иван Лежанкин](https://github.com/abyss7). + +### 8.2. Поддержка Mongo Atlas URI. + +[Александр Кузьменков](https://github.com/akuzm). + +### 8.3. Доработки globs (правильная поддержка диапазонов, уменьшение числа одновременных stream-ов). + +[Ольга Хвостикова](https://github.com/stavrolia). + +### 8.4. Унификация File, HDFS, S3 под URL. + +### 8.5. Аутентификация в S3. + +[Владимир Чеботарёв](https://github.com/excitoon), Altinity. + +### 8.6. Kerberos аутентификация для HDFS и Kafka. + +В очереди, возможно, [Иван Лежанкин](https://github.com/abyss7). + +### 8.7. Исправление мелочи HDFS на очень старых ядрах Linux. + +В ядрах 2.6 отсутствует один системный вызов, который библиотека hdfs3 использует без необходимости. +Тривиально, но исполнителя ещё нет. + +### 8.8. Поддержка виртуальных столбцов с именем файла и путём. + +[Ольга Хвостикова](https://github.com/stavrolia). + +### 8.9. Поддержка сжатых файлов (gz, bz) на чтение и запись. + +### 8.10. Запись в табличную функцию ODBC. + +### 8.11. Движок таблиц для чтения из Mongo. + +### 8.12. Пропуск столбцов в форматах Parquet, ORC. + +### 8.13. Поддержка массивов в Parquet, ORC. + +### 8.14. Запись данных в ORC. + +### 8.15. Запись данных в CapNProto. + +### 8.16. Поддержка формата Avro. + +Павел Круглов, ВШЭ и Яндекс. + +### 8.17. ClickHouse как MySQL реплика. + +Ильяс Адюгамов, ВШЭ. + +### 8.18. ClickHouse как Federated MySQL. + +### 8.19. Интеграция с RabbitMQ. + +Ксения Сумарокова, ВШЭ. + +### 8.20. Интеграция с SQS. + +### 8.21. Поддержка произвольного количества языков для имён регионов. + +Нужно для БК. Декабрь 2019. + + +## 9. Безопасность. + +### 9.1. Ограничение на хосты в запросах ко внешним системам. + +Михаил Коротов. + +### 9.2. Преднастроенные именованные соединения к внешним БД. + +Валерий Батурин, ВШЭ. + +### 9.3. Поддержка TLS для ZooKeeper. + + +## 10. Внешние словари. + +### 10.1. Исправление зависания в библиотеке доступа к YT. + +Библиотека для доступа к YT обладает идиотским поведением и не переживает учения. +Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT. + +### 10.2. Исправление SIGILL в библиотеке доступа к YT. + +Код YT зачем-то использует SIGILL вместо abort. Это, опять же, происходит при учениях. +Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT. + +### 10.3. Возможность чтения данных из статических таблиц в YT словарях. + +Нужно для БК и Метрики. + +### 10.4. Словарь из YDB (KikiMR). + +Нужно для Метрики, а делать будет Александр Гололобов. Или он сейчас это прочитает и скажет "я никогда не буду делать эту задачу". + +### 10.5. Закрытие соединений и уменьшение числа соединений для MySQL и ODBC. + +Нужно для Метрики. + +### 10.6. Словари из Cassandra и Couchbase. + +### 10.7. Поддержка Nullable в словарях. + +Артём Стрельцов, Николай Дегтеринский, Наталия Михненко, ВШЭ. + +### 10.8. Поддержка массивов в словарях. + +Артём Стрельцов, Николай Дегтеринский, Наталия Михненко, ВШЭ. + +### 10.9. Уменьшение блокировок для cache словарей за счёт одновременных запросов одного и того же. + +Нужно для БК, но на самом деле они так просто думают, а все проблемы из-за низко-качественной библиотеки для доступа в YT. + +### 10.10. Возможность использования старых значений из cache словаря пока они перезапрашиваются. + +Нужно для БК и Метрики. + +### 10.11. Возможность исключительно асинхронных запросов в cache словарях. + +Нужно для БК и Метрики. Требует 10.10. + +### 10.12. Layout direct для словарей. + +Артём Стрельцов, Николай Дегтеринский, Наталия Михненко, ВШЭ. + +### 10.13. Использование Join как generic layout для словарей. + +Артём Стрельцов, Николай Дегтеринский, Наталия Михненко, ВШЭ. + +### 10.14. Поддержка всех типов в функции transform. + +### 10.15. Использование словарей как специализированного layout для Join. + +### 10.16. Словари на локальном SSD. + +Никита Васильев, ВШЭ и Яндекс. + +### 10.17. Локальный дамп состояния словаря для быстрого старта сервера. + +### 10.18. Таблица Join или словарь на удалённом сервере как key-value БД для cache словаря. + + +## 11. Интерфейсы. + +### 11.1. Вставка состояний агрегатных функций в виде кортежа аргументов или массива кортежей аргументов. + +### 11.2. Возможность использовать движок JDBC из коробки. + +Нужно разобраться, как упаковывать Java в статический бинарник, возможно AppImage. + +### 11.3. Интеграционные тесты ODBC драйвера путём подключения ClickHouse к самому себе через ODBC. + +Денис Глазачев, Altinity. + +### 11.4. Исправление упячек с типами Date и Decimal в clickhouse-cpp. + +### 11.5. Поддержка TLS в clickhouse-cpp. + +Библиотеку clickhouse-cpp разрабатывал Павел Артёмкин в свободное время. + +### 11.6. Интеграционные тесты clickhouse-cpp. + +### 11.7. Интерактивный режим работы программы clickhouse-local. + +### 11.8. Поддержка протокола PostgreSQL. + +Элбакян Мовсес Андраникович, ВШЭ. + +### 11.9. Доработки ODBC драйвера. + +Денис Глазачев, Altinity. + +### 11.10. Преднастроенные HTTP handlers для запросов. + +zhang2014 + + +## 12. Управление пользователями и доступом. + +### 12.1. Role Based Access Control. + +[Виталий Баранов](https://github.com/vitlibar). Финальная стадия разработки, рабочая версия в декабре 2019. + +### 12.2. Управление пользователями и правами доступа с помощью SQL запросов. + +[Виталий Баранов](https://github.com/vitlibar). Финальная стадия разработки, рабочая версия в декабре 2019. + +### 12.3. Подключение справочника пользователей и прав доступа из LDAP. + +[Виталий Баранов](https://github.com/vitlibar). Требует 12.1. + +### 12.4. Подключение IDM системы Яндекса как справочника пользователей и прав доступа. + +Пока низкий приоритет. Нужно для Метрики. Требует 12.3. + +### 12.5. Pluggable аутентификация с помощью Kerberos (возможно, подключение GSASL). + +[Виталий Баранов](https://github.com/vitlibar). Требует 12.1. + +### 12.6. Информация о пользователях и квотах в системной таблице. + +[Виталий Баранов](https://github.com/vitlibar). Требует 12.1. + + +## 13. Разделение ресурсов, multi-tenancy. + +### 13.1. Overcommit запросов по памяти и вытеснение. + +Требует 2.1. Способ реализации обсуждается. + +### 13.2. Общий конвейер выполнения на сервер. + +Требует 2.1. [Николай Кочетов](https://github.com/KochetovNicolai). + +### 13.3. Пулы ресурсов. + +Требует 13.2 или сможем сделать более неудобную реализацию раньше. + + +## 14. Диалект SQL. + +### 14.1. Исправление семантики CAST для Nullable. + +Нужно для DataLens. А также для внедрения в BI инструмент Looker. + +### 14.2. Поддержка WITH для подзапросов. + +### 14.3. Поддержка подстановок для множеств в правой части IN. + +### 14.4. Поддержка подстановок для идентификаторов (имён) в SQL запросе. + +zhang2014 + +### 14.5. Поддержка задания множества как массива в правой части секции IN. + +Василий Немков, Altinity, делал эту задачу, но зачем-то забросил её. + +### 14.6. Глобальный scope для WITH. + +### 14.7. Nullable для WITH ROLLUP, WITH CUBE, WITH TOTALS. + +Простая задача. + +### 14.8. Модификаторы DISTINCT, ORDER BY для агрегатных функций. + +Софья Борзенкова, ВШЭ. + +### 14.9. Поддержка запроса EXPLAIN. + +Требует 2.1. [Николай Кочетов](https://github.com/KochetovNicolai). + +### 14.10. arrayReduce как функция высшего порядка. + +### 14.11. Функции для grouping sets. + +### 14.12. Функции обработки временных рядов. + +Сложная задача, так как вводит новый класс функций и требует его обработку в оптимизаторе запросов. + +### 14.13. Применимость функций высшего порядка для кортежей и Nested. + +### 14.14. Неявные преобразования типов констант. + +Требует 2.12. + +### 14.15. Неявные преобразования типов под настройкой. + +Требует 2.12. Для внедрения в BI инструмент Looker. + +### 14.16. Синонимы для функций из MySQL. + +### 14.17. Ввести понятие stateful функций. + +Для runningDifference, neighbour - их учёт в оптимизаторе запросов. + +### 14.18. UNION DISTINCT и возможность включить его по-умолчанию. + +Для BI систем. + +### 14.19. Совместимость парсера типов данных с SQL. + +Для BI систем. + +### 14.20. Позиционные аргументы для GROUP BY и ORDER BY. + +Тривиально и используется многими системами, но не входит в стандарт SQL. + +### 14.21. Приведение типов для IN (подзапрос) и для JOIN. + + +## 15. Улучшение поддержки JOIN. + +### 15.1. Доведение merge JOIN до продакшена. + +Артём Зуйков. Сейчас merge JOIN включается вручную опцией и всегда замедляет запросы. Хотим, чтобы он замедлял запросы только когда это неизбежно. +Кстати, смысл merge JOIN появляется только совместно с 15.2 и 15.3. + +### 15.2. Прокидывание условий в OUTER JOIN. + +Возможно, Артём Зуйков, но задача ещё не продумана до конца. Возможно, требует 2.1. + +### 15.3. Логический вывод для цепочек вида ON t1.x = t2.y WHERE t1.x = 10 + +Возможно, Артём Зуйков. Для полноценной работы 15.2. + +### 15.4. Distributed JOIN с помощью перераспределения данных. + +Артём Зуйков. + +### 15.5. Использование ключа таблицы для оптимизации merge JOIN. + + +## 16. Типы данных и функции. + +### 16.1. DateTime64. + +Василий Немков. + +### 16.2. Тип данных для JSON. + +zhang2014 + +### 16.3. Поддержка неконстантных аргументов с регулярными выражениями в функциях. + +Данила Кутенин, но только после секретного изменения в работе. + +### 16.4. Функция rowNumberForKey. + +### 16.5. Функции для XML и HTML escape. + +### 16.6. Функции нормализации и хэширования SQL запросов. + + +## 17. Работа с географическими данными. + +### 17.1. Гео-словари для определения региона по координатам. + +[Андрей Чулков](https://github.com/achulkov2), Антон Кваша, Артур Петуховский, ВШЭ. +Будет основано на коде от Арслана Урташева. + +### 17.2. GIS типы данных и операции. + +Алексей Коряков, Алексей Илюхов, ВШЭ, Яндекс.Карты. + +### 17.3. Ускорение greatCircleDistance. + +[Ольга Хвостикова](https://github.com/stavrolia), основано на коде Андрея Аксёнова, получено разрешение на использование кода. + +### 17.4. Ускорение geohash с помощью библиотеки из Аркадии. + +Предположительно, [Андрей Чулков](https://github.com/achulkov2). Изначальная реализация в Аркадии - Денис Шапошников. Получено одобрение от руководства. + +### 17.5. Проверки в функции pointInPolygon. + +[Николай Кочетов](https://github.com/KochetovNicolai). Сейчас функция тихо не работает в случае полигонов с самопересечениями, надо кидать исключение. + + +## 18. Машинное обучение и статистика. + +### 18.1. Инкрементальная кластеризация данных. + +Александр Кожихов, Максим Кузнецов. Обнаружена фундаментальная проблема в реализации, доделывает предположительно [Николай Кочетов](https://github.com/KochetovNicolai). Он может делегировать задачу кому угодно. + +### 18.2. Агрегатные функции для статистических тестов. + +Артём Цыганов, Руденский Константин Игоревич, Семёнов Денис, ВШЭ. + +### 18.3. Инфраструктура для тренировки моделей в ClickHouse. + +В очереди. Возможно, Александр Кожихов. У него сначала идёт задача 24.26. + + +## 19. Улучшение работы кластера. + +### 19.1. Параллельные кворумные вставки без линеаризуемости. + +Александра Латышева, ВШЭ и Яндекс. + +### 19.2. Подключение Etcd или Consul как альтернативы ZooKeeper. + +Алексей Лёвушкин, ВШЭ и Яндекс. + +### 19.3. Подключение YT Cypress или YDB как альтернативы ZooKeeper. + +Hold. Полезно для Яндекс.Облака и БК, но есть риски, что будет вредно, а не полезно. + +### 19.4. internal_replication = 'auto'. + +### 19.5. Реплицируемые базы данных. + +В очереди, возможно Валерий Батурин, ВШЭ. + +### 19.6. Одновременный выбор кусков для слияния многими репликами, отказ от leader election в ZK. + +### 19.7. Возможность записи данных при недоступности ZK и отказ от линейного порядка кусков в большинстве случаев. + +### 19.8. Отказ от хранения в ZK множества кусков для каждой реплики отдельно. + +### 19.9. Отказ от хранения в ZK лога вставок и мержей. Обмен данными о кусках напрямую. + +### 19.10. Облачные таблицы. + +Требует 1.6, 19.1, 19.6, 19.7, 19.8, 19.9. + + +## 20. Мутации данных. + +Пока все задачи по точечным UPDATE/DELETE имеют низкий приоритет, но ожидаем взять в работу в середине 2020. + +### 20.1. Поддержка DELETE путём запоминания множества затронутых кусков и ключей. + +### 20.2. Поддержка DELETE путём преобразования множества ключей в множество row_numbers на реплике, столбца флагов и индекса по диапазонам. + +### 20.3. Поддержка ленивых DELETE путём запоминания выражений и преобразования к множеству ключей в фоне. + +### 20.4. Поддержка UPDATE с помощью преобразования в DELETE и вставок. + + +## 21. Оптимизации производительности. + +### 21.1. Параллельный парсинг форматов. + +Начинал Олег Ершов, доделывает Никита Михайлов, помогает [Александр Кузьменков](https://github.com/akuzm). Почти всё готово. + +### 21.2. Параллельное форматирование форматов. + +После 21.1, предположительно Никита Михайлов. Задача сильно проще чем 21.1. + +### 21.3. Исправление низкой производительности анализа индекса в случае большого множества в секции IN. + +Нужно всем (Zen, БК, DataLens...) Пока ещё не выбран исполнитель. + +### 21.4. Использование ORDER BY ключа для оптимизации GROUP BY и DISTINCT. + +Дмитрий Рубашкин, ВШЭ. Помогает Антон Попов. + +### 21.5. Распараллеливание INSERT при INSERT SELECT, если это необходимо. + +### 21.6. Уменьшение числа потоков для SELECT в случае тривиального INSERT SELECT. + +### 21.7. Кэш результатов запросов. + +[Achimbab](https://github.com/achimbab). + +### 21.8. Взаимная интеграция аллокатора и кэша. + +Михаил Кот, ВШЭ. Задача сложная и рискованная. + +### 21.8.1. Отдельный аллокатор для кэшей с ASLR. + +В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения оказались неудачными. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд. + +### 21.9. Исправить push-down выражений с помощью Processors. + +[Николай Кочетов](https://github.com/KochetovNicolai). Требует 2.1. + +### 21.10. Улучшение эвристики PREWHERE. + +Amos Bird. + +### 21.11. Peephole оптимизации запросов. + +Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ + +### 21.12. Алгебраические оптимизации запросов. + +Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ + +### 21.13. Fusion агрегатных функций. + +После или совместно с 21.11. + +### 21.14. Оптимизация запросов с помощью constraints. + +Мария Нефедова, ВШЭ. + +### 21.15. Многоступенчатое чтение данных вместо PREWHERE. + +Требует 2.1 и 21.10. + +### 21.16. Оптимизация GROUP BY с большим количеством агрегатных функций путём вычисления в два прохода. + +Нужно для БК. + +### 21.17. Оптимизация GROUP BY при наличии ORDER BY по тем же ключам с LIMIT. + +Нужно для БК. + +### 21.18. Внутренняя параллелизация мержа больших состояний агрегатных функций. + +### 21.19. Оптимизация сортировки. + +Василий Морозов, Арслан Гумеров, Альберт Кидрачев, ВШЭ. +В прошлом году задачу начинал делать Евгений Правда, ВШЭ, но почти полностью просрал её. + +### 21.20. Использование материализованных представлений для оптимизации запросов. + +В ByteDance есть готовая реализация, но они её боятся из-за, возможно, низкого качества кода. + +### 21.21. Чтение больших файлов с помощью mmap. + +Тривиально, почти всё готово. + +### 21.22. Userspace page cache. + +Требует 21.8. + +### 21.23. Ускорение работы с вторичными индексами. + +zhang2014. + + +## 22. Долги и недоделанные возможности. + +### 22.1. Исправление неработающих таймаутов, если используется TLS. + +Сейчас смотрит [Александр Сапин](https://github.com/alesapin), но он может делегировать задачу кому угодно. Нужно для Яндекс.Облака. + +### 22.2. Убрать возможность изменить настройки в native протоколе в случае readonly. + +Алексей Миловидов или [Виталий Баранов](https://github.com/vitlibar). + +### 22.3. Защита от абсурдно заданных пользователем кодеков. + +В очереди, скорее всего [Ольга Хвостикова](https://github.com/stavrolia). + +### 22.4. Исправление оставшихся deadlocks в табличных RWLock-ах. + +Александр Казаков. Нужно для Яндекс.Метрики и Datalens. + +### 22.5. Исправление редких срабатываний TSan в stress тестах в CI. + +Александр Казаков. + +### 22.6. Изменение только DEFAULT в ALTER TABLE может поменять тип столбца. + +### 22.7. Row-Level Security не работает в случае наличия в запросе IN подзапросов. + +[Иван Лежанкин](https://github.com/abyss7). Нужно для Метрики. + +### 22.8. Исправить десериализацию параметров для параметризованных запросов. + +Хотел исправить Василий Немков, Altinity, но есть маленькие затруднения, наверное переделает Алексей Миловидов. + +### 22.9. Разобраться с десериализацией массивов со значениями по-умолчанию в Protobuf формате в случае protobuf 3. + +[Виталий Баранов](https://github.com/vitlibar). Возможно, это - фундаментальная проблема и следует её только документировать. + +### 22.10. Исправление дрифта при отслеживании потребления памяти запросами. + +Требует 6.3., но можно улучшить отдельными хаками. Нужно Метрике и БК. + +### 22.11. Более простая ser/de настроек запросов. + +[Виталий Баранов](https://github.com/vitlibar), почти всё готово. + +### 22.12. Исправление идиотско низкой производительности чтения из Kafka. + +[Иван Лежанкин](https://github.com/abyss7). + +### 22.13. Посмотреть, почему не работают некоторые collations. + +[Иван Лежанкин](https://github.com/abyss7), совмещается с 7.1. + +### 22.14. Посмотреть, почему не работает StorageSet для MergeTree таблиц при некоторых условиях. + +### 22.15. Нормализация коммитов в Kafka и идемпотентности операций. + +[Иван Лежанкин](https://github.com/abyss7), если он не сдастся. + +### 22.16. Исправление идиотско низкой производительности кодека DoubleDelta. + +Василий Немков, Altinity - сейчас старательно динамит эту задачу. + +### 22.17. Консистентно работающий POPULATE для MaterializedView. + +### 22.18. Исправление заметного падения производительности форматов после добавления доменов типов. + +Василий Немков, Altinity. + +### 22.19. Одновременное использование SAMPLE и PREWHERE. + +Нужно для Метрики. [Николай Кочетов](https://github.com/KochetovNicolai), ноябрь 2019. + +### 22.20. Неправильная работа PREWHERE при некоторых условиях. + +[Николай Кочетов](https://github.com/KochetovNicolai), декабрь 2019. + +### 22.21. Неправильное поведение DateTime в районе начала unix epoch. + +Алексей Миловидов. + +### 22.22. Nullable в функции transform и в CASE по множеству значений. + +После 10.14. + +### 22.23. Правильная обработка Nullable в функциях, которые кидают исключение на default значении: modulo, intDiv. + +### 22.24. Излишняя фильтрация ODBC connection string. + +Нужно для Метрики. Алексей Миловидов. + +### 22.25. Избавиться от библиотеки btrie. + +Алексей Миловидов. Низкий приоритет. + +### 22.26. Плохая производительность quantileTDigest. + +Алексей Миловидов или будет переназначено. + +### 22.27. Проверить несколько PR, которые были закрыты zhang2014 и sundy-li. + +Алексей Миловидов. + +### 22.28. Изучить и исправить поведение работы с Kafka при ребалансировке. + +[Иван Лежанкин](https://github.com/abyss7), если он не сдастся. + + +## 23. Default Festival. + +### 23.1. Включение minimalistic_part_header в ZooKeeper. + +Сильно уменьшает объём данных в ZooKeeper. Уже год в продакшене в Яндекс.Метрике. +Алексей Миловидов, ноябрь 2019. + +### 23.2. Включение distributed_aggregation_memory_efficient. + +Есть риски меньшей производительности лёгких запросов, хотя производительность тяжёлых запросов всегда увеличивается. + +### 23.3. Включение min_bytes_to_external_sort и min_bytes_to_external_group_by. + +Желательно 5.2. и 13.1. + +### 23.4. Включение синхронной записи в Distributed таблицы по-умолчанию. + +Есть гипотеза, что плохо работает на очень больших кластерах. + +### 23.5. Включение compile_expressions. + +Требует 7.2. Задачу изначально делал Денис Скоробогатов, ВШЭ и Яндекс, затем доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin). + +### 23.6. Включение учёта порядка столбцов в CSV. + +Просто аккуратно включить. + +### 23.7. Включение NULL as Default в CSV. + +Просто аккуратно включить. + +### 23.8. Включение оптимизации VALUES. + +Просто аккуратно включить. + +### 23.9. Включение Processors. + +[Николай Кочетов](https://github.com/KochetovNicolai). + +### 23.10. Включение mlock бинарника. + +Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много треш-программ. + + +## 24. Экспериментальные задачи. + +### 24.1. Веб-интерфейс для просмотра состояния кластера и профилирования запросов. + +Антон Мамонов, УрФУ, Яндекс. + +### 24.2. Экспериментальные алгоритмы сжатия. + +Анастасия Наумова, ВШЭ. + +### 24.3. Экспериментальные кодеки. + +Вероника Фалчикова, Лада Торчик, ВШЭ. + +### 24.4. Шифрование в ClickHouse на уровне кусков данных. + +Yuchen Dong, ICS. + +### 24.5. Поддержка функций шифрования для отдельных значений. + +Yuchen Dong, ICS. + +### 24.6. Userspace RAID. + +Глеб Новиков, ВШЭ. + +### 24.7. Вероятностные структуры данных для фильтрации по подзапросам. + +Рузель Ибрагимов, ВШЭ и Яндекс. + +### 24.8. Специализация векторизованного кода для AVX/AVX2/AVX512 и ARM NEON. + +Дмитрий Ковальков, ВШЭ и Яндекс. + +### 24.9. Общий подход к CPU dispatching в фабрике функций. + +Дмитрий Ковальков, ВШЭ и Яндекс. + +### 24.10. Поддержка типов half/bfloat16/unum. + +Рустам Гусейн-заде, ВШЭ. + +### 24.11. User Defined Functions. + +Игорь Минеев, ВШЭ. + +### 24.12. GPU offloading. + +Риск состоит в том, что даже известные GPU базы, такие как OmniSci, работают медленнее, чем ClickHouse. +Преимущество возможно только на полной сортировке и JOIN. +Алексей Соловей, nVidia и Рита Коннова, ВШЭ. + +### 24.13. Stream запросы. + +Пререквизит для ClickHouse как CEP-системы. + +### 24.14. Window функции. + +Требует 2.1. + +### 24.15. Поддержка полуструктурированных данных. + +Требует 1.14 и 2.10. + +### 24.16. Улучшение эвристики слияний. + +В прошлом году исследование по этой задаче сделал Егор Соловьёв, ВШЭ и Яндекс.Такси. Его исследование показало, что алгоритм нельзя существенно улучшить путём изменения параметров. Но исследование лажовое, так как рассмотрен только уже использующийся алгоритм. То есть, задача остаётся открытой. + +### 24.17. Экспериментальные способы ускорения параллельного GROUP BY. + +Максим Серебряков + +### 24.18. Не TCP протокол передачи файлов при репликации. + +### 24.19. Промежуточное состояние GROUP BY как структура данных для key-value доступа. + +### 24.20. Short-circuit вычисления некоторых выражений. + +Два года назад задачу попробовала сделать Анастасия Царькова, ВШЭ и Яндекс, но реализация получилась слишком неудобной и её удалили. + +### 24.21. Реализация в ClickHouse протокола распределённого консенсуса. + +Имеет смысл только после 19.2. + +### 24.22. Вывод типов по блоку данных. Вывод формата данных по примеру. + +Эльмир Марданов, ВШЭ. + +### 24.23. Минимальная поддержка транзакций для множества вставок/чтений. + +Максим Кузнецов, ВШЭ. + +### 24.24. Реализация алгоритмов differential privacy. + +Артём Вишняков, ВШЭ. + +### 24.25. Интеграция в ClickHouse функциональности обработки HTTP User Agent. + +Есть хороший код в Яндекс.Метрике. Получено согласие от руководства. +Михаил Филитов, ВШЭ. + +### 24.26. Поддержка open tracing или аналогов. + +Александр Кожихов, ВШЭ и Яндекс.YT. + +### 24.27. Реализация алгоритмов min-hash, sim-hash для нечёткого поиска полудубликатов. + +ucasFL, ICS. + +### 24.28. Другой sketch для квантилей. + +Похоже на quantileTiming, но с логарифмическими корзинами. + +### 24.29. Поддержка Arrow Flight. + +### 24.30. ClickHouse как графовая СУБД. + +Amos Bird, но его решение слишком громоздкое и пока не open-source. + +### 24.31. Кореллированные подзапросы. + +Перепиывание в JOIN. Не раньше 21.11, 21.12, 21.9. Низкий приоритет. + +### 24.32. Поддержка GRPC. + +Мария Конькова, ВШЭ и Яндекс. +Также смотрите 24.29. + + + +## 25. DevRel + +### 25.1. Перевод инструкции для начинающих разработчиков. + +Александр Казаков, ноябрь 2019. + +### 25.2. Вычитка и выкладка статьи про обфускацию данных на английском. + +Эми Жанель Кришниевски, Александр Казаков, Алексей Миловидов, ноябрь 2019. + +### 25.3. Подготовка статьи "Секреты оптимизации производительности ClickHouse". + +Алексей Миловидов, Леонид Клюев. + +### 25.4. Подготовка статьи "Профайлер запросов: трудный путь". + +Алексей Миловидов, Леонид Клюев. + +### 25.5. Подготовка статьи "Тестирование ClickHouse, которое мы заслужили". + +### 25.6. Перевод этих статей на английский. + +Требует 25.3, 25.4, 25.5. Эми Жанель Кришниевски + +### 25.7. Перевод статьи Данилы Кутенина на английский. + +Эми Жанель Кришниевски + +### 25.8. Выступление keynote на BDTC. + +Алексей Миловидов + +### 25.9. Подготовка докладчиков: khvostikao, ilezhankin, nikitamikhailov, akuzm и другие. + +[Ольга Хвостикова](https://github.com/stavrolia), [Иван Лежанкин](https://github.com/abyss7), Никита Михайлов, [Александр Кузьменков](https://github.com/akuzm). +Уже готовые докладчики: Алексей Миловидов, [Николай Кочетов](https://github.com/KochetovNicolai), [Александр Сапин](https://github.com/alesapin). +Получаем минимум 7 докладчиков в 2020 году. + +### 25.10. Митапы в России: Москва x2 + митап для разработчиков или хакатон, Санкт-Петербург, Минск, Нижний Новгород, Екатеринбург, Новосибирск и/или Академгородок, Иннополис или Казань. + +Екатерина Миназова - организация + +### 25.11. Митапы зарубежные: восток США (Нью Йорк, возможно Raleigh), возможно северо-запад (Сиэтл), Китай (Пекин снова, возможно митап для разработчиков или хакатон), Лондон. + +[Иван Блинков](https://github.com/blinkov/) - организация + +### 25.12. Статья "научная" - про устройство хранения данных и индексов или whitepaper по архитектуре. Есть вариант подать на VLDB. + +Низкий приоритет. Алексей Миловидов. + +### 25.13. Участие во всех мероприятиях Яндекса, которые связаны с разработкой бэкенда, C++ разработкой или с базами данных, возможно участие в DevRel мероприятиях. + +Алексей Миловидов и все подготовленные докладчики + +### 25.14. Конференции в России: все HighLoad, возможно CodeFest, DUMP или UWDC, возможно C++ Russia. + +Алексей Миловидов и все подготовленные докладчики + +### 25.15. Конференции зарубежные: Percona, DataOps, возможно Big Data Warsaw, попытка попасть на более крупные. + +Алексей Миловидов и все подготовленные докладчики + +### 25.16. Сайт play.clickhouse. + +### 25.17. Взаимодействие с ВУЗами: ВШЭ, УрФУ, ICS Beijing. + +Алексей Миловидов и вся группа разработки + +### 25.18. Лекция в ШАД. + +Алексей Миловидов + +### 25.19. Участие в курсе разработки на C++ в ШАД. + +### 25.20. Ещё одно сравнение производительности аналитических СУБД. + +Матвей Бубнов, УрФУ + +### 25.21. Повторное награждение контрибьюторов в Китае. + +### 25.22. On-site помощь с ClickHouse компаниям в дни рядом с мероприятиями. + +[Иван Блинков](https://github.com/blinkov/) - организация + +### 25.23. Новый мерч для ClickHouse. + +### 25.24. Конкурсы bughunter или оптимизации кода на C++. + +Проведение конкурсов должно начинаться для сотрудников Яндекса, пока нет согласования от Лили Надеждиной и Алексея Башкеева. + +### 25.25. Семинары для потенциальных клиентов Яндекс.Облака. + +По мере необходимости. Алексей Миловидов, организация - Всеволод Грабельников. + +### 25.26. Участие в GSoC. + +Андрей Бородин пытается уговорить нас участвовать, но пока загружены задачей 25.17. + +### 25.27. Обновить сайт ClickHouse. + +Иван Блинков. Есть риск, что станет хуже. diff --git a/docs/ru/interfaces/third-party/gui.md b/docs/ru/interfaces/third-party/gui.md index 9c3a1f6d936..f1487b43c4c 100644 --- a/docs/ru/interfaces/third-party/gui.md +++ b/docs/ru/interfaces/third-party/gui.md @@ -79,6 +79,7 @@ [clickhouse-cli](https://github.com/hatarist/clickhouse-cli) - это альтернативный клиент командной строки для ClickHouse, написанный на Python 3. Основные возможности: + - Автодополнение; - Подсветка синтаксиса для запросов и вывода данных; - Поддержка постраничного просмотра для результирующих данных; diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index 66ae2b1891f..fa2af4ce631 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -223,6 +223,7 @@ nav: - 'Changelog': 'changelog.md' - 'Security changelog': 'security_changelog.md' - 'Roadmap': 'roadmap.md' + - 'Подробный roadmap 2020': 'extended_roadmap.md' - 'F.A.Q.': - 'Общие вопросы': 'faq/general.md' diff --git a/docs/zh/interfaces/third-party/gui.md b/docs/zh/interfaces/third-party/gui.md index 31a533e229d..b8143792981 100644 --- a/docs/zh/interfaces/third-party/gui.md +++ b/docs/zh/interfaces/third-party/gui.md @@ -63,6 +63,7 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). [clickhouse-cli](https://github.com/hatarist/clickhouse-cli) 是ClickHouse的替代命令行客户端,用Python 3编写。 特征: + - 自动完成。 - 查询和数据输出的语法高亮显示。 - 寻呼机支持数据输出。 @@ -74,6 +75,18 @@ ClickHouse Web 界面 [Tabix](https://github.com/tabixio/tabix). ## 商业 +### Holistics Software + +[Holistics](https://www.holistics.io/) 在2019年被Gartner FrontRunners列为可用性最高排名第二的商业智能工具之一。 Holistics是一个基于SQL的全栈数据平台和商业智能工具,用于设置您的分析流程。 + +特征: + +-自动化的电子邮件,Slack和Google表格报告时间表。 +-强大的SQL编辑器,具有版本控制,自动完成,可重用的查询组件和动态过滤器。 +-通过iframe在自己的网站或页面中嵌入仪表板。 +-数据准备和ETL功能。 +-SQL数据建模支持数据的关系映射。 + ### DataGrip [DataGrip](https://www.jetbrains.com/datagrip/) 是JetBrains的数据库IDE,专门支持ClickHouse。 它还嵌入到其他基于IntelliJ的工具中:PyCharm,IntelliJ IDEA,GoLand,PhpStorm等。 diff --git a/docs/zh/introduction/features_considered_disadvantages.md b/docs/zh/introduction/features_considered_disadvantages.md index 9aa6e90a4d8..015f1481b80 100644 --- a/docs/zh/introduction/features_considered_disadvantages.md +++ b/docs/zh/introduction/features_considered_disadvantages.md @@ -1,4 +1,4 @@ -# ClickHouse可以考虑缺点的功能 +# ClickHouse可以认为是缺点的功能 1. 没有完整的事务支持。 2. 缺少高频率,低延迟的修改或删除已存在数据的能力。仅能用于批量删除或修改数据,但这符合 [GDPR](https://gdpr-info.eu)。 diff --git a/libs/libcommon/include/common/iostream_debug_helpers.h b/libs/libcommon/include/common/iostream_debug_helpers.h index 9149ffb5ed0..72891ed03a5 100644 --- a/libs/libcommon/include/common/iostream_debug_helpers.h +++ b/libs/libcommon/include/common/iostream_debug_helpers.h @@ -140,7 +140,7 @@ Out & dump(Out & out, const char * name, T && x) #pragma clang diagnostic ignored "-Wgnu-zero-variadic-macro-arguments" #endif -#define DUMPVAR(VAR) dump(std::cerr, #VAR, (VAR)); std::cerr << "; "; +#define DUMPVAR(VAR) ::dump(std::cerr, #VAR, (VAR)); std::cerr << "; "; #define DUMPHEAD std::cerr << __FILE__ << ':' << __LINE__ << " [ " << getThreadNumber() << " ] "; #define DUMPTAIL std::cerr << '\n'; diff --git a/libs/libcommon/include/common/logger_useful.h b/libs/libcommon/include/common/logger_useful.h index b4693115cb3..ea1a25cc8fa 100644 --- a/libs/libcommon/include/common/logger_useful.h +++ b/libs/libcommon/include/common/logger_useful.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #ifndef QUERY_PREVIEW_LENGTH