From 846804fed085680d4d4ae1ac5f34329e39006486 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 11:39:04 +0000 Subject: [PATCH 01/26] Add separate handshake_timeout for receiving Hello packet from replica --- docs/en/operations/settings/settings.md | 6 +++ src/Client/Connection.cpp | 6 ++- src/Client/Connection.h | 2 +- src/Client/ConnectionParameters.cpp | 3 +- src/Core/Settings.h | 1 + src/IO/ConnectionTimeouts.cpp | 58 +++++++++++++++---------- src/IO/ConnectionTimeouts.h | 14 ++++-- src/Server/TCPHandler.cpp | 7 +++ 8 files changed, 65 insertions(+), 32 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index cddde2090f8..20779eba360 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1048,6 +1048,12 @@ Timeouts in seconds on the socket used for communicating with the client. Default value: 10, 300, 300. +## handshake_timeout_ms {#handshake-timeout-ms} + +Timeout in milliseconds for receiving Hello packet from replicas during handshake. + +Default value: 300000. + ## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} Cancels HTTP read-only queries (e.g. SELECT) when a client closes the connection without waiting for the response. diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d39148d3016..09145bcdf1b 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -190,7 +190,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) connected = true; sendHello(); - receiveHello(); + receiveHello(timeouts.handshake_timeout); if (server_revision >= DBMS_MIN_PROTOCOL_VERSION_WITH_ADDENDUM) sendAddendum(); @@ -305,8 +305,10 @@ void Connection::sendAddendum() } -void Connection::receiveHello() +void Connection::receiveHello(const Poco::Timespan & handshake_timeout) { + TimeoutSetter timeout_setter(*socket, socket->getSendTimeout(), handshake_timeout); + /// Receive hello packet. UInt64 packet_type = 0; diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 77dbe5e3398..cb3f2507cb9 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -256,7 +256,7 @@ private: void connect(const ConnectionTimeouts & timeouts); void sendHello(); void sendAddendum(); - void receiveHello(); + void receiveHello(const Poco::Timespan & handshake_timeout); #if USE_SSL void sendClusterNameAndSalt(); diff --git a/src/Client/ConnectionParameters.cpp b/src/Client/ConnectionParameters.cpp index 2031036eb58..c47d217d432 100644 --- a/src/Client/ConnectionParameters.cpp +++ b/src/Client/ConnectionParameters.cpp @@ -67,7 +67,8 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati Poco::Timespan(config.getInt("connect_timeout", DBMS_DEFAULT_CONNECT_TIMEOUT_SEC), 0), Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0), Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0), - Poco::Timespan(config.getInt("tcp_keep_alive_timeout", 0), 0)); + Poco::Timespan(config.getInt("tcp_keep_alive_timeout", 0), 0), + Poco::Timespan(config.getInt("handshake_timeout_ms", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000), 0)); timeouts.sync_request_timeout = Poco::Timespan(config.getInt("sync_request_timeout", DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC), 0); } diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 610c7135a75..d5fb864ca6b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,6 +55,7 @@ class IColumn; M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \ M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ + M(Milliseconds, handshake_timeout_ms, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000, "Timeout for receiving HELLO packet from replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 1000, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 1000, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 401afb7baac..01fbaa4f817 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -17,22 +17,7 @@ ConnectionTimeouts::ConnectionTimeouts( , secure_connection_timeout(connection_timeout) , hedged_connection_timeout(receive_timeout_) , receive_data_timeout(receive_timeout_) -{ -} - -ConnectionTimeouts::ConnectionTimeouts( - Poco::Timespan connection_timeout_, - Poco::Timespan send_timeout_, - Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_) - : connection_timeout(connection_timeout_) - , send_timeout(send_timeout_) - , receive_timeout(receive_timeout_) - , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) - , http_keep_alive_timeout(0) - , secure_connection_timeout(connection_timeout) - , hedged_connection_timeout(receive_timeout_) - , receive_data_timeout(receive_timeout_) + , handshake_timeout(receive_timeout_) { } @@ -41,7 +26,26 @@ ConnectionTimeouts::ConnectionTimeouts( Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_) + Poco::Timespan handshake_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) + , http_keep_alive_timeout(0) + , secure_connection_timeout(connection_timeout) + , hedged_connection_timeout(receive_timeout_) + , receive_data_timeout(receive_timeout_) + , handshake_timeout(handshake_timeout_) +{ +} + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_, + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan handshake_timeout_) : connection_timeout(connection_timeout_) , send_timeout(send_timeout_) , receive_timeout(receive_timeout_) @@ -50,6 +54,7 @@ ConnectionTimeouts::ConnectionTimeouts( , secure_connection_timeout(connection_timeout) , hedged_connection_timeout(receive_timeout_) , receive_data_timeout(receive_timeout_) + , handshake_timeout(handshake_timeout_) { } @@ -60,16 +65,18 @@ ConnectionTimeouts::ConnectionTimeouts( Poco::Timespan tcp_keep_alive_timeout_, Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, - Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_) + Poco::Timespan hedged_connection_timeout_, + Poco::Timespan receive_data_timeout_, + Poco::Timespan handshake_timeout_) : connection_timeout(connection_timeout_) , send_timeout(send_timeout_) , receive_timeout(receive_timeout_) , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) , http_keep_alive_timeout(http_keep_alive_timeout_) , secure_connection_timeout(secure_connection_timeout_) - , hedged_connection_timeout(receive_hello_timeout_) + , hedged_connection_timeout(hedged_connection_timeout_) , receive_data_timeout(receive_data_timeout_) + , handshake_timeout(handshake_timeout_) { } @@ -90,13 +97,14 @@ ConnectionTimeouts ConnectionTimeouts::getSaturated(Poco::Timespan limit) const saturate(http_keep_alive_timeout, limit), saturate(secure_connection_timeout, limit), saturate(hedged_connection_timeout, limit), - saturate(receive_data_timeout, limit)); + saturate(receive_data_timeout, limit), + saturate(handshake_timeout, limit)); } /// Timeouts for the case when we have just single attempt to connect. ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Settings & settings) { - return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout); + return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout, settings.handshake_timeout_ms); } /// Timeouts for the case when we will try many addresses in a loop. @@ -110,7 +118,8 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings 0, settings.connect_timeout_with_failover_secure_ms, settings.hedged_connection_timeout_ms, - settings.receive_data_timeout_ms); + settings.receive_data_timeout_ms, + settings.handshake_timeout_ms); } ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout) @@ -120,7 +129,8 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings settings.http_send_timeout, settings.http_receive_timeout, settings.tcp_keep_alive_timeout, - http_keep_alive_timeout); + http_keep_alive_timeout, + settings.http_receive_timeout); } } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 368288ee022..684af42827f 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -23,6 +23,9 @@ struct ConnectionTimeouts Poco::Timespan hedged_connection_timeout; Poco::Timespan receive_data_timeout; + /// Timeout for receiving HELLO packet + Poco::Timespan handshake_timeout; + /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus) Poco::Timespan sync_request_timeout = Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0); @@ -35,13 +38,15 @@ struct ConnectionTimeouts ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_); + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan handshake_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_); + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan handshake_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, @@ -49,8 +54,9 @@ struct ConnectionTimeouts Poco::Timespan tcp_keep_alive_timeout_, Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, - Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_); + Poco::Timespan hedged_connection_timeout_, + Poco::Timespan receive_data_timeout_, + Poco::Timespan handshake_timeout_); static Poco::Timespan saturate(Poco::Timespan timespan, Poco::Timespan limit); ConnectionTimeouts getSaturated(Poco::Timespan limit) const; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 0522b6d8a48..a076e248a9f 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1296,6 +1296,13 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { + if (unlikely(sleep_in_send_tables_status.totalMilliseconds())) + { + out->next(); + std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds()); + std::this_thread::sleep_for(ms); + } + writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); From 2ff3c8badd5a5c18f14ca76438978a415fe73d74 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 May 2023 11:41:00 +0000 Subject: [PATCH 02/26] Remove testing code --- src/Server/TCPHandler.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index a076e248a9f..0522b6d8a48 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1296,13 +1296,6 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { - if (unlikely(sleep_in_send_tables_status.totalMilliseconds())) - { - out->next(); - std::chrono::milliseconds ms(sleep_in_send_tables_status.totalMilliseconds()); - std::this_thread::sleep_for(ms); - } - writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); From 3c1aeaaa793f080e20005bfa42a18746f969ad39 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 23 May 2023 11:39:40 +0000 Subject: [PATCH 03/26] Change default value of handshake_timeout to 10 sec, fix possible use-after-free --- docs/en/operations/settings/settings.md | 2 +- src/Client/Connection.cpp | 4 ++++ src/Core/Settings.h | 2 +- src/IO/TimeoutSetter.cpp | 21 +++++++++++++------ src/IO/TimeoutSetter.h | 6 +++++- .../test_reload_clusters_config/test.py | 2 +- .../test.py | 4 ++-- 7 files changed, 29 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b6ade78b1e1..0810b642039 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1054,7 +1054,7 @@ Default value: 10, 300, 300. Timeout in milliseconds for receiving Hello packet from replicas during handshake. -Default value: 300000. +Default value: 10000. ## cancel_http_readonly_queries_on_client_close {#cancel-http-readonly-queries-on-client-close} diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 09145bcdf1b..35423012424 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -361,6 +361,10 @@ void Connection::receiveHello(const Poco::Timespan & handshake_timeout) receiveException()->rethrow(); else { + /// Reset timeout_setter before disconnect, + /// because after disconnect socket will be invalid. + timeout_setter.reset(); + /// Close connection, to not stay in unsynchronised state. disconnect(); throwUnexpectedPacket(packet_type, "Hello or Exception"); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ba3a28af614..03cf32e5aaa 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,7 +55,7 @@ class IColumn; M(UInt64, max_query_size, DBMS_DEFAULT_MAX_QUERY_SIZE, "The maximum number of bytes of a query string parsed by the SQL parser. Data in the VALUES clause of INSERT queries is processed by a separate stream parser (that consumes O(1) RAM) and not affected by this restriction.", 0) \ M(UInt64, interactive_delay, 100000, "The interval in microseconds to check if the request is cancelled, and to send progress info.", 0) \ M(Seconds, connect_timeout, DBMS_DEFAULT_CONNECT_TIMEOUT_SEC, "Connection timeout if there are no replicas.", 0) \ - M(Milliseconds, handshake_timeout_ms, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000, "Timeout for receiving HELLO packet from replicas.", 0) \ + M(Milliseconds, handshake_timeout_ms, 10000, "Timeout for receiving HELLO packet from replicas.", 0) \ M(Milliseconds, connect_timeout_with_failover_ms, 1000, "Connection timeout for selecting first healthy replica.", 0) \ M(Milliseconds, connect_timeout_with_failover_secure_ms, 1000, "Connection timeout for selecting first healthy replica (for secure connections).", 0) \ M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "Timeout for receiving data from network, in seconds. If no bytes were received in this interval, exception is thrown. If you set this setting on client, the 'send_timeout' for the socket will be also set on the corresponding connection end on the server.", 0) \ diff --git a/src/IO/TimeoutSetter.cpp b/src/IO/TimeoutSetter.cpp index ed21383ccd4..b8b7a814703 100644 --- a/src/IO/TimeoutSetter.cpp +++ b/src/IO/TimeoutSetter.cpp @@ -29,14 +29,12 @@ TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, Poco::Timespan t TimeoutSetter::~TimeoutSetter() { + if (was_reset) + return; + try { - bool connected = socket.impl()->initialized(); - if (!connected) - return; - - socket.setSendTimeout(old_send_timeout); - socket.setReceiveTimeout(old_receive_timeout); + reset(); } catch (...) { @@ -44,4 +42,15 @@ TimeoutSetter::~TimeoutSetter() } } +void TimeoutSetter::reset() +{ + bool connected = socket.impl()->initialized(); + if (!connected) + return; + + socket.setSendTimeout(old_send_timeout); + socket.setReceiveTimeout(old_receive_timeout); + was_reset = true; +} + } diff --git a/src/IO/TimeoutSetter.h b/src/IO/TimeoutSetter.h index 31c37ea07af..3479986d7fe 100644 --- a/src/IO/TimeoutSetter.h +++ b/src/IO/TimeoutSetter.h @@ -6,7 +6,7 @@ namespace DB { -/// Temporarily overrides socket send/receive timeouts and reset them back into destructor +/// Temporarily overrides socket send/receive timeouts and reset them back into destructor (or manually by calling reset method) /// If "limit_max_timeout" is true, timeouts could be only decreased (maxed by previous value). struct TimeoutSetter { @@ -19,6 +19,9 @@ struct TimeoutSetter ~TimeoutSetter(); + /// Reset timeouts back. + void reset(); + Poco::Net::StreamSocket & socket; Poco::Timespan send_timeout; @@ -26,5 +29,6 @@ struct TimeoutSetter Poco::Timespan old_send_timeout; Poco::Timespan old_receive_timeout; + bool was_reset = false; }; } diff --git a/tests/integration/test_reload_clusters_config/test.py b/tests/integration/test_reload_clusters_config/test.py index a52871890e9..73ca4a01f34 100644 --- a/tests/integration/test_reload_clusters_config/test.py +++ b/tests/integration/test_reload_clusters_config/test.py @@ -169,7 +169,7 @@ test_config3 = """ def send_repeated_query(table, count=5): for i in range(count): node.query_and_get_error( - "SELECT count() FROM {} SETTINGS receive_timeout=1".format(table) + "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format(table) ) diff --git a/tests/integration/test_system_clusters_actual_information/test.py b/tests/integration/test_system_clusters_actual_information/test.py index 0658d0c7576..e90a6cdeb3f 100644 --- a/tests/integration/test_system_clusters_actual_information/test.py +++ b/tests/integration/test_system_clusters_actual_information/test.py @@ -40,8 +40,8 @@ def test(started_cluster): cluster.pause_container("node_1") node.query("SYSTEM RELOAD CONFIG") - node.query_and_get_error( - "SELECT count() FROM distributed SETTINGS receive_timeout=1" + error = node.query_and_get_error( + "SELECT count() FROM distributed SETTINGS receive_timeout=1, handshake_timeout_ms=1" ) result = node.query( From 8bc25c4ea3a1359affc36599bcc982b741ea5360 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 24 May 2023 14:40:27 +0200 Subject: [PATCH 04/26] Fix style --- tests/integration/test_reload_clusters_config/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_reload_clusters_config/test.py b/tests/integration/test_reload_clusters_config/test.py index 73ca4a01f34..cb003bbe04e 100644 --- a/tests/integration/test_reload_clusters_config/test.py +++ b/tests/integration/test_reload_clusters_config/test.py @@ -169,7 +169,9 @@ test_config3 = """ def send_repeated_query(table, count=5): for i in range(count): node.query_and_get_error( - "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format(table) + "SELECT count() FROM {} SETTINGS receive_timeout=1, handshake_timeout_ms=1".format( + table + ) ) From c9d0d217f5d180f5c78286b1b819534dd8cc1b59 Mon Sep 17 00:00:00 2001 From: frinkr Date: Wed, 31 May 2023 19:53:06 +0800 Subject: [PATCH 05/26] fix Keeper deadlock on exception when preprocessing requests --- src/Coordination/KeeperStateMachine.cpp | 11 ++++++++++- src/Coordination/KeeperStateMachine.h | 2 ++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index 6635c74149a..f787cc8722e 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -272,7 +272,8 @@ bool KeeperStateMachine::preprocess(const KeeperStorage::RequestForSession & req } catch (...) { - rollbackRequest(request_for_session, true); + tryLogCurrentException(__PRETTY_FUNCTION__); + rollbackRequestNoLock(request_for_session, true); throw; } @@ -411,6 +412,14 @@ void KeeperStateMachine::rollbackRequest(const KeeperStorage::RequestForSession storage->rollbackRequest(request_for_session.zxid, allow_missing); } +void KeeperStateMachine::rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing) +{ + if (request_for_session.request->getOpNum() == Coordination::OpNum::SessionID) + return; + + storage->rollbackRequest(request_for_session.zxid, allow_missing); +} + nuraft::ptr KeeperStateMachine::last_snapshot() { /// Just return the latest snapshot. diff --git a/src/Coordination/KeeperStateMachine.h b/src/Coordination/KeeperStateMachine.h index fbd97fd8631..f6d81d23056 100644 --- a/src/Coordination/KeeperStateMachine.h +++ b/src/Coordination/KeeperStateMachine.h @@ -68,6 +68,8 @@ public: // (can happen in case of exception during preprocessing) void rollbackRequest(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + void rollbackRequestNoLock(const KeeperStorage::RequestForSession & request_for_session, bool allow_missing); + uint64_t last_commit_index() override { return last_committed_idx; } /// Apply preliminarily saved (save_logical_snp_obj) snapshot to our state. From 68d46c81b99726a1e0467fc21d02a5311b1e49ca Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 11 Apr 2023 09:53:48 +0000 Subject: [PATCH 06/26] Do not store blocks in hash join if nothing is inseted --- src/Interpreters/HashJoin.cpp | 55 +++++++++++++++++++++++------------ 1 file changed, 37 insertions(+), 18 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 146b57049a6..92597f87f4b 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -543,16 +543,20 @@ namespace template struct Inserter { - static ALWAYS_INLINE void insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, + static ALWAYS_INLINE bool insertOne(const HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); if (emplace_result.isInserted() || join.anyTakeLastRow()) + { new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); + return true; + } + return false; } - static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE bool insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -563,9 +567,10 @@ namespace /// The first element of the list is stored in the value of the hash table, the rest in the pool. emplace_result.getMapped().insert({stored_block, i}, pool); } + return true; } - static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + static ALWAYS_INLINE bool insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -575,6 +580,7 @@ namespace if (emplace_result.isInserted()) time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality())); (*time_series_map)->insert(asof_column, stored_block, i); + return true; } }; @@ -582,7 +588,7 @@ namespace template size_t NO_INLINE insertFromBlockImplTypeCase( HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { [[maybe_unused]] constexpr bool mapped_one = std::is_same_v; constexpr bool is_asof_join = STRICTNESS == JoinStrictness::Asof; @@ -593,6 +599,7 @@ namespace auto key_getter = createKeyGetter(key_columns, key_sizes); + is_inserted = false; for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) @@ -603,11 +610,11 @@ namespace continue; if constexpr (is_asof_join) - Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + is_inserted |= Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) - Inserter::insertOne(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + is_inserted |= Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } return map.getBufferSizeInCells(); } @@ -616,32 +623,37 @@ namespace template size_t insertFromBlockImplType( HashJoin & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { if (null_map) return insertFromBlockImplTypeCase( - join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); + join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); else return insertFromBlockImplTypeCase( - join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); + join, map, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); } template size_t insertFromBlockImpl( HashJoin & join, HashJoin::Type type, Maps & maps, size_t rows, const ColumnRawPtrs & key_columns, - const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool) + const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, UInt8ColumnDataPtr join_mask, Arena & pool, bool & is_inserted) { switch (type) { - case HashJoin::Type::EMPTY: return 0; - case HashJoin::Type::CROSS: return 0; /// Do nothing. We have already saved block, and it is enough. + case HashJoin::Type::EMPTY: + [[fallthrough]]; + case HashJoin::Type::CROSS: + /// Do nothing. We will only save block, and it is enough + is_inserted = true; + return 0; #define M(TYPE) \ case HashJoin::Type::TYPE: \ return insertFromBlockImplType>::Type>(\ - join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool); \ + join, *maps.TYPE, rows, key_columns, key_sizes, stored_block, null_map, join_mask, pool, is_inserted); \ break; + APPLY_FOR_JOIN_VARIANTS(M) #undef M } @@ -816,6 +828,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) } } + bool is_inserted = false; if (kind != JoinKind::Cross) { joinDispatch(kind, strictness, data->maps[onexpr_idx], [&](auto kind_, auto strictness_, auto & map) @@ -824,28 +837,34 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) *this, data->type, map, rows, key_columns, key_sizes[onexpr_idx], stored_block, null_map, /// If mask is false constant, rows are added to hashmap anyway. It's not a happy-flow, so this case is not optimized join_mask_col.getData(), - data->pool); + data->pool, is_inserted); if (multiple_disjuncts) used_flags.reinit(stored_block); - else + else if (is_inserted) /// Number of buckets + 1 value from zero storage used_flags.reinit(size + 1); }); } - if (!multiple_disjuncts && save_nullmap) + if (!multiple_disjuncts && save_nullmap && is_inserted) { data->blocks_nullmaps_allocated_size += null_map_holder->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, null_map_holder); } - if (!multiple_disjuncts && not_joined_map) + if (!multiple_disjuncts && not_joined_map && is_inserted) { data->blocks_nullmaps_allocated_size += not_joined_map->allocatedBytes(); data->blocks_nullmaps.emplace_back(stored_block, std::move(not_joined_map)); } + if (!multiple_disjuncts && !is_inserted) + { + data->blocks_allocated_size -= stored_block->allocatedBytes(); + data->blocks.pop_back(); + } + if (!check_limits) return true; From 62f950ddaff7178fc479e2ccab236be39567e0a7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 3 May 2023 12:15:28 +0000 Subject: [PATCH 07/26] Keep blocks with nulls for right and full join --- src/Interpreters/HashJoin.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 92597f87f4b..436ecd382cd 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -603,7 +603,12 @@ namespace for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) + { + /// nulls are not inserted into hash table, + /// keep them for RIGHT and FULL joins + is_inserted = true; continue; + } /// Check condition for right table from ON section if (join_mask && !(*join_mask)[i]) @@ -861,6 +866,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block_, bool check_limits) if (!multiple_disjuncts && !is_inserted) { + LOG_TRACE(log, "Skipping inserting block with {} rows", rows); data->blocks_allocated_size -= stored_block->allocatedBytes(); data->blocks.pop_back(); } From 0f0958f82d5048c0aa3b28bdf9c5ce655e853219 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 4 May 2023 14:08:03 +0000 Subject: [PATCH 08/26] Add test 02725_any_join_single_row --- .../02725_any_join_single_row.reference | 3 +++ .../0_stateless/02725_any_join_single_row.sql | 26 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 tests/queries/0_stateless/02725_any_join_single_row.reference create mode 100644 tests/queries/0_stateless/02725_any_join_single_row.sql diff --git a/tests/queries/0_stateless/02725_any_join_single_row.reference b/tests/queries/0_stateless/02725_any_join_single_row.reference new file mode 100644 index 00000000000..5d748fc6dbb --- /dev/null +++ b/tests/queries/0_stateless/02725_any_join_single_row.reference @@ -0,0 +1,3 @@ +Join(ANY, LEFT, key) 0 1 +Join(ANY, LEFT, key) 1 1 +Join(ANY, LEFT, key) 1 1 diff --git a/tests/queries/0_stateless/02725_any_join_single_row.sql b/tests/queries/0_stateless/02725_any_join_single_row.sql new file mode 100644 index 00000000000..5e5c959c278 --- /dev/null +++ b/tests/queries/0_stateless/02725_any_join_single_row.sql @@ -0,0 +1,26 @@ +CREATE TABLE join_test +( + `key` UInt64, + `value` UInt64 +) +ENGINE = Join(ANY, LEFT, key); + +CREATE TEMPORARY TABLE initial_table_size AS + SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +SELECT engine_full, total_rows, total_bytes < 100_000 FROM initial_table_size; + +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); + + +CREATE TEMPORARY TABLE one_row_table_size AS + SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +SELECT engine_full, total_rows, total_bytes < 2 * (SELECT total_bytes FROM initial_table_size) FROM one_row_table_size; + +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(10_000); + +SELECT engine_full, total_rows, total_bytes == (SELECT total_bytes FROM one_row_table_size) FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); From 56fa98cb77c30cba3f93dfcd5b04c33dd5b1ec68 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 09:23:39 +0000 Subject: [PATCH 09/26] Rename variables for better readability --- src/Functions/FunctionsHashing.h | 71 ++++++++++++++++---------------- 1 file changed, 36 insertions(+), 35 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 3de757bfa3f..70adc7eba0f 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1073,42 +1073,43 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h; + ToType hash; if constexpr (Impl::use_int_hash_for_pods) { if constexpr (std::is_same_v) - h = IntHash64Impl::apply(bit_cast(vec_from[i])); + hash = IntHash64Impl::apply(bit_cast(vec_from[i])); else - h = IntHash32Impl::apply(bit_cast(vec_from[i])); + hash = IntHash32Impl::apply(bit_cast(vec_from[i])); } else { if constexpr (std::is_same_v) - h = JavaHashImpl::apply(vec_from[i]); + hash = JavaHashImpl::apply(vec_from[i]); else { - FromType v = vec_from[i]; + FromType value = vec_from[i]; if constexpr (std::endian::native == std::endian::big) { - FromType tmp_v; - reverseMemcpy(&tmp_v, &v, sizeof(v)); - v = tmp_v; + FromType value_reversed; + reverseMemcpy(&value_reversed, &value, sizeof(value)); + value = value_reversed; } - h = apply(key, reinterpret_cast(&v), sizeof(v)); + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } } if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); ToType hash; + if constexpr (std::is_same_v) hash = IntHash64Impl::apply(bit_cast(value)); else @@ -1139,45 +1140,45 @@ private: size_t size = vec_from.size(); for (size_t i = 0; i < size; ++i) { - ToType h; + ToType hash; if constexpr (std::endian::native == std::endian::little) { - h = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); + hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); } else { char tmp_buffer[sizeof(vec_from[i])]; reverseMemcpy(tmp_buffer, &vec_from[i], sizeof(vec_from[i])); - h = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); + hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(vec_from[i])); } if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (auto col_from_const = checkAndGetColumnConst(column)) { auto value = col_from_const->template getValue(); - ToType h; + ToType hash; if constexpr (std::endian::native == std::endian::little) { - h = apply(key, reinterpret_cast(&value), sizeof(value)); + hash = apply(key, reinterpret_cast(&value), sizeof(value)); } else { char tmp_buffer[sizeof(value)]; reverseMemcpy(tmp_buffer, &value, sizeof(value)); - h = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); + hash = apply(key, reinterpret_cast(tmp_buffer), sizeof(value)); } size_t size = vec_to.size(); if constexpr (first) - vec_to.assign(size, h); + vec_to.assign(size, hash); else { for (size_t i = 0; i < size; ++i) - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else @@ -1191,11 +1192,11 @@ private: for (size_t i = 0, size = column->size(); i < size; ++i) { StringRef bytes = column->getDataAt(i); - const ToType h = apply(key, bytes.data, bytes.size); + const ToType hash = apply(key, bytes.data, bytes.size); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } @@ -1211,14 +1212,14 @@ private: ColumnString::Offset current_offset = 0; for (size_t i = 0; i < size; ++i) { - const ToType h = apply(key, + const ToType hash = apply(key, reinterpret_cast(&data[current_offset]), offsets[i] - current_offset - 1); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); current_offset = offsets[i]; } @@ -1231,11 +1232,11 @@ private: for (size_t i = 0; i < size; ++i) { - const ToType h = apply(key, reinterpret_cast(&data[i * n]), n); + const ToType hash = apply(key, reinterpret_cast(&data[i * n]), n); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); } } else if (const ColumnConst * col_from_const = checkAndGetColumnConstStringOrFixedString(column)) @@ -1283,16 +1284,16 @@ private: { ColumnArray::Offset next_offset = offsets[i]; - ToType h; + ToType hash; if constexpr (std::is_same_v) - h = IntHash64Impl::apply(next_offset - current_offset); + hash = IntHash64Impl::apply(next_offset - current_offset); else - h = IntHash32Impl::apply(next_offset - current_offset); + hash = IntHash32Impl::apply(next_offset - current_offset); if constexpr (first) - vec_to[i] = h; + vec_to[i] = hash; else - vec_to[i] = combineHashes(key, vec_to[i], h); + vec_to[i] = combineHashes(key, vec_to[i], hash); for (size_t j = current_offset; j < next_offset; ++j) vec_to[i] = combineHashes(key, vec_to[i], vec_temp[j]); From 93ba75b370c6ece34fe8480440193b4775b3d105 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 12:23:04 +0000 Subject: [PATCH 10/26] Remove parentheses from single-line if/for --- src/Functions/FunctionsHashing.h | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 70adc7eba0f..9896adca7f3 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1119,10 +1119,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) vec_to[i] = combineHashes(key, vec_to[i], hash); - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1142,9 +1140,7 @@ private: { ToType hash; if constexpr (std::endian::native == std::endian::little) - { hash = apply(key, reinterpret_cast(&vec_from[i]), sizeof(vec_from[i])); - } else { char tmp_buffer[sizeof(vec_from[i])]; @@ -1163,9 +1159,7 @@ private: ToType hash; if constexpr (std::endian::native == std::endian::little) - { hash = apply(key, reinterpret_cast(&value), sizeof(value)); - } else { char tmp_buffer[sizeof(value)]; @@ -1176,10 +1170,8 @@ private: if constexpr (first) vec_to.assign(size, hash); else - { for (size_t i = 0; i < size; ++i) vec_to[i] = combineHashes(key, vec_to[i], hash); - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of argument of function {}", @@ -1246,16 +1238,10 @@ private: const size_t size = vec_to.size(); if constexpr (first) - { vec_to.assign(size, hash); - } else - { for (size_t i = 0; i < size; ++i) - { vec_to[i] = combineHashes(key, vec_to[i], hash); - } - } } else throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", From 03628bde422823922d1fdd52531d34212270edae Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 12:54:34 +0000 Subject: [PATCH 11/26] Fix hashing of const integer values --- src/Functions/FunctionsHashing.h | 26 ++++++++++++++++--- .../0_stateless/02534_keyed_siphash.reference | 3 +++ .../0_stateless/02534_keyed_siphash.sql | 9 +++++++ 3 files changed, 34 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsHashing.h b/src/Functions/FunctionsHashing.h index 9896adca7f3..32e3fbbd4ea 100644 --- a/src/Functions/FunctionsHashing.h +++ b/src/Functions/FunctionsHashing.h @@ -1096,7 +1096,7 @@ private: value = value_reversed; } hash = apply(key, reinterpret_cast(&value), sizeof(value)); - } + } } if constexpr (first) @@ -1110,10 +1110,28 @@ private: auto value = col_from_const->template getValue(); ToType hash; - if constexpr (std::is_same_v) - hash = IntHash64Impl::apply(bit_cast(value)); + if constexpr (Impl::use_int_hash_for_pods) + { + if constexpr (std::is_same_v) + hash = IntHash64Impl::apply(bit_cast(value)); + else + hash = IntHash32Impl::apply(bit_cast(value)); + } else - hash = IntHash32Impl::apply(bit_cast(value)); + { + if constexpr (std::is_same_v) + hash = JavaHashImpl::apply(value); + else + { + if constexpr (std::endian::native == std::endian::big) + { + FromType value_reversed; + reverseMemcpy(&value_reversed, &value, sizeof(value)); + value = value_reversed; + } + hash = apply(key, reinterpret_cast(&value), sizeof(value)); + } + } size_t size = vec_to.size(); if constexpr (first) diff --git a/tests/queries/0_stateless/02534_keyed_siphash.reference b/tests/queries/0_stateless/02534_keyed_siphash.reference index 3606b9a41db..ccc514e7ea2 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.reference +++ b/tests/queries/0_stateless/02534_keyed_siphash.reference @@ -194,3 +194,6 @@ E28DBDE7FE22E41C 1 E28DBDE7FE22E41C 1 +Check bug with hashing of const integer values +11862823756610506724 +11862823756610506724 diff --git a/tests/queries/0_stateless/02534_keyed_siphash.sql b/tests/queries/0_stateless/02534_keyed_siphash.sql index 9c914f586f0..900b99f548a 100644 --- a/tests/queries/0_stateless/02534_keyed_siphash.sql +++ b/tests/queries/0_stateless/02534_keyed_siphash.sql @@ -272,3 +272,12 @@ select hex(sipHash64()); SELECT hex(sipHash128()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128()) = '1CE422FEE7BD8DE20000000000000000'; select hex(sipHash64Keyed()); SELECT hex(sipHash128Keyed()) = hex(reverse(unhex('1CE422FEE7BD8DE20000000000000000'))) or hex(sipHash128Keyed()) = '1CE422FEE7BD8DE20000000000000000'; + +SELECT 'Check bug with hashing of const integer values'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (key Tuple(UInt64, UInt64), val UInt64) ENGINE=Memory; +INSERT INTO tab VALUES ((2, 2), 4); +-- these two statements must produce the same result +SELECT sipHash64Keyed(key, val) FROM tab; +SELECT sipHash64Keyed(key, 4::UInt64) FROM tab; +DROP TABLE tab; From bdb192cf2742d6f3059f621068bbc59d78124229 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Jun 2023 15:43:37 +0000 Subject: [PATCH 12/26] Test right join in 02725_any_join_single_row, style code changes --- src/Interpreters/HashJoin.cpp | 14 ++++---- .../02725_any_join_single_row.reference | 3 ++ .../0_stateless/02725_any_join_single_row.sql | 33 ++++++++++++++----- 3 files changed, 34 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 436ecd382cd..9306c9b99eb 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -556,7 +556,7 @@ namespace return false; } - static ALWAYS_INLINE bool insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void insertAll(const HashJoin &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -567,10 +567,9 @@ namespace /// The first element of the list is stored in the value of the hash table, the rest in the pool. emplace_result.getMapped().insert({stored_block, i}, pool); } - return true; } - static ALWAYS_INLINE bool insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + static ALWAYS_INLINE void insertAsof(HashJoin & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, const IColumn & asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -580,7 +579,6 @@ namespace if (emplace_result.isInserted()) time_series_map = new (time_series_map) typename Map::mapped_type(createAsofRowRef(asof_type, join.getAsofInequality())); (*time_series_map)->insert(asof_column, stored_block, i); - return true; } }; @@ -599,7 +597,9 @@ namespace auto key_getter = createKeyGetter(key_columns, key_sizes); - is_inserted = false; + /// For ALL and ASOF join always insert values + is_inserted = !mapped_one || is_asof_join; + for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) @@ -615,11 +615,11 @@ namespace continue; if constexpr (is_asof_join) - is_inserted |= Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, *asof_column); else if constexpr (mapped_one) is_inserted |= Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - is_inserted |= Inserter::insertAll(join, map, key_getter, stored_block, i, pool); + Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } return map.getBufferSizeInCells(); } diff --git a/tests/queries/0_stateless/02725_any_join_single_row.reference b/tests/queries/0_stateless/02725_any_join_single_row.reference index 5d748fc6dbb..1e940bdc71e 100644 --- a/tests/queries/0_stateless/02725_any_join_single_row.reference +++ b/tests/queries/0_stateless/02725_any_join_single_row.reference @@ -1,3 +1,6 @@ Join(ANY, LEFT, key) 0 1 Join(ANY, LEFT, key) 1 1 Join(ANY, LEFT, key) 1 1 +1 +1 +1 diff --git a/tests/queries/0_stateless/02725_any_join_single_row.sql b/tests/queries/0_stateless/02725_any_join_single_row.sql index 5e5c959c278..f7ddd2f402b 100644 --- a/tests/queries/0_stateless/02725_any_join_single_row.sql +++ b/tests/queries/0_stateless/02725_any_join_single_row.sql @@ -1,26 +1,41 @@ -CREATE TABLE join_test -( - `key` UInt64, - `value` UInt64 -) -ENGINE = Join(ANY, LEFT, key); +DROP TABLE IF EXISTS join_test; +DROP TABLE IF EXISTS join_test_right; +CREATE TABLE join_test ( `key` UInt64, `value` UInt64 ) ENGINE = Join(ANY, LEFT, key); + +-- Save table size before inserting any rows CREATE TEMPORARY TABLE initial_table_size AS SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); +-- Check that table size is less than 100K SELECT engine_full, total_rows, total_bytes < 100_000 FROM initial_table_size; INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); - +-- Save table size after inserting one row CREATE TEMPORARY TABLE one_row_table_size AS SELECT engine_full, total_rows, total_bytes FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); +-- Check that table size is less than 2x after inserting one row SELECT engine_full, total_rows, total_bytes < 2 * (SELECT total_bytes FROM initial_table_size) FROM one_row_table_size; -INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); -INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); +-- Insert some more rows with the same key INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(1); INSERT INTO join_test (key, value) SELECT 1, number FROM numbers(10_000); +-- Check that rows with the same key are not duplicated SELECT engine_full, total_rows, total_bytes == (SELECT total_bytes FROM one_row_table_size) FROM system.tables WHERE (name = 'join_test') AND (database = currentDatabase()); + +-- For RIGHT join we save all rows from the right table +CREATE TABLE join_test_right ( `key` UInt64, `value` UInt64 ) ENGINE = Join(ANY, RIGHT, key); + +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(1); +SELECT count() == 3 FROM (SELECT 1 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; +INSERT INTO join_test_right (key, value) SELECT 1, number FROM numbers(7); +SELECT count() == 10 FROM (SELECT 1 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; +SELECT count() == 10 FROM (SELECT 2 as key) t1 ANY RIGHT JOIN join_test_right ON t1.key = join_test_right.key; + +DROP TABLE IF EXISTS join_test; +DROP TABLE IF EXISTS join_test_right; From 1544067fb37b53b5ba0e1101db9ab068e9903217 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 1 Jun 2023 23:28:19 +0300 Subject: [PATCH 13/26] Update run.sh --- docker/test/unit/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/unit/run.sh b/docker/test/unit/run.sh index abc35fa40d2..a4784466e27 100644 --- a/docker/test/unit/run.sh +++ b/docker/test/unit/run.sh @@ -3,5 +3,5 @@ set -x service zookeeper start && sleep 7 && /usr/share/zookeeper/bin/zkCli.sh -server localhost:2181 -create create /clickhouse_test ''; -gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt +timeout 40m gdb -q -ex 'set print inferior-events off' -ex 'set confirm off' -ex 'set print thread-events off' -ex run -ex bt -ex quit --args ./unit_tests_dbms | tee test_output/test_result.txt ./process_unit_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv From c5165c2236bfceed4ec9246d6964fd2f77f68a07 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 2 Jun 2023 03:25:12 +0200 Subject: [PATCH 14/26] More thorough check for the availability of TaskStats --- src/Common/TaskStatsInfoGetter.cpp | 5 ++++- src/Common/ThreadStatus.h | 1 - 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 25030ee9670..d21229609dd 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -9,6 +9,7 @@ #include "hasLinuxCapability.h" #include +#include #include #include @@ -202,10 +203,12 @@ bool checkPermissionsImpl() /// Check that we can successfully initialize TaskStatsInfoGetter. /// It will ask about family id through Netlink. /// On some LXC containers we have capability but we still cannot use Netlink. + /// There is an evidence that Linux fedora-riscv 6.1.22 gives something strange instead of the expected result. try { - TaskStatsInfoGetter(); + ::taskstats stats{}; + TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 600dfc56d2b..061959d9f1f 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -37,7 +37,6 @@ class QueryThreadLog; class TasksStatsCounters; struct RUsageCounters; struct PerfEventsCounters; -class TaskStatsInfoGetter; class InternalTextLogsQueue; struct ViewRuntimeData; class QueryViewsLog; From f6da66cbb87d0c7c54c9b885cb5efeb0c330f801 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Thu, 1 Jun 2023 18:27:34 +0000 Subject: [PATCH 15/26] Fix some typos --- docs/en/sql-reference/functions/files.md | 8 +- .../functions/functions-for-nulls.md | 117 ++-- .../functions/logical-functions.md | 24 +- .../functions/other-functions.md | 549 +++++++----------- .../functions/random-functions.md | 181 ++++-- 5 files changed, 442 insertions(+), 437 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 5cd2d8e0a74..73d72aa50e5 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -6,7 +6,7 @@ sidebar_label: Files ## file -Reads file as string and loads the data into the specified column. The actual file content is not interpreted. +Reads a file as string and loads the data into the specified column. The file content is not interpreted. Also see table function [file](../table-functions/file.md). @@ -18,15 +18,13 @@ file(path[, default]) **Arguments** -- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports the following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. -- `default` — The value that will be returned in the case the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). +- `path` — The path of the file relative to [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Supports wildcards `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` are numbers and `'abc', 'def'` are strings. +- `default` — The value returned if the file does not exist or cannot be accessed. Supported data types: [String](../../sql-reference/data-types/string.md) and [NULL](../../sql-reference/syntax.md#null-literal). **Example** Inserting data from files a.txt and b.txt into a table as strings: -Query: - ``` sql INSERT INTO table SELECT file('a.txt'), file('b.txt'); ``` diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index 0b7df54b776..f4ca27e9b16 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -8,7 +8,7 @@ sidebar_label: Nullable ## isNull -Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). ``` sql isNull(x) @@ -18,7 +18,7 @@ Alias: `ISNULL`. **Arguments** -- `x` — A value with a non-compound data type. +- `x` — A value of non-compound data type. **Returned value** @@ -27,7 +27,7 @@ Alias: `ISNULL`. **Example** -Input table +Table: ``` text ┌─x─┬────y─┐ @@ -36,12 +36,14 @@ Input table └───┴──────┘ ``` -Query +Query: ``` sql SELECT x FROM t_null WHERE isNull(y); ``` +Result: + ``` text ┌─x─┐ │ 1 │ @@ -50,7 +52,7 @@ SELECT x FROM t_null WHERE isNull(y); ## isNotNull -Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal). +Returns whether the argument is not [NULL](../../sql-reference/syntax.md#null-literal). ``` sql isNotNull(x) @@ -58,16 +60,16 @@ isNotNull(x) **Arguments:** -- `x` — A value with a non-compound data type. +- `x` — A value of non-compound data type. **Returned value** -- `0` if `x` is `NULL`. - `1` if `x` is not `NULL`. +- `0` if `x` is `NULL`. **Example** -Input table +Table: ``` text ┌─x─┬────y─┐ @@ -76,12 +78,14 @@ Input table └───┴──────┘ ``` -Query +Query: ``` sql SELECT x FROM t_null WHERE isNotNull(y); ``` +Result: + ``` text ┌─x─┐ │ 2 │ @@ -90,7 +94,7 @@ SELECT x FROM t_null WHERE isNotNull(y); ## coalesce -Checks from left to right whether `NULL` arguments were passed and returns the first non-`NULL` argument. +Returns the leftmost non-`NULL` argument. ``` sql coalesce(x,...) @@ -98,11 +102,11 @@ coalesce(x,...) **Arguments:** -- Any number of parameters of a non-compound type. All parameters must be compatible by data type. +- Any number of parameters of non-compound type. All parameters must be of mutually compatible data types. **Returned values** -- The first non-`NULL` argument. +- The first non-`NULL` argument - `NULL`, if all arguments are `NULL`. **Example** @@ -110,10 +114,10 @@ coalesce(x,...) Consider a list of contacts that may specify multiple ways to contact a customer. ``` text -┌─name─────┬─mail─┬─phone─────┬──icq─┐ -│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ -│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -└──────────┴──────┴───────────┴──────┘ +┌─name─────┬─mail─┬─phone─────┬──telegram─┐ +│ client 1 │ ᴺᵁᴸᴸ │ 123-45-67 │ 123 │ +│ client 2 │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└──────────┴──────┴───────────┴───────────┘ ``` The `mail` and `phone` fields are of type String, but the `icq` field is `UInt32`, so it needs to be converted to `String`. @@ -121,22 +125,22 @@ The `mail` and `phone` fields are of type String, but the `icq` field is `UInt32 Get the first available contact method for the customer from the contact list: ``` sql -SELECT name, coalesce(mail, phone, CAST(icq,'Nullable(String)')) FROM aBook; +SELECT name, coalesce(mail, phone, CAST(telegram,'Nullable(String)')) FROM aBook; ``` ``` text -┌─name─────┬─coalesce(mail, phone, CAST(icq, 'Nullable(String)'))─┐ -│ client 1 │ 123-45-67 │ -│ client 2 │ ᴺᵁᴸᴸ │ -└──────────┴──────────────────────────────────────────────────────┘ +┌─name─────┬─coalesce(mail, phone, CAST(telegram, 'Nullable(String)'))─┐ +│ client 1 │ 123-45-67 │ +│ client 2 │ ᴺᵁᴸᴸ │ +└──────────┴───────────────────────────────────────────────────────────┘ ``` ## ifNull -Returns an alternative value if the main argument is `NULL`. +Returns an alternative value if the argument is `NULL`. ``` sql -ifNull(x,alt) +ifNull(x, alt) ``` **Arguments:** @@ -146,25 +150,33 @@ ifNull(x,alt) **Returned values** -- The value `x`, if `x` is not `NULL`. -- The value `alt`, if `x` is `NULL`. +- `x` if `x` is not `NULL`. +- `alt` if `x` is `NULL`. **Example** +Query: + ``` sql SELECT ifNull('a', 'b'); ``` +Result: + ``` text ┌─ifNull('a', 'b')─┐ │ a │ └──────────────────┘ ``` +Query: + ``` sql SELECT ifNull(NULL, 'b'); ``` +Result: + ``` text ┌─ifNull(NULL, 'b')─┐ │ b │ @@ -173,7 +185,7 @@ SELECT ifNull(NULL, 'b'); ## nullIf -Returns `NULL` if the arguments are equal. +Returns `NULL` if both arguments are equal. ``` sql nullIf(x, y) @@ -181,29 +193,37 @@ nullIf(x, y) **Arguments:** -`x`, `y` — Values for comparison. They must be compatible types, or ClickHouse will generate an exception. +`x`, `y` — Values to compare. Must be of compatible types. **Returned values** -- `NULL`, if the arguments are equal. -- The `x` value, if the arguments are not equal. +- `NULL` if the arguments are equal. +- `x` if the arguments are not equal. **Example** +Query: + ``` sql SELECT nullIf(1, 1); ``` +Result: + ``` text ┌─nullIf(1, 1)─┐ │ ᴺᵁᴸᴸ │ └──────────────┘ ``` +Query: + ``` sql SELECT nullIf(1, 2); ``` +Result: + ``` text ┌─nullIf(1, 2)─┐ │ 1 │ @@ -212,7 +232,7 @@ SELECT nullIf(1, 2); ## assumeNotNull -Results in an equivalent non-`Nullable` value for a [Nullable](../../sql-reference/data-types/nullable.md) type. In case the original value is `NULL` the result is undetermined. See also `ifNull` and `coalesce` functions. +Returns the corresponding non-`Nullable` value for a value of [Nullable](../../sql-reference/data-types/nullable.md) type. If the original value is `NULL`, an arbitrary result can be returned. See also functions `ifNull` and `coalesce`. ``` sql assumeNotNull(x) @@ -224,36 +244,29 @@ assumeNotNull(x) **Returned values** -- The original value from the non-`Nullable` type, if it is not `NULL`. -- Implementation specific result if the original value was `NULL`. +- The input value as non-`Nullable` type, if it is not `NULL`. +- An arbirary value, if the input value is `NULL`. **Example** -Consider the `t_null` table. - -``` sql -SHOW CREATE TABLE t_null; -``` +Table: ``` text -┌─statement─────────────────────────────────────────────────────────────────┐ -│ CREATE TABLE default.t_null ( x Int8, y Nullable(Int8)) ENGINE = TinyLog │ -└───────────────────────────────────────────────────────────────────────────┘ -``` -``` text ┌─x─┬────y─┐ │ 1 │ ᴺᵁᴸᴸ │ │ 2 │ 3 │ └───┴──────┘ ``` -Apply the `assumeNotNull` function to the `y` column. +Query: ``` sql -SELECT assumeNotNull(y) FROM t_null; +SELECT assumeNotNull(y) FROM table; ``` +Result: + ``` text ┌─assumeNotNull(y)─┐ │ 0 │ @@ -261,10 +274,14 @@ SELECT assumeNotNull(y) FROM t_null; └──────────────────┘ ``` +Query: + ``` sql SELECT toTypeName(assumeNotNull(y)) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(assumeNotNull(y))─┐ │ Int8 │ @@ -282,28 +299,36 @@ toNullable(x) **Arguments:** -- `x` — The value of any non-compound type. +- `x` — A value of non-compound type. **Returned value** -- The input value with a `Nullable` type. +- The input value but of `Nullable` type. **Example** +Query: + ``` sql SELECT toTypeName(10); ``` +Result: + ``` text ┌─toTypeName(10)─┐ │ UInt8 │ └────────────────┘ ``` +Query: + ``` sql SELECT toTypeName(toNullable(10)); ``` +Result: + ``` text ┌─toTypeName(toNullable(10))─┐ │ Nullable(UInt8) │ diff --git a/docs/en/sql-reference/functions/logical-functions.md b/docs/en/sql-reference/functions/logical-functions.md index f5a1a6aac12..2e1a8f28227 100644 --- a/docs/en/sql-reference/functions/logical-functions.md +++ b/docs/en/sql-reference/functions/logical-functions.md @@ -12,7 +12,9 @@ Zero as an argument is considered `false`, non-zero values are considered `true` ## and -Calculates the logical conjunction between two or more values. +Calculates the logical conjunction of two or more values. + +Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`. **Syntax** @@ -20,9 +22,7 @@ Calculates the logical conjunction between two or more values. and(val1, val2...) ``` -Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `(val_1 AND val_2 AND ... AND val_{i-1})` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT and(number = 2, intDiv(1, number)) FROM numbers(5)`. - -Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-operator). +Alias: The [AND operator](../../sql-reference/operators/index.md#logical-and-operator). **Arguments** @@ -30,7 +30,7 @@ Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-ope **Returned value** -- `0`, if there at least one argument evaluates to `false`, +- `0`, if at least one argument evaluates to `false`, - `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`, - `1`, otherwise. @@ -66,7 +66,9 @@ Result: ## or -Calculates the logical disjunction between two or more values. +Calculates the logical disjunction of two or more values. + +Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`. **Syntax** @@ -74,9 +76,7 @@ Calculates the logical disjunction between two or more values. or(val1, val2...) ``` -Setting [short_circuit_function_evaluation](../../operations/settings/settings.md#short-circuit-function-evaluation) controls whether short-circuit evaluation is used. If enabled, `val_i` is evaluated only if `((NOT val_1) AND (NOT val_2) AND ... AND (NOT val_{i-1}))` is `true`. For example, with short-circuit evaluation, no division-by-zero exception is thrown when executing the query `SELECT or(number = 0, intDiv(1, number) != 0) FROM numbers(5)`. - -Alias: The [OR Operator](../../sql-reference/operators/index.md#logical-or-operator). +Alias: The [OR operator](../../sql-reference/operators/index.md#logical-or-operator). **Arguments** @@ -120,7 +120,7 @@ Result: ## not -Calculates logical negation of a value. +Calculates the logical negation of a value. **Syntax** @@ -128,7 +128,7 @@ Calculates logical negation of a value. not(val); ``` -Alias: The [Negation Operator](../../sql-reference/operators/index.md#logical-negation-operator). +Alias: The [Negation operator](../../sql-reference/operators/index.md#logical-negation-operator). **Arguments** @@ -158,7 +158,7 @@ Result: ## xor -Calculates the logical exclusive disjunction between two or more values. For more than two values the function first xor-s the first two values, then xor-s the result with the third value etc. +Calculates the logical exclusive disjunction of two or more values. For more than two input values, the function first xor-s the first two values, then xor-s the result with the third value etc. **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index efe1a77c285..037eb9e63c5 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -8,12 +8,12 @@ sidebar_label: Other ## hostName() -Returns a string with the name of the host that this function was performed on. For distributed processing, this is the name of the remote server host, if the function is performed on a remote server. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +Returns the name of the host on which this function was executed. If the function executes on a remote server (distributed processing), the remote server name is returned. +If the function executes in the context of a distributed table, it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## getMacro -Gets a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. +Returns a named value from the [macros](../../operations/server-configuration-parameters/settings.md#macros) section of the server configuration. **Syntax** @@ -23,7 +23,7 @@ getMacro(name); **Arguments** -- `name` — Name to retrieve from the `macros` section. [String](../../sql-reference/data-types/string.md#string). +- `name` — Macro name to retrieve from the `` section. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -33,7 +33,7 @@ Type: [String](../../sql-reference/data-types/string.md). **Example** -The example `macros` section in the server configuration file: +Example `` section in the server configuration file: ``` xml @@ -55,7 +55,7 @@ Result: └──────────────────┘ ``` -An alternative way to get the same value: +The same value can be retrieved as follows: ``` sql SELECT * FROM system.macros @@ -70,7 +70,7 @@ WHERE macro = 'test'; ## FQDN -Returns the fully qualified domain name. +Returns the fully qualified domain name of the ClickHouse server. **Syntax** @@ -88,8 +88,6 @@ Type: `String`. **Example** -Query: - ``` sql SELECT FQDN(); ``` @@ -104,52 +102,61 @@ Result: ## basename -Extracts the trailing part of a string after the last slash or backslash. This function if often used to extract the filename from a path. +Extracts the tail of a string following its last slash or backslash. This function if often used to extract the filename from a path. ``` sql -basename( expr ) +basename(expr) ``` **Arguments** -- `expr` — Expression resulting in a [String](../../sql-reference/data-types/string.md) type value. All the backslashes must be escaped in the resulting value. +- `expr` — A value of type [String](../../sql-reference/data-types/string.md). Backslashes must be escaped. **Returned Value** A string that contains: -- The trailing part of a string after the last slash or backslash. - - If the input string contains a path ending with slash or backslash, for example, `/` or `c:\`, the function returns an empty string. - +- The tail of the input string after its last slash or backslash. If the input string ends with a slash or backslash (e.g. `/` or `c:\`), the function returns an empty string. - The original string if there are no slashes or backslashes. **Example** +Query: + ``` sql SELECT 'some/long/path/to/file' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT 'some\\long\\path\\to\\file' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────────────┬─basename('some\\long\\path\\to\\file')─┐ │ some\long\path\to\file │ file │ └────────────────────────┴────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT 'some-file-name' AS a, basename(a) ``` +Result: + ``` text ┌─a──────────────┬─basename('some-file-name')─┐ │ some-file-name │ some-file-name │ @@ -159,7 +166,7 @@ SELECT 'some-file-name' AS a, basename(a) ## visibleWidth(x) Calculates the approximate width when outputting values to the console in text format (tab-separated). -This function is used by the system for implementing Pretty formats. +This function is used by the system to implement Pretty formats. `NULL` is represented as a string corresponding to `NULL` in `Pretty` formats. @@ -175,18 +182,18 @@ SELECT visibleWidth(NULL) ## toTypeName(x) -Returns a string containing the type name of the passed argument. +Returns the type name of the passed argument. -If `NULL` is passed to the function as input, then it returns the `Nullable(Nothing)` type, which corresponds to an internal `NULL` representation in ClickHouse. +If `NULL` is passed, then the function returns type `Nullable(Nothing)`, which corresponds to ClickHouse's internal `NULL` representation. ## blockSize() -Gets the size of the block. -In ClickHouse, queries are always run on blocks (sets of column parts). This function allows getting the size of the block that you called it for. +In ClickHouse, queries are processed in blocks (chunks). +This function returns the size (row count) of the block the function is called on. ## byteSize -Returns estimation of uncompressed byte size of its arguments in memory. +Returns an estimation of uncompressed byte size of its arguments in memory. **Syntax** @@ -206,7 +213,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** -For [String](../../sql-reference/data-types/string.md) arguments the funtion returns the string length + 9 (terminating zero + length). +For [String](../../sql-reference/data-types/string.md) arguments, the funtion returns the string length + 9 (terminating zero + length). Query: @@ -265,7 +272,7 @@ byteSize(Float32): 4 byteSize(Float64): 8 ``` -If the function takes multiple arguments, it returns their combined byte size. +If the function has multiple arguments, the function accumulates their byte sizes. Query: @@ -283,30 +290,30 @@ Result: ## materialize(x) -Turns a constant into a full column containing just one value. -In ClickHouse, full columns and constants are represented differently in memory. Functions work differently for constant arguments and normal arguments (different code is executed), although the result is almost always the same. This function is for debugging this behavior. +Turns a constant into a full column containing a single value. +Full columns and constants are represented differently in memory. Functions usually execute different code for normal and constant arguments, although the result should typically be the same. This function can be used to debug this behavior. ## ignore(…) -Accepts any arguments, including `NULL`. Always returns 0. -However, the argument is still evaluated. This can be used for benchmarks. +Accepts any arguments, including `NULL` and does nothing. Always returns 0. +The argument is internally still evaluated. Useful e.g. for benchmarks. ## sleep(seconds) -Sleeps ‘seconds’ seconds on each data block. You can specify an integer or a floating-point number. +Sleeps ‘seconds’ seconds for each data block. The sleep time can be specified as integer or as floating-point number. ## sleepEachRow(seconds) -Sleeps ‘seconds’ seconds on each row. You can specify an integer or a floating-point number. +Sleeps ‘seconds’ seconds for each row. The sleep time can be specified as integer or as floating-point number. ## currentDatabase() Returns the name of the current database. -You can use this function in table engine parameters in a CREATE TABLE query where you need to specify the database. +Useful in table engine parameters of `CREATE TABLE` queries where you need to specify the database. ## currentUser() -Returns the login of current user. Login of user, that initiated query, will be returned in case distributed query. +Returns the name of the current user. In case of a distributed query, the name of the user who initiated the query is returned. ``` sql SELECT currentUser(); @@ -316,15 +323,13 @@ Alias: `user()`, `USER()`. **Returned values** -- Login of current user. -- Login of user that initiated query in case of distributed query. +- The name of the current user. +- In distributed queries, the login of the user who initiated the query. Type: `String`. **Example** -Query: - ``` sql SELECT currentUser(); ``` @@ -339,11 +344,11 @@ Result: ## isConstant -Checks whether the argument is a constant expression. +Returns whether the argument is a constant expression. -A constant expression means an expression whose resulting value is known at the query analysis (i.e. before execution). For example, expressions over [literals](../../sql-reference/syntax.md#literals) are constant expressions. +A constant expression is an expression whose result is known during query analysis, i.e. before execution. For example, expressions over [literals](../../sql-reference/syntax.md#literals) are constant expressions. -The function is intended for development, debugging and demonstration. +This function is mostly intended for development, debugging and demonstration. **Syntax** @@ -357,8 +362,8 @@ isConstant(x) **Returned values** -- `1` — `x` is constant. -- `0` — `x` is non-constant. +- `1` if `x` is constant. +- `0` if `x` is non-constant. Type: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -408,23 +413,25 @@ Result: ## isFinite(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is not infinite and not a NaN, otherwise 0. +Returns 1 if the Float32 or Float64 argument not infinite and not a NaN, otherwise this function returns 0. ## isInfinite(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is infinite, otherwise 0. Note that 0 is returned for a NaN. +Returns 1 if the Float32 or Float64 argument is infinite, otherwise this function returns 0. Note that 0 is returned for a NaN. ## ifNotFinite -Checks whether floating point value is finite. +Checks whether a floating point value is finite. **Syntax** - ifNotFinite(x,y) +``` sql +ifNotFinite(x,y) +``` **Arguments** -- `x` — Value to be checked for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). +- `x` — Value to check for infinity. Type: [Float\*](../../sql-reference/data-types/float.md). - `y` — Fallback value. Type: [Float\*](../../sql-reference/data-types/float.md). **Returned value** @@ -444,23 +451,23 @@ Result: │ inf │ 42 │ └─────────┴───────────────────────────────┘ -You can get similar result by using [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. +You can get similar result by using the [ternary operator](../../sql-reference/functions/conditional-functions.md#ternary-operator): `isFinite(x) ? x : y`. ## isNaN(x) -Accepts Float32 and Float64 and returns UInt8 equal to 1 if the argument is a NaN, otherwise 0. +Returns 1 if the Float32 and Float64 argument is NaN, otherwise this function 0. ## hasColumnInTable(\[‘hostname’\[, ‘username’\[, ‘password’\]\],\] ‘database’, ‘table’, ‘column’) -Accepts constant strings: database name, table name, and column name. Returns a UInt8 constant expression equal to 1 if there is a column, otherwise 0. If the hostname parameter is set, the test will run on a remote server. -The function throws an exception if the table does not exist. +Given the database name, the table name, and the column name as constant strings, returns 1 if the given column exists, otherwise 0. If parameter `hostname` is given, the check is performed on a remote server. +If the table does not exist, an exception is thrown. For elements in a nested data structure, the function checks for the existence of a column. For the nested data structure itself, the function returns 0. ## bar -Allows building a unicode-art diagram. +Builds a bar chart. -`bar(x, min, max, width)` draws a band with a width proportional to `(x - min)` and equal to `width` characters when `x = max`. +`bar(x, min, max, width)` draws a band with width proportional to `(x - min)` and equal to `width` characters when `x = max`. **Arguments** @@ -520,23 +527,23 @@ There are two variations of this function: `x` – What to transform. -`array_from` – Constant array of values for converting. +`array_from` – Constant array of values to convert. `array_to` – Constant array of values to convert the values in ‘from’ to. `default` – Which value to use if ‘x’ is not equal to any of the values in ‘from’. -`array_from` and `array_to` – Arrays of the same size. +`array_from` and `array_to` must have equally many elements. -Types: +Signature: + +For `x` equal to one of the elements in `array_from`, the function returns the corresponding element in `array_to`, i.e. the one at the same array index. Otherwise, it returns `default`. If multiple matching elements exist `array_from`, an arbitrary corresponding element from `array_to` is returned. `transform(T, Array(T), Array(U), U) -> U` `T` and `U` can be numeric, string, or Date or DateTime types. -Where the same letter is indicated (T or U), for numeric types these might not be matching types, but types that have a common type. -For example, the first argument can have the Int64 type, while the second has the Array(UInt16) type. - -If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the existing element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘default’. If there are multiple matching elements in ‘array_from’, it returns one of the matches. +The same letter (T or U) means that types must be mutually compatible and not necessarily equal. +For example, the first argument could have type `Int64`, while the second argument could have type `Array(UInt16)`. Example: @@ -560,12 +567,7 @@ ORDER BY c DESC ### transform(x, array_from, array_to) -Differs from the first variation in that the ‘default’ argument is omitted. -If the ‘x’ value is equal to one of the elements in the ‘array_from’ array, it returns the matching element (that is numbered the same) from the ‘array_to’ array. Otherwise, it returns ‘x’. - -Types: - -`transform(T, Array(T), Array(T)) -> T` +Similar to the other variation but has no ‘default’ argument. In case no match can be found, `x` is returned. Example: @@ -595,7 +597,7 @@ LIMIT 10 ## formatReadableDecimalSize(x) -Accepts the size (number of bytes). Returns a rounded size with a suffix (KB, MB, etc.) as a string. +Given a size (number of bytes), this function returns a readable, rounded size with suffix (KB, MB, etc.) as string. Example: @@ -616,7 +618,7 @@ SELECT ## formatReadableSize(x) -Accepts the size (number of bytes). Returns a rounded size with a suffix (KiB, MiB, etc.) as a string. +Given a size (number of bytes), this function returns a readable, rounded size with suffix (KiB, MiB, etc.) as string. Example: @@ -637,9 +639,7 @@ SELECT ## formatReadableQuantity(x) -Accepts the number. Returns a rounded number with a suffix (thousand, million, billion, etc.) as a string. - -It is useful for reading big numbers by human. +Given a number, this function returns a rounded number with suffix (thousand, million, billion, etc.) as string. Example: @@ -660,7 +660,7 @@ SELECT ## formatReadableTimeDelta -Accepts the time delta in seconds. Returns a time delta with (year, month, day, hour, minute, second) as a string. +Given a time interval (delta) in seconds, this function returns a time delta with year/month/day/hour/minute/second as string. **Syntax** @@ -670,8 +670,8 @@ formatReadableTimeDelta(column[, maximum_unit]) **Arguments** -- `column` — A column with numeric time delta. -- `maximum_unit` — Optional. Maximum unit to show. Acceptable values seconds, minutes, hours, days, months, years. +- `column` — A column with a numeric time delta. +- `maximum_unit` — Optional. Maximum unit to show. Acceptable values `seconds`, `minutes`, `hours`, `days`, `months`, `years`. Example: @@ -746,33 +746,32 @@ SELECT parseTimeDelta('1yr2mo') ## least(a, b) -Returns the smallest value from a and b. +Returns the smaller value of a and b. ## greatest(a, b) -Returns the largest value of a and b. +Returns the larger value of a and b. ## uptime() Returns the server’s uptime in seconds. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## version() -Returns the version of the server as a string. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +Returns the server version as a string. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. ## buildId() Returns the build ID generated by a compiler for the running ClickHouse server binary. -If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. - -## blockNumber +## blockNumber() Returns the sequence number of the data block where the row is located. -## rowNumberInBlock +## rowNumberInBlock() Returns the ordinal number of the row in the data block. Different data blocks are always recalculated. @@ -782,7 +781,7 @@ Returns the ordinal number of the row in the data block. This function only cons ## neighbor -The window function that provides access to a row at a specified offset which comes before or after the current row of a given column. +The window function that provides access to a row at a specified offset before or after the current row of a given column. **Syntax** @@ -792,23 +791,23 @@ neighbor(column, offset[, default_value]) The result of the function depends on the affected data blocks and the order of data in the block. -:::tip -It can reach the neighbor rows only inside the currently processed data block. +:::note +Only returns neighbor inside the currently processed data block. ::: -The rows order used during the calculation of `neighbor` can differ from the order of rows returned to the user. -To prevent that you can make a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. +The order of rows during calculation of `neighbor()` can differ from the order of rows returned to the user. +To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. **Arguments** - `column` — A column name or scalar expression. -- `offset` — The number of rows forwards or backwards from the current row of `column`. [Int64](../../sql-reference/data-types/int-uint.md). -- `default_value` — Optional. The value to be returned if offset goes beyond the scope of the block. Type of data blocks affected. +- `offset` — The number of rows to look before or ahead of the current row in `column`. [Int64](../../sql-reference/data-types/int-uint.md). +- `default_value` — Optional. The returned value if offset is beyond the block boundaries. Type of data blocks affected. **Returned values** -- Value for `column` in `offset` distance from current row if `offset` value is not outside block bounds. -- Default value for `column` if `offset` value is outside block bounds. If `default_value` is given, then it will be used. +- Value of `column` with `offset` distance from current row, if `offset` is not outside the block boundaries. +- The default value of `column` or `default_value` (if given), if `offset` is outside the block boundaries. Type: type of data blocks affected or default value type. @@ -899,17 +898,17 @@ Result: ## runningDifference(x) -Calculates the difference between successive row values ​​in the data block. -Returns 0 for the first row and the difference from the previous row for each subsequent row. +Calculates the difference between two consecutive row values in the data block. +Returns 0 for the first row, and for subsequent rows the difference to the previous row. -:::tip -It can reach the previous row only inside the currently processed data block. +:::note +Only returns differences inside the currently processed data block. ::: The result of the function depends on the affected data blocks and the order of data in the block. -The rows order used during the calculation of `runningDifference` can differ from the order of rows returned to the user. -To prevent that you can make a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. +The order of rows during calculation of `runningDifference()` can differ from the order of rows returned to the user. +To prevent that you can create a subquery with [ORDER BY](../../sql-reference/statements/select/order-by.md) and call the function from outside the subquery. Example: @@ -940,7 +939,7 @@ FROM └─────────┴─────────────────────┴───────┘ ``` -Please note - block size affects the result. With each new block, the `runningDifference` state is reset. +Please note that the block size affects the result. The internal state of `runningDifference` state is reset for each new block. ``` sql SELECT @@ -977,7 +976,7 @@ WHERE diff != 1 ## runningDifferenceStartingWithFirstValue -Same as for [runningDifference](./other-functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row. +Same as [runningDifference](./other-functions.md#other_functions-runningdifference), but returns the value of the first row as the value on the first row. ## runningConcurrency @@ -1039,7 +1038,7 @@ Result: ## MACNumToString(num) -Accepts a UInt64 number. Interprets it as a MAC address in big endian. Returns a string containing the corresponding MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). +Interprets a UInt64 number as a MAC address in big endian format. Returns the corresponding MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form) as string. ## MACStringToNum(s) @@ -1047,11 +1046,12 @@ The inverse function of MACNumToString. If the MAC address has an invalid format ## MACStringToOUI(s) -Accepts a MAC address in the format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form). Returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. +Given a MAC address in format AA:BB:CC:DD:EE:FF (colon-separated numbers in hexadecimal form), returns the first three octets as a UInt64 number. If the MAC address has an invalid format, it returns 0. ## getSizeOfEnumType Returns the number of fields in [Enum](../../sql-reference/data-types/enum.md). +An exception is thrown if the type is not `Enum`. ``` sql getSizeOfEnumType(value) @@ -1064,7 +1064,6 @@ getSizeOfEnumType(value) **Returned values** - The number of fields with `Enum` input values. -- An exception is thrown if the type is not `Enum`. **Example** @@ -1080,7 +1079,7 @@ SELECT getSizeOfEnumType( CAST('a' AS Enum8('a' = 1, 'b' = 2) ) ) AS x ## blockSerializedSize -Returns size on disk (without taking into account compression). +Returns the size on disk without considering compression. ``` sql blockSerializedSize(value[, value[, ...]]) @@ -1092,7 +1091,7 @@ blockSerializedSize(value[, value[, ...]]) **Returned values** -- The number of bytes that will be written to disk for block of values (without compression). +- The number of bytes that will be written to disk for block of values without compression. **Example** @@ -1112,7 +1111,7 @@ Result: ## toColumnTypeName -Returns the name of the class that represents the data type of the column in RAM. +Returns the internal name of the data type that represents the value. ``` sql toColumnTypeName(value) @@ -1124,31 +1123,39 @@ toColumnTypeName(value) **Returned values** -- A string with the name of the class that is used for representing the `value` data type in RAM. +- The internal data type name used to represent `value`. -**Example of the difference between`toTypeName ' and ' toColumnTypeName`** +**Example** + +Difference between `toTypeName ' and ' toColumnTypeName`: ``` sql SELECT toTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` +Result: + ``` text ┌─toTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ DateTime │ └─────────────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT toColumnTypeName(CAST('2018-01-01 01:02:03' AS DateTime)) ``` +Result: + ``` text ┌─toColumnTypeName(CAST('2018-01-01 01:02:03', 'DateTime'))─┐ │ Const(UInt32) │ └───────────────────────────────────────────────────────────┘ ``` -The example shows that the `DateTime` data type is stored in memory as `Const(UInt32)`. +The example shows that the `DateTime` data type is internally stored as `Const(UInt32)`. ## dumpColumnStructure @@ -1164,7 +1171,7 @@ dumpColumnStructure(value) **Returned values** -- A string describing the structure that is used for representing the `value` data type in RAM. +- A description of the column structure used for representing `value`. **Example** @@ -1180,7 +1187,7 @@ SELECT dumpColumnStructure(CAST('2018-01-01 01:02:03', 'DateTime')) ## defaultValueOfArgumentType -Outputs the default value for the data type. +Returns the default value for the given data type. Does not include default values for custom columns set by the user. @@ -1200,20 +1207,28 @@ defaultValueOfArgumentType(expression) **Example** +Query: + ``` sql SELECT defaultValueOfArgumentType( CAST(1 AS Int8) ) ``` +Result: + ``` text ┌─defaultValueOfArgumentType(CAST(1, 'Int8'))─┐ │ 0 │ └─────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ``` +Result: + ``` text ┌─defaultValueOfArgumentType(CAST(1, 'Nullable(Int8)'))─┐ │ ᴺᵁᴸᴸ │ @@ -1222,7 +1237,7 @@ SELECT defaultValueOfArgumentType( CAST(1 AS Nullable(Int8) ) ) ## defaultValueOfTypeName -Outputs the default value for given type name. +Returns the default value for the given type name. Does not include default values for custom columns set by the user. @@ -1242,20 +1257,28 @@ defaultValueOfTypeName(type) **Example** +Query: + ``` sql SELECT defaultValueOfTypeName('Int8') ``` +Result: + ``` text ┌─defaultValueOfTypeName('Int8')─┐ │ 0 │ └────────────────────────────────┘ ``` +Query: + ``` sql SELECT defaultValueOfTypeName('Nullable(Int8)') ``` +Result: + ``` text ┌─defaultValueOfTypeName('Nullable(Int8)')─┐ │ ᴺᵁᴸᴸ │ @@ -1263,9 +1286,10 @@ SELECT defaultValueOfTypeName('Nullable(Int8)') ``` ## indexHint -The function is intended for debugging and introspection purposes. The function ignores it's argument and always returns 1. Arguments are not even evaluated. -But for the purpose of index analysis, the argument of this function is analyzed as if it was present directly without being wrapped inside `indexHint` function. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. +This function is intended for debugging and introspection. It ignores its argument and always returns 1. The arguments are not evaluated. + +But during index analysis, the argument of this function is assumed to be not wrapped in `indexHint`. This allows to select data in index ranges by the corresponding condition but without further filtering by this condition. The index in ClickHouse is sparse and using `indexHint` will yield more data than specifying the same condition directly. **Syntax** @@ -1275,13 +1299,13 @@ SELECT * FROM table WHERE indexHint() **Returned value** -1. Type: [Uint8](https://clickhouse.com/docs/en/data_types/int_uint/#diapazony-uint). +Type: [Uint8](https://clickhouse.com/docs/en/data_types/int_uint/#diapazony-uint). **Example** Here is the example of test data from the table [ontime](../../getting-started/example-datasets/ontime.md). -Input table: +Table: ```sql SELECT count() FROM ontime @@ -1295,9 +1319,7 @@ SELECT count() FROM ontime The table has indexes on the fields `(FlightDate, (Year, FlightDate))`. -Create a query, where the index is not used. - -Query: +Create a query which does not use the index: ```sql SELECT FlightDate AS k, count() FROM ontime GROUP BY k ORDER BY k @@ -1318,15 +1340,13 @@ Result: └────────────┴─────────┘ ``` -To apply the index, select a specific date. - -Query: +To apply the index, select a specific date: ```sql SELECT FlightDate AS k, count() FROM ontime WHERE k = '2017-09-15' GROUP BY k ORDER BY k ``` -By using the index, ClickHouse processed a significantly smaller number of rows (`Processed 32.74 thousand rows`). +ClickHouse now uses the index to process a significantly smaller number of rows (`Processed 32.74 thousand rows`). Result: @@ -1336,7 +1356,7 @@ Result: └────────────┴─────────┘ ``` -Now wrap the expression `k = '2017-09-15'` into `indexHint` function. +Now wrap the expression `k = '2017-09-15'` in function `indexHint`: Query: @@ -1350,9 +1370,9 @@ GROUP BY k ORDER BY k ASC ``` -ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). +ClickHouse used the index the same way as previously (`Processed 32.74 thousand rows`). The expression `k = '2017-09-15'` was not used when generating the result. -In examle the `indexHint` function allows to see adjacent dates. +In example, the `indexHint` function allows to see adjacent dates. Result: @@ -1369,7 +1389,7 @@ Result: Creates an array with a single value. -Used for internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). +Used for the internal implementation of [arrayJoin](../../sql-reference/functions/array-join.md#functions_arrayjoin). ``` sql SELECT replicate(x, arr); @@ -1377,12 +1397,12 @@ SELECT replicate(x, arr); **Arguments:** -- `arr` — Original array. ClickHouse creates a new array of the same length as the original and fills it with the value `x`. -- `x` — The value that the resulting array will be filled with. +- `arr` — An array. +- `x` — The value to fill the result array with. **Returned value** -An array filled with the value `x`. +An array of the lame length as `arr` filled with value `x`. Type: `Array`. @@ -1404,7 +1424,7 @@ Result: ## filesystemAvailable -Returns amount of remaining space on the filesystem where the files of the databases located. It is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for OS. +Returns the amount of free space in the filesystem hosting the database persistence. The returned value is always smaller than total free space ([filesystemFree](#filesystemfree)) because some space is reserved for the operating system. **Syntax** @@ -1423,20 +1443,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemAvailable()) AS "Available space", toTypeName(filesystemAvailable()) AS "Type"; +SELECT formatReadableSize(filesystemAvailable()) AS "Available space"; ``` Result: ``` text -┌─Available space─┬─Type───┐ -│ 30.75 GiB │ UInt64 │ -└─────────────────┴────────┘ +┌─Available space─┐ +│ 30.75 GiB │ +└─────────────────┘ ``` ## filesystemFree -Returns total amount of the free space on the filesystem where the files of the databases located. See also `filesystemAvailable` +Returns the total amount of the free space on the filesystem hosting the database persistence. See also `filesystemAvailable` **Syntax** @@ -1446,7 +1466,7 @@ filesystemFree() **Returned value** -- Amount of free space in bytes. +- The amount of free space in bytes. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -1455,20 +1475,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemFree()) AS "Free space", toTypeName(filesystemFree()) AS "Type"; +SELECT formatReadableSize(filesystemFree()) AS "Free space"; ``` Result: ``` text -┌─Free space─┬─Type───┐ -│ 32.39 GiB │ UInt64 │ -└────────────┴────────┘ +┌─Free space─┐ +│ 32.39 GiB │ +└────────────┘ ``` ## filesystemCapacity -Returns the capacity of the filesystem in bytes. For evaluation, the [path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) to the data directory must be configured. +Returns the capacity of the filesystem in bytes. Needs the [path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) to the data directory to be configured. **Syntax** @@ -1478,7 +1498,7 @@ filesystemCapacity() **Returned value** -- Capacity information of the filesystem in bytes. +- Capacity of the filesystem in bytes. Type: [UInt64](../../sql-reference/data-types/int-uint.md). @@ -1487,20 +1507,20 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). Query: ``` sql -SELECT formatReadableSize(filesystemCapacity()) AS "Capacity", toTypeName(filesystemCapacity()) AS "Type" +SELECT formatReadableSize(filesystemCapacity()) AS "Capacity"; ``` Result: ``` text -┌─Capacity──┬─Type───┐ -│ 39.32 GiB │ UInt64 │ -└───────────┴────────┘ +┌─Capacity──┐ +│ 39.32 GiB │ +└───────────┘ ``` ## initializeAggregation -Calculates result of aggregate function based on single value. It is intended to use this function to initialize aggregate functions with combinator [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state). You can create states of aggregate functions and insert them to columns of type [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction) or use initialized aggregates as default values. +Calculates the result of an aggregate function based on a single value. This function can be used to initialize aggregate functions with combinator [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state). You can create states of aggregate functions and insert them to columns of type [AggregateFunction](../../sql-reference/data-types/aggregatefunction.md#data-type-aggregatefunction) or use initialized aggregates as default values. **Syntax** @@ -1539,6 +1559,7 @@ Query: ```sql SELECT finalizeAggregation(state), toTypeName(state) FROM (SELECT initializeAggregation('sumState', number % 3) AS state FROM numbers(5)); ``` + Result: ```text @@ -1568,11 +1589,12 @@ INSERT INTO metrics VALUES (0, initializeAggregation('sumState', toUInt64(42))) ``` **See Also** + - [arrayReduce](../../sql-reference/functions/array-functions.md#arrayreduce) ## finalizeAggregation -Takes state of aggregate function. Returns result of aggregation (or finalized state when using[-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) combinator). +Given a state of aggregate function, this function returns the result of aggregation (or finalized state when using a [-State](../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) combinator). **Syntax** @@ -1667,15 +1689,16 @@ Result: ``` **See Also** + - [arrayReduce](../../sql-reference/functions/array-functions.md#arrayreduce) - [initializeAggregation](#initializeaggregation) ## runningAccumulate -Accumulates states of an aggregate function for each row of a data block. +Accumulates the states of an aggregate function for each row of a data block. -:::tip -The state is reset for each new data block. +:::note +The state is reset for each new block of data. ::: **Syntax** @@ -1726,10 +1749,10 @@ The subquery generates `sumState` for every number from `0` to `9`. `sumState` r The whole query does the following: -1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. -2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. -3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. -4. The actions are repeated until the block ends. +1. For the first row, `runningAccumulate` takes `sumState(0)` and returns `0`. +2. For the second row, the function merges `sumState(0)` and `sumState(1)` resulting in `sumState(0 + 1)`, and returns `1` as a result. +3. For the third row, the function merges `sumState(0 + 1)` and `sumState(2)` resulting in `sumState(0 + 1 + 2)`, and returns `3` as a result. +4. The actions are repeated until the block ends. The following example shows the `groupping` parameter usage: @@ -1780,7 +1803,7 @@ As you can see, `runningAccumulate` merges states for each group of rows separat The function lets you extract data from the table the same way as from a [dictionary](../../sql-reference/dictionaries/index.md). -Gets data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. +Gets the data from [Join](../../engines/table-engines/special/join.md#creating-a-table) tables using the specified join key. Only supports tables created with the `ENGINE = Join(ANY, LEFT, )` statement. @@ -1792,13 +1815,13 @@ joinGet(join_storage_table_name, `value_column`, join_keys) **Arguments** -- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicates where search is performed. The identifier is searched in the default database (see parameter `default_database` in the config file). To override the default database, use the `USE db_name` or specify the database and the table through the separator `db_name.db_table`, see the example. +- `join_storage_table_name` — an [identifier](../../sql-reference/syntax.md#syntax-identifiers) indicating where the search is performed. The identifier is searched in the default database (see setting `default_database` in the config file). To override the default database, use `USE db_name` or specify the database and the table through the separator `db_name.db_table` as in the example. - `value_column` — name of the column of the table that contains required data. - `join_keys` — list of keys. **Returned value** -Returns list of values corresponded to list of keys. +Returns a list of values corresponded to list of keys. If certain does not exist in source table then `0` or `null` will be returned based on [join_use_nulls](../../operations/settings/settings.md#join_use_nulls) setting. @@ -1825,7 +1848,7 @@ INSERT INTO db_test.id_val VALUES (1,11)(2,12)(4,13) Query: ``` sql -SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 +SELECT joinGet(db_test.id_val, 'val', toUInt32(number)) from numbers(4) SETTINGS join_use_nulls = 1 ``` Result: @@ -1845,7 +1868,7 @@ Result: This function is not available in ClickHouse Cloud. ::: -Evaluate external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing. +Evaluate an external catboost model. [CatBoost](https://catboost.ai) is an open-source gradient boosting library developed by Yandex for machine learing. Accepts a path to a catboost model and model arguments (features). Returns Float64. ``` sql @@ -1886,16 +1909,24 @@ See [Training and applying models](https://catboost.ai/docs/features/training.ht ## throwIf(x\[, message\[, error_code\]\]) -Throw an exception if the argument is non zero. -`message` - is an optional parameter: a constant string providing a custom error message -`error_code` - is an optional parameter: a constant integer providing a custom error code +Throw an exception if argument `x` is true. + +**Arguments** + +- `x` - the condition to check. +- `message` - a constant string providing a custom error message. Optional. +- `error_code` - A constant integer providing a custom error code. Optional. To use the `error_code` argument, configuration parameter `allow_custom_error_code_in_throwif` must be enabled. +**Example** + ``` sql SELECT throwIf(number = 3, 'Too many') FROM numbers(10); ``` +Result: + ``` text ↙ Progress: 0.00 rows, 0.00 B (0.00 rows/s., 0.00 B/s.) Received exception from server (version 19.14.1): Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. @@ -1903,7 +1934,7 @@ Code: 395. DB::Exception: Received from localhost:9000. DB::Exception: Too many. ## identity -Returns the same value that was used as its argument. Used for debugging and testing, allows to cancel using index, and get the query performance of a full scan. When query is analyzed for possible use of index, the analyzer does not look inside `identity` functions. Also constant folding is not applied too. +Returns its argument. Intended for debugging and testing. Allows to cancel using index, and get the query performance of a full scan. When the query is analyzed for possible use of an index, the analyzer ignores everything in `identity` functions. Also disables constant folding. **Syntax** @@ -1916,7 +1947,7 @@ identity(x) Query: ``` sql -SELECT identity(42) +SELECT identity(42); ``` Result: @@ -1927,164 +1958,6 @@ Result: └──────────────┘ ``` -## randomPrintableASCII - -Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. - -**Syntax** - -``` sql -randomPrintableASCII(length) -``` - -**Arguments** - -- `length` — Resulting string length. Positive integer. - - If you pass `length < 0`, behavior of the function is undefined. - -**Returned value** - -- String with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. - -Type: [String](../../sql-reference/data-types/string.md) - -**Example** - -``` sql -SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 -``` - -``` text -┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ -│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ -│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ -│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ -└────────┴────────────────────────────────┴──────────────────────────────────┘ -``` - -## randomString - -Generates a binary string of the specified length filled with random bytes (including zero bytes). - -**Syntax** - -``` sql -randomString(length) -``` - -**Arguments** - -- `length` — String length. Positive integer. - -**Returned value** - -- String filled with random bytes. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -``` sql -SELECT randomString(30) AS str, length(str) AS len FROM numbers(2) FORMAT Vertical; -``` - -Result: - -``` text -Row 1: -────── -str: 3 G : pT ?w тi k aV f6 -len: 30 - -Row 2: -────── -str: 9 ,] ^ ) ]?? 8 -len: 30 -``` - -**See Also** - -- [generateRandom](../../sql-reference/table-functions/generate.md#generaterandom) -- [randomPrintableASCII](../../sql-reference/functions/other-functions.md#randomascii) - - -## randomFixedString - -Generates a binary string of the specified length filled with random bytes (including zero bytes). - -**Syntax** - -``` sql -randomFixedString(length); -``` - -**Arguments** - -- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). - -**Returned value(s)** - -- String filled with random bytes. - -Type: [FixedString](../../sql-reference/data-types/fixedstring.md). - -**Example** - -Query: - -```sql -SELECT randomFixedString(13) as rnd, toTypeName(rnd) -``` - -Result: - -```text -┌─rnd──────┬─toTypeName(randomFixedString(13))─┐ -│ j▒h㋖HɨZ'▒ │ FixedString(13) │ -└──────────┴───────────────────────────────────┘ - -``` - -## randomStringUTF8 - -Generates a random string of a specified length. Result string contains valid UTF-8 code points. The value of code points may be outside of the range of assigned Unicode. - -**Syntax** - -``` sql -randomStringUTF8(length); -``` - -**Arguments** - -- `length` — Required length of the resulting string in code points. [UInt64](../../sql-reference/data-types/int-uint.md). - -**Returned value(s)** - -- UTF-8 random string. - -Type: [String](../../sql-reference/data-types/string.md). - -**Example** - -Query: - -```sql -SELECT randomStringUTF8(13) -``` - -Result: - -```text -┌─randomStringUTF8(13)─┐ -│ 𘤗𙉝д兠庇󡅴󱱎󦐪􂕌𔊹𓰛 │ -└──────────────────────┘ - -``` - ## getSetting Returns the current value of a [custom setting](../../operations/settings/index.md#custom_settings). @@ -2101,7 +1974,7 @@ getSetting('custom_setting'); **Returned value** -- The setting current value. +- The setting's current value. **Example** @@ -2110,7 +1983,7 @@ SET custom_a = 123; SELECT getSetting('custom_a'); ``` -**Result** +Result: ``` 123 @@ -2122,7 +1995,7 @@ SELECT getSetting('custom_a'); ## isDecimalOverflow -Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is out of its (or specified) precision. +Checks whether the [Decimal](../../sql-reference/data-types/decimal.md) value is outside its precision or outside the specified precision. **Syntax** @@ -2133,11 +2006,11 @@ isDecimalOverflow(d, [p]) **Arguments** - `d` — value. [Decimal](../../sql-reference/data-types/decimal.md). -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this paratemer could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). +- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This paratemer can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned values** -- `1` — Decimal value has more digits then it's precision allow, +- `1` — Decimal value has more digits then allowed by its precision, - `0` — Decimal value satisfies the specified precision. **Example** @@ -2159,7 +2032,7 @@ Result: ## countDigits -Returns number of decimal digits you need to represent the value. +Returns number of decimal digits need to represent a value. **Syntax** @@ -2199,9 +2072,7 @@ Result: ## errorCodeToName -**Returned value** - -- Variable name for the error code. +Returns the textual name of an error code. Type: [LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md). @@ -2220,7 +2091,7 @@ UNSUPPORTED_METHOD ## tcpPort Returns [native interface](../../interfaces/tcp.md) TCP port number listened by this server. -If it is executed in the context of a distributed table, then it generates a normal column, otherwise it produces a constant value. +If executed in the context of a distributed table, this function generates a normal column with values relevant to each shard. Otherwise it produces a constant value. **Syntax** @@ -2310,7 +2181,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## currentRoles -Returns the names of the roles which are current for the current user. The current roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If the `SET ROLE` statement was not used, the function `currentRoles` returns the same as `defaultRoles`. +Returns the roles assigned to the current user. The roles can be changed by the [SET ROLE](../../sql-reference/statements/set-role.md#set-role-statement) statement. If no `SET ROLE` statement was not, the function `currentRoles` returns the same as `defaultRoles`. **Syntax** @@ -2320,7 +2191,7 @@ currentRoles() **Returned value** -- List of the current roles for the current user. +- A list of the current roles for the current user. Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md)). @@ -2342,7 +2213,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## defaultRoles -Returns the names of the roles which are enabled by default for the current user when he logins. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. +Returns the roles which are enabled by default for the current user when he logs in. Initially these are all roles granted to the current user (see [GRANT](../../sql-reference/statements/grant.md#grant-select)), but that can be changed with the [SET DEFAULT ROLE](../../sql-reference/statements/set-role.md#set-default-role-statement) statement. **Syntax** @@ -2358,7 +2229,7 @@ Type: [Array](../../sql-reference/data-types/array.md)([String](../../sql-refere ## getServerPort -Returns the number of the server port. When the port is not used by the server, throws an exception. +Returns the server port number. When the port is not used by the server, throws an exception. **Syntax** @@ -2407,7 +2278,7 @@ Result: Returns the ID of the current query. Other parameters of a query can be extracted from the [system.query_log](../../operations/system-tables/query_log.md) table via `query_id`. -In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see example). +In contrast to [initialQueryID](#initial-query-id) function, `queryID` can return different results on different shards (see the example). **Syntax** @@ -2477,7 +2348,7 @@ Result: ## shardNum -Returns the index of a shard which processes a part of data for a distributed query. Indices are started from `1`. +Returns the index of a shard which processes a part of data in a distributed query. Indices are started from `1`. If a query is not distributed then constant value `0` is returned. **Syntax** diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index e90d537fb74..63d5174b494 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -31,7 +31,7 @@ Uses a linear congruential generator. ## randCanonical -Returns a Float64 value, evenly distributed in [0, 1). +Returns a random Float64 value, evenly distributed in interval [0, 1). ## randConstant @@ -54,11 +54,9 @@ Result: └────────────┴────────────┴──────────────┴────────────────┴─────────────────┴──────────────────────┘ ``` -# Functions for Generating Random Numbers based on a Distribution - ## randUniform -Returns a Float64 drawn uniformly from the interval between `min` and `max` ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)). +Returns a random Float64 drawn uniformly from interval [`min`, `max`) ([continuous uniform distribution](https://en.wikipedia.org/wiki/Continuous_uniform_distribution)). **Syntax** @@ -68,8 +66,8 @@ randUniform(min, max) **Arguments** -- `min` - `Float64` - min value of the range, -- `max` - `Float64` - max value of the range. +- `min` - `Float64` - left boundary of the range, +- `max` - `Float64` - reight boundary of the range. **Returned value** @@ -97,7 +95,7 @@ Result: ## randNormal -Returns a Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). +Returns a random Float64 drawn from a [normal distribution](https://en.wikipedia.org/wiki/Normal_distribution). **Syntax** @@ -108,7 +106,7 @@ randNormal(mean, variance) **Arguments** - `mean` - `Float64` - mean value of distribution, -- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance). +- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance) of the distribution. **Returned value** @@ -136,7 +134,7 @@ Result: ## randLogNormal -Returns a Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution). +Returns a random Float64 drawn from a [log-normal distribution](https://en.wikipedia.org/wiki/Log-normal_distribution). **Syntax** @@ -147,7 +145,7 @@ randLogNormal(mean, variance) **Arguments** - `mean` - `Float64` - mean value of distribution, -- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance). +- `variance` - `Float64` - [variance](https://en.wikipedia.org/wiki/Variance) of the distribution. **Returned value** @@ -175,7 +173,7 @@ Result: ## randBinomial -Returns a UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution). +Returns a random UInt64 drawn from a [binomial distribution](https://en.wikipedia.org/wiki/Binomial_distribution). **Syntax** @@ -186,7 +184,7 @@ randBinomial(experiments, probability) **Arguments** - `experiments` - `UInt64` - number of experiments, -- `probability` - `Float64` - probability of success in each experiment (values in `0...1` range only). +- `probability` - `Float64` - probability of success in each experiment, a value between 0 and 1. **Returned value** @@ -214,7 +212,7 @@ Result: ## randNegativeBinomial -Returns a UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution). +Returns a random UInt64 drawn from a [negative binomial distribution](https://en.wikipedia.org/wiki/Negative_binomial_distribution). **Syntax** @@ -225,7 +223,7 @@ randNegativeBinomial(experiments, probability) **Arguments** - `experiments` - `UInt64` - number of experiments, -- `probability` - `Float64` - probability of failure in each experiment (values in `0...1` range only). +- `probability` - `Float64` - probability of failure in each experiment, a value between 0 and 1. **Returned value** @@ -253,7 +251,7 @@ Result: ## randPoisson -Returns a UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution). +Returns a random UInt64 drawn from a [Poisson distribution](https://en.wikipedia.org/wiki/Poisson_distribution). **Syntax** @@ -291,7 +289,7 @@ Result: ## randBernoulli -Returns a UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution). +Returns a random UInt64 drawn from a [Bernoulli distribution](https://en.wikipedia.org/wiki/Bernoulli_distribution). **Syntax** @@ -301,7 +299,7 @@ randBernoulli(probability) **Arguments** -- `probability` - `Float64` - probability of success (values in `0...1` range only). +- `probability` - `Float64` - probability of success, a value between 0 and 1. **Returned value** @@ -329,7 +327,7 @@ Result: ## randExponential -Returns a Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution). +Returns a random Float64 drawn from a [exponential distribution](https://en.wikipedia.org/wiki/Exponential_distribution). **Syntax** @@ -367,7 +365,7 @@ Result: ## randChiSquared -Returns a Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables. +Returns a random Float64 drawn from a [Chi-square distribution](https://en.wikipedia.org/wiki/Chi-squared_distribution) - a distribution of a sum of the squares of k independent standard normal random variables. **Syntax** @@ -405,7 +403,7 @@ Result: ## randStudentT -Returns a Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution). +Returns a random Float64 drawn from a [Student's t-distribution](https://en.wikipedia.org/wiki/Student%27s_t-distribution). **Syntax** @@ -443,7 +441,7 @@ Result: ## randFisherF -Returns a Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution). +Returns a random Float64 drawn from a [F-distribution](https://en.wikipedia.org/wiki/F-distribution). **Syntax** @@ -480,47 +478,160 @@ Result: └─────────────────────┘ ``` -# Functions for Generating Random Strings - ## randomString -Returns a random String of specified `length`. Not all characters may be printable. +Generates a string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. **Syntax** -```sql +``` sql randomString(length) ``` +**Arguments** + +- `length` — String length in bytes. Positive integer. + +**Returned value** + +- String filled with random bytes. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + +``` sql +SELECT randomString(30) AS str, length(str) AS len FROM numbers(2) FORMAT Vertical; +``` + +Result: + +``` text +Row 1: +────── +str: 3 G : pT ?w тi k aV f6 +len: 30 + +Row 2: +────── +str: 9 ,] ^ ) ]?? 8 +len: 30 +``` + ## randomFixedString -Like `randomString` but returns a FixedString. - -## randomPrintableASCII - -Returns a random String of specified `length`. All characters are printable. +Generates a binary string of the specified length filled with random bytes (including zero bytes). Not all characters may be printable. **Syntax** +``` sql +randomFixedString(length); +``` + +**Arguments** + +- `length` — String length in bytes. [UInt64](../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** + +- String filled with random bytes. + +Type: [FixedString](../../sql-reference/data-types/fixedstring.md). + +**Example** + +Query: + ```sql +SELECT randomFixedString(13) as rnd, toTypeName(rnd) +``` + +Result: + +```text +┌─rnd──────┬─toTypeName(randomFixedString(13))─┐ +│ j▒h㋖HɨZ'▒ │ FixedString(13) │ +└──────────┴───────────────────────────────────┘ +``` + +## randomPrintableASCII + +Generates a string with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) characters. All characters are printable. +If you pass `length < 0`, the behavior of the function is undefined. + +**Syntax** + +``` sql randomPrintableASCII(length) ``` +**Arguments** + +- `length` — String length in bytes. Positive integer. + +**Returned value** + +- String with a random set of [ASCII](https://en.wikipedia.org/wiki/ASCII#Printable_characters) printable characters. + +Type: [String](../../sql-reference/data-types/string.md) + +**Example** + +``` sql +SELECT number, randomPrintableASCII(30) as str, length(str) FROM system.numbers LIMIT 3 +``` + +``` text +┌─number─┬─str────────────────────────────┬─length(randomPrintableASCII(30))─┐ +│ 0 │ SuiCOSTvC0csfABSw=UcSzp2.`rv8x │ 30 │ +│ 1 │ 1Ag NlJ &RCN:*>HVPG;PE-nO"SUFD │ 30 │ +│ 2 │ /"+<"wUTh:=LjJ Vm!c&hI*m#XTfzz │ 30 │ +└────────┴────────────────────────────────┴──────────────────────────────────┘ +``` + ## randomStringUTF8 -Returns a random String containing `length` many UTF8 codepoints. Not all characters may be printable +Generates a random string of a specified length. Result string contains valid UTF-8 code points. The value of code points may be outside of the range of assigned Unicode. **Syntax** +``` sql +randomStringUTF8(length); +``` + +**Arguments** + +- `length` — Length of the string in code points. [UInt64](../../sql-reference/data-types/int-uint.md). + +**Returned value(s)** + +- UTF-8 random string. + +Type: [String](../../sql-reference/data-types/string.md). + +**Example** + +Query: + ```sql -randomStringUTF8(length) +SELECT randomStringUTF8(13) +``` + +Result: + +```text +┌─randomStringUTF8(13)─┐ +│ 𘤗𙉝д兠庇󡅴󱱎󦐪􂕌𔊹𓰛 │ +└──────────────────────┘ ``` ## fuzzBits **Syntax** -Inverts the bits of String or FixedString `s`, each with probability `prob`. +Flips the bits of String or FixedString `s`, each with probability `prob`. **Syntax** @@ -529,8 +640,8 @@ fuzzBits(s, prob) ``` **Arguments** -- `s` - `String` or `FixedString` -- `prob` - constant `Float32/64` +- `s` - `String` or `FixedString`, +- `prob` - constant `Float32/64` between 0.0 and 1.0. **Returned value** From 0181ea63993bc6dd9c924c092bc7294a1bf71e2f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 12:55:46 +0300 Subject: [PATCH 16/26] Revert "make filter push down through cross join" --- .../Optimizations/filterPushDown.cpp | 6 +++--- .../01763_filter_push_down_bugs.reference | 19 ------------------- .../01763_filter_push_down_bugs.sql | 19 ------------------- 3 files changed, 3 insertions(+), 41 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index db29038999b..37bc894339f 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -272,7 +272,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { /// If totals step has HAVING expression, skip it for now. /// TODO: - /// We can merge HAVING expression with current filter. + /// We can merge HAVING expression with current filer. /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; @@ -323,9 +323,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes { const auto & table_join = join ? join->getJoin()->getTableJoin() : filled_join->getJoin()->getTableJoin(); - /// Only inner, cross and left(/right) join are supported. Other types may generate default values for left table keys. + /// Only inner and left(/right) join are supported. Other types may generate default values for left table keys. /// So, if we push down a condition like `key != 0`, not all rows may be filtered. - if (table_join.kind() != JoinKind::Inner && table_join.kind() != JoinKind::Cross && table_join.kind() != kind) + if (table_join.kind() != JoinKind::Inner && table_join.kind() != kind) return 0; bool is_left = kind == JoinKind::Left; diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference index 7df35e2948d..5aa2e645509 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.reference +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.reference @@ -6,22 +6,3 @@ String1_0 String2_0 String3_0 String4_0 1 String1_0 String2_0 String3_0 String4_0 1 1 [0,1,2] 1 -Expression ((Projection + Before ORDER BY)) - Filter (WHERE) - Join (JOIN FillRightFirst) - Filter (( + Before JOIN)) - ReadFromMergeTree (default.t1) - Indexes: - PrimaryKey - Keys: - id - Condition: (id in [101, 101]) - Parts: 1/1 - Granules: 1/1 - Expression ((Joined actions + (Rename joined columns + (Projection + Before ORDER BY)))) - ReadFromMergeTree (default.t2) - Indexes: - PrimaryKey - Condition: true - Parts: 1/1 - Granules: 1/1 diff --git a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql index 2ee249b5ce7..1058bf75144 100644 --- a/tests/queries/0_stateless/01763_filter_push_down_bugs.sql +++ b/tests/queries/0_stateless/01763_filter_push_down_bugs.sql @@ -38,25 +38,6 @@ DROP TABLE IF EXISTS Test; select x, y from (select [0, 1, 2] as y, 1 as a, 2 as b) array join y as x where a = 1 and b = 2 and (x = 1 or x != 1) and x = 1; -DROP TABLE IF EXISTS t; create table t(a UInt8) engine=MergeTree order by a; insert into t select * from numbers(2); select a from t t1 join t t2 on t1.a = t2.a where t1.a; -DROP TABLE IF EXISTS t; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; -CREATE TABLE t1 (id Int64, create_time DateTime) ENGINE = MergeTree ORDER BY id; -CREATE TABLE t2 (delete_time DateTime) ENGINE = MergeTree ORDER BY delete_time; - -insert into t1 values (101, '2023-05-28 00:00:00'), (102, '2023-05-28 00:00:00'); -insert into t2 values ('2023-05-31 00:00:00'); - -EXPLAIN indexes=1 SELECT id, delete_time FROM t1 - CROSS JOIN ( - SELECT delete_time - FROM t2 -) AS d WHERE create_time < delete_time AND id = 101; - -DROP TABLE IF EXISTS t1; -DROP TABLE IF EXISTS t2; From 8cadd89ebedc2fee73c3081992d35bbf8ad3280c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Fri, 2 Jun 2023 12:34:22 +0200 Subject: [PATCH 17/26] Update src/Common/TaskStatsInfoGetter.cpp --- src/Common/TaskStatsInfoGetter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index d21229609dd..867a50c8cce 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -208,7 +208,7 @@ bool checkPermissionsImpl() try { ::taskstats stats{}; - TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); + TaskStatsInfoGetter().getStat(stats, static_cast(getThreadId())); } catch (const Exception & e) { From 572f15b2cd70fa7b4293c7ec9682e361c9989d77 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 11:14:52 +0000 Subject: [PATCH 18/26] Fix typo --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index f4ca27e9b16..6f82fedaab7 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -245,7 +245,7 @@ assumeNotNull(x) **Returned values** - The input value as non-`Nullable` type, if it is not `NULL`. -- An arbirary value, if the input value is `NULL`. +- An arbitrary value, if the input value is `NULL`. **Example** From 73db383727550e040a939d367e6e59bb037780bd Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 2 Jun 2023 14:10:26 +0200 Subject: [PATCH 19/26] Limit the number of in-flight tasks for loading outdated parts (#50450) * Done * Update programs/local/LocalServer.cpp Co-authored-by: Alexander Tokmakov * Bump --------- Co-authored-by: Alexander Tokmakov --- programs/local/LocalServer.cpp | 2 +- programs/server/Server.cpp | 2 +- src/Core/ServerSettings.h | 1 - src/Interpreters/threadPoolCallbackRunner.h | 4 ++-- 4 files changed, 4 insertions(+), 5 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 944a60d4e4c..96c1ca261b5 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -138,7 +138,7 @@ void LocalServer::initialize(Poco::Util::Application & self) OutdatedPartsLoadingThreadPool::initialize( config().getUInt("max_outdated_parts_loading_thread_pool_size", 16), 0, // We don't need any threads one all the parts will be loaded - config().getUInt("outdated_part_loading_thread_pool_queue_size", 10000)); + config().getUInt("max_outdated_parts_loading_thread_pool_size", 16)); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 03ada89e86f..9eb3e6c9ebc 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -696,7 +696,7 @@ try OutdatedPartsLoadingThreadPool::initialize( server_settings.max_outdated_parts_loading_thread_pool_size, 0, // We don't need any threads one all the parts will be loaded - server_settings.outdated_part_loading_thread_pool_queue_size); + server_settings.max_outdated_parts_loading_thread_pool_size); /// Initialize global local cache for remote filesystem. if (config().has("local_cache_for_remote_fs")) diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 511b8d68f6d..cb43d62ecd1 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -22,7 +22,6 @@ namespace DB M(UInt64, max_io_thread_pool_free_size, 0, "Max free size for IO thread pool.", 0) \ M(UInt64, io_thread_pool_queue_size, 10000, "Queue size for IO thread pool.", 0) \ M(UInt64, max_outdated_parts_loading_thread_pool_size, 32, "The maximum number of threads that would be used for loading outdated data parts on startup", 0) \ - M(UInt64, outdated_part_loading_thread_pool_queue_size, 10000, "Queue size for parts loading thread pool.", 0) \ M(UInt64, max_replicated_fetches_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ M(UInt64, max_replicated_sends_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ M(UInt64, max_remote_read_network_bandwidth_for_server, 0, "The maximum speed of data exchange over the network in bytes per second for read. Zero means unlimited.", 0) \ diff --git a/src/Interpreters/threadPoolCallbackRunner.h b/src/Interpreters/threadPoolCallbackRunner.h index 55c6a848b77..f7324bfafe6 100644 --- a/src/Interpreters/threadPoolCallbackRunner.h +++ b/src/Interpreters/threadPoolCallbackRunner.h @@ -13,7 +13,7 @@ namespace DB template > using ThreadPoolCallbackRunner = std::function(Callback &&, Priority)>; -/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrow()'. +/// Creates CallbackRunner that runs every callback with 'pool->scheduleOrThrowOnError()'. template > ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & pool, const std::string & thread_name) { @@ -44,7 +44,7 @@ ThreadPoolCallbackRunner threadPoolCallbackRunner(ThreadPool & auto future = task->get_future(); - my_pool->scheduleOrThrow([my_task = std::move(task)]{ (*my_task)(); }, priority); + my_pool->scheduleOrThrowOnError([my_task = std::move(task)]{ (*my_task)(); }, priority); return future; }; From 65cc92a78d89f088c6c160dd4cb1748f48ed726d Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 11:30:05 +0000 Subject: [PATCH 20/26] CI: Fix aspell on nested docs --- .../database-engines/materialized-mysql.md | 2 +- .../materialized-postgresql.md | 4 +- docs/en/engines/database-engines/sqlite.md | 2 +- .../table-engines/integrations/hdfs.md | 4 +- .../table-engines/integrations/hive.md | 2 +- .../table-engines/integrations/nats.md | 4 +- .../table-engines/integrations/postgresql.md | 2 +- .../engines/table-engines/integrations/s3.md | 6 +- .../mergetree-family/annindexes.md | 4 +- .../custom-partitioning-key.md | 2 +- .../mergetree-family/mergetree.md | 4 +- .../table-engines/special/executable.md | 4 +- .../table-engines/special/keepermap.md | 2 +- .../example-datasets/amazon-reviews.md | 4 +- .../example-datasets/cell-towers.md | 4 +- .../example-datasets/github.md | 4 +- .../example-datasets/opensky.md | 2 +- .../example-datasets/reddit-comments.md | 4 +- .../example-datasets/youtube-dislikes.md | 6 +- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/system-tables/columns.md | 2 +- .../system-tables/dropped_tables.md | 2 +- .../system-tables/information_schema.md | 2 +- docs/en/operations/system-tables/licenses.md | 2 +- docs/en/operations/system-tables/parts.md | 2 +- docs/en/operations/system-tables/tables.md | 2 +- docs/en/operations/system-tables/trace_log.md | 2 +- .../operations/system-tables/zookeeper_log.md | 4 +- .../utilities/clickhouse-obfuscator.md | 2 +- .../aggregate-functions/reference/cramersv.md | 4 +- .../reference/cramersvbiascorrected.md | 2 +- .../reference/groupbitand.md | 2 +- .../reference/groupbitor.md | 2 +- .../reference/groupbitxor.md | 2 +- .../reference/kolmogorovsmirnovtest.md | 4 +- .../reference/quantiletdigestweighted.md | 2 +- .../reference/stochasticlinearregression.md | 2 +- docs/en/sql-reference/data-types/datetime.md | 8 +- docs/en/sql-reference/data-types/index.md | 2 +- docs/en/sql-reference/dictionaries/index.md | 2 +- .../sql-reference/functions/bit-functions.md | 4 +- .../functions/encryption-functions.md | 8 +- docs/en/sql-reference/functions/geo/h3.md | 2 +- docs/en/sql-reference/functions/geo/s2.md | 2 +- .../sql-reference/functions/hash-functions.md | 2 +- .../functions/logical-functions.md | 2 +- .../sql-reference/functions/math-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../functions/string-functions.md | 2 +- .../functions/string-replace-functions.md | 2 +- docs/en/sql-reference/functions/udf.md | 2 +- docs/en/sql-reference/operators/in.md | 2 +- .../sql-reference/statements/alter/quota.md | 2 +- .../sql-reference/statements/create/quota.md | 2 +- docs/en/sql-reference/statements/explain.md | 4 +- .../sql-reference/statements/select/from.md | 2 +- .../statements/select/order-by.md | 2 +- .../sql-reference/window-functions/index.md | 4 +- .../aspell-ignore/en/aspell-dict.txt | 2429 +++++++++++++++-- utils/check-style/check-doc-aspell | 3 + 60 files changed, 2300 insertions(+), 302 deletions(-) diff --git a/docs/en/engines/database-engines/materialized-mysql.md b/docs/en/engines/database-engines/materialized-mysql.md index 1f1c996d4bf..f7cc52e622e 100644 --- a/docs/en/engines/database-engines/materialized-mysql.md +++ b/docs/en/engines/database-engines/materialized-mysql.md @@ -119,7 +119,7 @@ When working with the `MaterializedMySQL` database engine, [ReplacingMergeTree]( The data of TIME type in MySQL is converted to microseconds in ClickHouse. -Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws exception "Unhandled data type" and stops replication. +Other types are not supported. If MySQL table contains a column of such type, ClickHouse throws an exception and stops replication. ## Specifics and Recommendations {#specifics-and-recommendations} diff --git a/docs/en/engines/database-engines/materialized-postgresql.md b/docs/en/engines/database-engines/materialized-postgresql.md index 08e9f998626..33d75dc9582 100644 --- a/docs/en/engines/database-engines/materialized-postgresql.md +++ b/docs/en/engines/database-engines/materialized-postgresql.md @@ -55,7 +55,7 @@ ATTACH TABLE postgres_database.new_table; ``` :::warning -Before version 22.1, adding a table to replication left an unremoved temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. +Before version 22.1, adding a table to replication left a non-removed temporary replication slot (named `{db_name}_ch_replication_slot_tmp`). If attaching tables in ClickHouse version before 22.1, make sure to delete it manually (`SELECT pg_drop_replication_slot('{db_name}_ch_replication_slot_tmp')`). Otherwise disk usage will grow. This issue is fixed in 22.1. ::: ## Dynamically removing tables from replication {#dynamically-removing-table-from-replication} @@ -257,7 +257,7 @@ Please note that this should be used only if it is actually needed. If there is 1. [CREATE PUBLICATION](https://postgrespro.ru/docs/postgresql/14/sql-createpublication) -- create query privilege. -2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privelege. +2. [CREATE_REPLICATION_SLOT](https://postgrespro.ru/docs/postgrespro/10/protocol-replication#PROTOCOL-REPLICATION-CREATE-SLOT) -- replication privilege. 3. [pg_drop_replication_slot](https://postgrespro.ru/docs/postgrespro/9.5/functions-admin#functions-replication) -- replication privilege or superuser. diff --git a/docs/en/engines/database-engines/sqlite.md b/docs/en/engines/database-engines/sqlite.md index fc2a6525a68..0fa3c0fff58 100644 --- a/docs/en/engines/database-engines/sqlite.md +++ b/docs/en/engines/database-engines/sqlite.md @@ -30,7 +30,7 @@ Allows to connect to [SQLite](https://www.sqlite.org/index.html) database and pe ## Specifics and Recommendations {#specifics-and-recommendations} -SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multitasked. +SQLite stores the entire database (definitions, tables, indices, and the data itself) as a single cross-platform file on a host machine. During writing SQLite locks the entire database file, therefore write operations are performed sequentially. Read operations can be multi-tasked. SQLite does not require service management (such as startup scripts) or access control based on `GRANT` and passwords. Access control is handled by means of file-system permissions given to the database file itself. ## Usage Example {#usage-example} diff --git a/docs/en/engines/table-engines/integrations/hdfs.md b/docs/en/engines/table-engines/integrations/hdfs.md index b9db0fae68f..08cd88826e5 100644 --- a/docs/en/engines/table-engines/integrations/hdfs.md +++ b/docs/en/engines/table-engines/integrations/hdfs.md @@ -156,7 +156,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | rpc\_client\_connect\_timeout | 600 * 1000 | | rpc\_client\_read\_timeout | 3600 * 1000 | | rpc\_client\_write\_timeout | 3600 * 1000 | -| rpc\_client\_socekt\_linger\_timeout | -1 | +| rpc\_client\_socket\_linger\_timeout | -1 | | rpc\_client\_connect\_retry | 10 | | rpc\_client\_timeout | 3600 * 1000 | | dfs\_default\_replica | 3 | @@ -176,7 +176,7 @@ Similar to GraphiteMergeTree, the HDFS engine supports extended configuration us | output\_write\_timeout | 3600 * 1000 | | output\_close\_timeout | 3600 * 1000 | | output\_packetpool\_size | 1024 | -| output\_heeartbeat\_interval | 10 * 1000 | +| output\_heartbeat\_interval | 10 * 1000 | | dfs\_client\_failover\_max\_attempts | 15 | | dfs\_client\_read\_shortcircuit\_streams\_cache\_size | 256 | | dfs\_client\_socketcache\_expiryMsec | 3000 | diff --git a/docs/en/engines/table-engines/integrations/hive.md b/docs/en/engines/table-engines/integrations/hive.md index adcb73605bb..5d10e417ae3 100644 --- a/docs/en/engines/table-engines/integrations/hive.md +++ b/docs/en/engines/table-engines/integrations/hive.md @@ -6,7 +6,7 @@ sidebar_label: Hive # Hive -The Hive engine allows you to perform `SELECT` quries on HDFS Hive table. Currently it supports input formats as below: +The Hive engine allows you to perform `SELECT` queries on HDFS Hive table. Currently it supports input formats as below: - Text: only supports simple scalar column types except `binary` diff --git a/docs/en/engines/table-engines/integrations/nats.md b/docs/en/engines/table-engines/integrations/nats.md index 7f09c516d6f..a82d74e0d95 100644 --- a/docs/en/engines/table-engines/integrations/nats.md +++ b/docs/en/engines/table-engines/integrations/nats.md @@ -10,7 +10,7 @@ This engine allows integrating ClickHouse with [NATS](https://nats.io/). `NATS` lets you: -- Publish or subcribe to message subjects. +- Publish or subscribe to message subjects. - Process new messages as they become available. ## Creating a Table {#table_engine-redisstreams-creating-a-table} @@ -46,7 +46,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Required parameters: - `nats_url` – host:port (for example, `localhost:5672`).. -- `nats_subjects` – List of subject for NATS table to subscribe/publsh to. Supports wildcard subjects like `foo.*.bar` or `baz.>` +- `nats_subjects` – List of subject for NATS table to subscribe/publish to. Supports wildcard subjects like `foo.*.bar` or `baz.>` - `nats_format` – Message format. Uses the same notation as the SQL `FORMAT` function, such as `JSONEachRow`. For more information, see the [Formats](../../../interfaces/formats.md) section. Optional parameters: diff --git a/docs/en/engines/table-engines/integrations/postgresql.md b/docs/en/engines/table-engines/integrations/postgresql.md index f27d4d48f75..51b3048706f 100644 --- a/docs/en/engines/table-engines/integrations/postgresql.md +++ b/docs/en/engines/table-engines/integrations/postgresql.md @@ -57,7 +57,7 @@ or via config (since version 21.11): ``` -Some parameters can be overriden by key value arguments: +Some parameters can be overridden by key value arguments: ``` sql SELECT * FROM postgresql(postgres1, schema='schema1', table='table1'); ``` diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 595bc0c344f..f2eaacd92a5 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -23,7 +23,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) - `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. - `format` — The [format](../../../interfaces/formats.md#formats) of the file. - `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). -- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will auto-detect compression by file extension. ### PARTITION BY @@ -140,8 +140,8 @@ The following settings can be set before query execution or placed into configur - `s3_max_get_rps` — Maximum GET requests per second rate before throttling. Default value is `0` (unlimited). - `s3_max_get_burst` — Max number of requests that can be issued simultaneously before hitting request per second limit. By default (`0` value) equals to `s3_max_get_rps`. - `s3_upload_part_size_multiply_factor` - Multiply `s3_min_upload_part_size` by this factor each time `s3_multiply_parts_count_threshold` parts were uploaded from a single write to S3. Default values is `2`. -- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. DEfault value us `500`. -- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each inflight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enought, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. +- `s3_upload_part_size_multiply_parts_count_threshold` - Each time this number of parts was uploaded to S3 `s3_min_upload_part_size multiplied` by `s3_upload_part_size_multiply_factor`. Default value us `500`. +- `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Its number should be limited. The value `0` means unlimited. Default value is `20`. Each in-flight part has a buffer with size `s3_min_upload_part_size` for the first `s3_upload_part_size_multiply_factor` parts and more when file is big enough, see `upload_part_size_multiply_factor`. With default settings one uploaded file consumes not more than `320Mb` for a file which is less than `8G`. The consumption is greater for a larger file. Security consideration: if malicious user can specify arbitrary S3 URLs, `s3_max_redirects` must be set to zero to avoid [SSRF](https://en.wikipedia.org/wiki/Server-side_request_forgery) attacks; or alternatively, `remote_host_filter` must be specified in server configuration. diff --git a/docs/en/engines/table-engines/mergetree-family/annindexes.md b/docs/en/engines/table-engines/mergetree-family/annindexes.md index f841f157376..03617a1a709 100644 --- a/docs/en/engines/table-engines/mergetree-family/annindexes.md +++ b/docs/en/engines/table-engines/mergetree-family/annindexes.md @@ -78,7 +78,7 @@ ENGINE = MergeTree ORDER BY id; ``` -With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyperparameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. +With greater `GRANULARITY` indexes remember the data structure better. The `GRANULARITY` indicates how many granules will be used to construct the index. The more data is provided for the index, the more of it can be handled by one index and the more chances that with the right hyper parameters the index will remember the data structure better. But some indexes can't be built if they don't have enough data, so this granule will always participate in the query. For more information, see the description of indexes. As the indexes are built only during insertions into table, `INSERT` and `OPTIMIZE` queries are slower than for ordinary table. At this stage indexes remember all the information about the given data. ANNIndexes should be used if you have immutable or rarely changed data and many read requests. @@ -135,7 +135,7 @@ ORDER BY id; Annoy supports `L2Distance` and `cosineDistance`. -In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time tradeoff between better accuracy and speed. +In the `SELECT` in the settings (`ann_index_select_query_params`) you can specify the size of the internal buffer (more details in the description above or in the [original repository](https://github.com/spotify/annoy)). During the query it will inspect up to `search_k` nodes which defaults to `n_trees * n` if not provided. `search_k` gives you a run-time trade-off between better accuracy and speed. __Example__: ``` sql diff --git a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md index edb320a2507..7e564b23676 100644 --- a/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/en/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -165,7 +165,7 @@ Performance of such a query heavily depends on the table layout. Because of that The key factors for a good performance: -- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will underutilize the machine +- number of partitions involved in the query should be sufficiently large (more than `max_threads / 2`), otherwise query will under-utilize the machine - partitions shouldn't be too small, so batch processing won't degenerate into row-by-row processing - partitions should be comparable in size, so all threads will do roughly the same amount of work diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 79ced0b6ce5..1ab0f4057ff 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -856,7 +856,7 @@ Tags: - `perform_ttl_move_on_insert` — Disables TTL move on data part INSERT. By default if we insert a data part that already expired by the TTL move rule it immediately goes to a volume/disk declared in move rule. This can significantly slowdown insert in case if destination volume/disk is slow (e.g. S3). - `load_balancing` - Policy for disk balancing, `round_robin` or `least_used`. -Cofiguration examples: +Configuration examples: ``` xml @@ -1224,7 +1224,7 @@ Limit parameters (mainly for internal usage): * `max_single_read_retries` - Limits the number of attempts to read a chunk of data from Blob Storage. * `max_single_download_retries` - Limits the number of attempts to download a readable buffer from Blob Storage. * `thread_pool_size` - Limits the number of threads with which `IDiskRemote` is instantiated. -* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurenly for one object. +* `s3_max_inflight_parts_for_one_file` - Limits the number of put requests that can be run concurrently for one object. Other parameters: * `metadata_path` - Path on local FS to store metadata files for Blob Storage. Default value is `/var/lib/clickhouse/disks//`. diff --git a/docs/en/engines/table-engines/special/executable.md b/docs/en/engines/table-engines/special/executable.md index 25049d7b46e..d976beeab6c 100644 --- a/docs/en/engines/table-engines/special/executable.md +++ b/docs/en/engines/table-engines/special/executable.md @@ -65,7 +65,7 @@ if __name__ == "__main__": main() ``` -The following `my_executable_table` is built from the output of `my_script.py`, which will generate 10 random strings everytime you run a `SELECT` from `my_executable_table`: +The following `my_executable_table` is built from the output of `my_script.py`, which will generate 10 random strings every time you run a `SELECT` from `my_executable_table`: ```sql CREATE TABLE my_executable_table ( @@ -223,4 +223,4 @@ SETTINGS pool_size = 4; ``` -ClickHouse will maintain 4 processes on-demand when your client queries the `sentiment_pooled` table. \ No newline at end of file +ClickHouse will maintain 4 processes on-demand when your client queries the `sentiment_pooled` table. diff --git a/docs/en/engines/table-engines/special/keepermap.md b/docs/en/engines/table-engines/special/keepermap.md index a1c7009b712..6ce56adbae9 100644 --- a/docs/en/engines/table-engines/special/keepermap.md +++ b/docs/en/engines/table-engines/special/keepermap.md @@ -72,7 +72,7 @@ Additionally, number of keys will have a soft limit of 4 for the number of keys. If multiple tables are created on the same ZooKeeper path, the values are persisted until there exists at least 1 table using it. As a result, it is possible to use `ON CLUSTER` clause when creating the table and sharing the data from multiple ClickHouse instances. -Of course, it's possible to manually run `CREATE TABLE` with same path on nonrelated ClickHouse instances to have same data sharing effect. +Of course, it's possible to manually run `CREATE TABLE` with same path on unrelated ClickHouse instances to have same data sharing effect. ## Supported operations {#table_engine-KeeperMap-supported-operations} diff --git a/docs/en/getting-started/example-datasets/amazon-reviews.md b/docs/en/getting-started/example-datasets/amazon-reviews.md index f35806aa66f..75e4549cb78 100644 --- a/docs/en/getting-started/example-datasets/amazon-reviews.md +++ b/docs/en/getting-started/example-datasets/amazon-reviews.md @@ -87,7 +87,7 @@ ORDER BY (marketplace, review_date, product_category); 3. We are now ready to insert the data into ClickHouse. Before we do, check out the [list of files in the dataset](https://s3.amazonaws.com/amazon-reviews-pds/tsv/index.txt) and decide which ones you want to include. -4. We will insert all of the US reviews - which is about 151M rows. The following `INSERT` command uses the `s3Cluster` table function, which allows the processing of mulitple S3 files in parallel using all the nodes of your cluster. We also use a wildcard to insert any file that starts with the name `https://s3.amazonaws.com/amazon-reviews-pds/tsv/amazon_reviews_us_`: +4. We will insert all of the US reviews - which is about 151M rows. The following `INSERT` command uses the `s3Cluster` table function, which allows the processing of multiple S3 files in parallel using all the nodes of your cluster. We also use a wildcard to insert any file that starts with the name `https://s3.amazonaws.com/amazon-reviews-pds/tsv/amazon_reviews_us_`: ```sql INSERT INTO amazon_reviews @@ -473,4 +473,4 @@ It runs quite a bit faster - which means the cache is helping us out here: └────────────┴───────────────────────────────────────────────────────────────────────┴────────────────────┴───────┘ 50 rows in set. Elapsed: 33.954 sec. Processed 150.96 million rows, 68.95 GB (4.45 million rows/s., 2.03 GB/s.) -``` \ No newline at end of file +``` diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index 048eecb285b..a84eb5d561f 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -317,7 +317,7 @@ To build a Superset dashboard using the OpenCelliD dataset you should: Make sure that you set **SSL** on when connecting to ClickHouse Cloud or other ClickHouse systems that enforce the use of SSL. ::: - ![Add ClickHouse as a Superset datasource](@site/docs/en/getting-started/example-datasets/images/superset-connect-a-database.png) + ![Add ClickHouse as a Superset data source](@site/docs/en/getting-started/example-datasets/images/superset-connect-a-database.png) ### Add the table **cell_towers** as a Superset **dataset** @@ -364,5 +364,5 @@ The data is also available for interactive queries in the [Playground](https://p This [example](https://play.clickhouse.com/play?user=play#U0VMRUNUIG1jYywgY291bnQoKSBGUk9NIGNlbGxfdG93ZXJzIEdST1VQIEJZIG1jYyBPUkRFUiBCWSBjb3VudCgpIERFU0M=) will populate the username and even the query for you. -Although you cannot create tables in the Playground, you can run all of the queries and even use Superset (adjust the hostname and port number). +Although you cannot create tables in the Playground, you can run all of the queries and even use Superset (adjust the host name and port number). ::: diff --git a/docs/en/getting-started/example-datasets/github.md b/docs/en/getting-started/example-datasets/github.md index 02965ed5e33..9ed8782e512 100644 --- a/docs/en/getting-started/example-datasets/github.md +++ b/docs/en/getting-started/example-datasets/github.md @@ -806,7 +806,7 @@ FROM 31 rows in set. Elapsed: 0.043 sec. Processed 7.54 million rows, 40.53 MB (176.71 million rows/s., 950.40 MB/s.) ``` -Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unrealiable due to the filtering of the docs filter during data insertion. +Maybe a little more near the end of the month, but overall we keep a good even distribution. Again this is unreliable due to the filtering of the docs filter during data insertion. ## Authors with the most diverse impact @@ -940,7 +940,7 @@ LIMIT 10 10 rows in set. Elapsed: 0.106 sec. Processed 798.15 thousand rows, 13.97 MB (7.51 million rows/s., 131.41 MB/s.) ``` -This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the basename of the file to identify his popular files - this allows for renames and should focus on code contributions. +This makes sense because Alexey has been responsible for maintaining the Change log. But what if we use the base name of the file to identify his popular files - this allows for renames and should focus on code contributions. [play](https://play.clickhouse.com/play?user=play#U0VMRUNUCiAgICBiYXNlLAogICAgY291bnQoKSBBUyBjCkZST00gZ2l0X2NsaWNraG91c2UuZmlsZV9jaGFuZ2VzCldIRVJFIChhdXRob3IgPSAnQWxleGV5IE1pbG92aWRvdicpIEFORCAoZmlsZV9leHRlbnNpb24gSU4gKCdoJywgJ2NwcCcsICdzcWwnKSkKR1JPVVAgQlkgYmFzZW5hbWUocGF0aCkgQVMgYmFzZQpPUkRFUiBCWSBjIERFU0MKTElNSVQgMTA=) diff --git a/docs/en/getting-started/example-datasets/opensky.md b/docs/en/getting-started/example-datasets/opensky.md index 7093a2df04f..df28809495c 100644 --- a/docs/en/getting-started/example-datasets/opensky.md +++ b/docs/en/getting-started/example-datasets/opensky.md @@ -9,7 +9,7 @@ The data in this dataset is derived and cleaned from the full OpenSky dataset to Source: https://zenodo.org/record/5092942#.YRBCyTpRXYd -Martin Strohmeier, Xavier Olive, Jannis Lübbe, Matthias Schäfer, and Vincent Lenders +Martin Strohmeier, Xavier Olive, Jannis Luebbe, Matthias Schaefer, and Vincent Lenders "Crowdsourced air traffic data from the OpenSky Network 2019–2020" Earth System Science Data 13(2), 2021 https://doi.org/10.5194/essd-13-357-2021 diff --git a/docs/en/getting-started/example-datasets/reddit-comments.md b/docs/en/getting-started/example-datasets/reddit-comments.md index e1e372746c9..49c7bd25f9f 100644 --- a/docs/en/getting-started/example-datasets/reddit-comments.md +++ b/docs/en/getting-started/example-datasets/reddit-comments.md @@ -469,7 +469,7 @@ The response is: 10 rows in set. Elapsed: 27.824 sec. Processed 6.74 billion rows, 53.26 GB (242.22 million rows/s., 1.91 GB/s.) ``` -11. Let's see which subreddits had the biggest increase in commnents from 2018 to 2019: +11. Let's see which subreddits had the biggest increase in comments from 2018 to 2019: ```sql SELECT @@ -633,4 +633,4 @@ ORDER BY quarter ASC; └────────────┴────────────┴───────────┴──────────┘ 58 rows in set. Elapsed: 2663.751 sec. Processed 6.74 billion rows, 1.21 TB (2.53 million rows/s., 454.37 MB/s.) -``` \ No newline at end of file +``` diff --git a/docs/en/getting-started/example-datasets/youtube-dislikes.md b/docs/en/getting-started/example-datasets/youtube-dislikes.md index 5f4ef696b8b..e24c6e5a6dc 100644 --- a/docs/en/getting-started/example-datasets/youtube-dislikes.md +++ b/docs/en/getting-started/example-datasets/youtube-dislikes.md @@ -22,7 +22,7 @@ The steps below will easily work on a local install of ClickHouse too. The only ## Step-by-step instructions -1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the reult: +1. Let's see what the data looks like. The `s3cluster` table function returns a table, so we can `DESCRIBE` the result: ```sql DESCRIBE s3Cluster( @@ -322,7 +322,7 @@ ORDER BY month ASC; A spike of uploaders [around covid is noticeable](https://www.theverge.com/2020/3/27/21197642/youtube-with-me-style-videos-views-coronavirus-cook-workout-study-home-beauty). -### More subtitiles over time and when +### More subtitles over time and when With advances in speech recognition, it’s easier than ever to create subtitles for video with youtube adding auto-captioning in late 2009 - was the jump then? @@ -484,4 +484,4 @@ ARRAY JOIN │ 20th │ 16 │ │ 10th │ 6 │ └────────────┴─────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 4f3b4e43358..ad113c58bce 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3492,7 +3492,7 @@ Default value: `0`. ## database_replicated_initial_query_timeout_sec {#database_replicated_initial_query_timeout_sec} -Sets how long initial DDL query should wait for Replicated database to precess previous DDL queue entries in seconds. +Sets how long initial DDL query should wait for Replicated database to process previous DDL queue entries in seconds. Possible values: diff --git a/docs/en/operations/system-tables/columns.md b/docs/en/operations/system-tables/columns.md index ccdc2d8c742..2915b053458 100644 --- a/docs/en/operations/system-tables/columns.md +++ b/docs/en/operations/system-tables/columns.md @@ -28,7 +28,7 @@ The `system.columns` table contains the following columns (the column type is sh - `is_in_sampling_key` ([UInt8](../../sql-reference/data-types/int-uint.md)) — Flag that indicates whether the column is in the sampling key expression. - `compression_codec` ([String](../../sql-reference/data-types/string.md)) — Compression codec name. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/dropped_tables.md b/docs/en/operations/system-tables/dropped_tables.md index 144c03109ac..e2a09094c87 100644 --- a/docs/en/operations/system-tables/dropped_tables.md +++ b/docs/en/operations/system-tables/dropped_tables.md @@ -12,7 +12,7 @@ Columns: - `table` ([String](../../sql-reference/data-types/string.md)) — Table name. - `uuid` ([UUID](../../sql-reference/data-types/uuid.md)) — Table uuid. - `engine` ([String](../../sql-reference/data-types/string.md)) — Table engine name. -- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadate_dropped directory. +- `metadata_dropped_path` ([String](../../sql-reference/data-types/string.md)) — Path of table's metadata file in metadata_dropped directory. - `table_dropped_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — The time when the next attempt to remove table's data is scheduled on. Usually it's the table when the table was dropped plus `database_atomic_delay_before_drop_table_sec` **Example** diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 07e9a9e2f58..35fd3a753b5 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -43,7 +43,7 @@ Columns: - `data_type` ([String](../../sql-reference/data-types/string.md)) — Column type. - `character_maximum_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. - `character_octet_length` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Maximum length in bytes for binary data, character data, or text data and images. In ClickHouse makes sense only for `FixedString` data type. Otherwise, the `NULL` value is returned. -- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bitness for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. +- `numeric_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Accuracy of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse it is bit width for integer types and decimal precision for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_precision_radix` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The base of the number system is the accuracy of approximate numeric data, exact numeric data, integer data or monetary data. In ClickHouse it's 2 for integer types and 10 for `Decimal` types. Otherwise, the `NULL` value is returned. - `numeric_scale` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — The scale of approximate numeric data, exact numeric data, integer data, or monetary data. In ClickHouse makes sense only for `Decimal` types. Otherwise, the `NULL` value is returned. - `datetime_precision` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) — Decimal precision of `DateTime64` data type. For other data types, the `NULL` value is returned. diff --git a/docs/en/operations/system-tables/licenses.md b/docs/en/operations/system-tables/licenses.md index 9296e78c797..0f09d559d8b 100644 --- a/docs/en/operations/system-tables/licenses.md +++ b/docs/en/operations/system-tables/licenses.md @@ -3,7 +3,7 @@ slug: /en/operations/system-tables/licenses --- # licenses -Сontains licenses of third-party libraries that are located in the [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) directory of ClickHouse sources. +Contains licenses of third-party libraries that are located in the [contrib](https://github.com/ClickHouse/ClickHouse/tree/master/contrib) directory of ClickHouse sources. Columns: diff --git a/docs/en/operations/system-tables/parts.md b/docs/en/operations/system-tables/parts.md index e61c6ed2ba4..9159d1e9284 100644 --- a/docs/en/operations/system-tables/parts.md +++ b/docs/en/operations/system-tables/parts.md @@ -100,7 +100,7 @@ Columns: - `move_ttl_info.expression` ([Array](../../sql-reference/data-types/array.md)([String](../../sql-reference/data-types/string.md))) — Array of expressions. Each expression defines a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). :::note -The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simpliest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. +The `move_ttl_info.expression` array is kept mostly for backward compatibility, now the simplest way to check `TTL MOVE` rule is to use the `move_ttl_info.min` and `move_ttl_info.max` fields. ::: - `move_ttl_info.min` ([Array](../../sql-reference/data-types/array.md)([DateTime](../../sql-reference/data-types/datetime.md))) — Array of date and time values. Each element describes the minimum key value for a [TTL MOVE rule](../../engines/table-engines/mergetree-family/mergetree.md/#table_engine-mergetree-ttl). diff --git a/docs/en/operations/system-tables/tables.md b/docs/en/operations/system-tables/tables.md index 82e9fa206ea..e4461e14236 100644 --- a/docs/en/operations/system-tables/tables.md +++ b/docs/en/operations/system-tables/tables.md @@ -50,7 +50,7 @@ Columns: - [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md#table_engine-mergetree-multiple-volumes) - [Distributed](../../engines/table-engines/special/distributed.md#distributed) -- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underying `Buffer` table). +- `total_rows` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of rows, if it is possible to quickly determine exact number of rows in the table, otherwise `NULL` (including underlying `Buffer` table). - `total_bytes` ([Nullable](../../sql-reference/data-types/nullable.md)([UInt64](../../sql-reference/data-types/int-uint.md))) - Total number of bytes, if it is possible to quickly determine exact number of bytes for the table on storage, otherwise `NULL` (does not includes any underlying storage). diff --git a/docs/en/operations/system-tables/trace_log.md b/docs/en/operations/system-tables/trace_log.md index a5aae422be7..89d54adc30d 100644 --- a/docs/en/operations/system-tables/trace_log.md +++ b/docs/en/operations/system-tables/trace_log.md @@ -43,7 +43,7 @@ Columns: - `event` ([LowCardinality(String)](../../sql-reference/data-types/lowcardinality.md)) - For trace type `ProfileEvent` is the name of updated profile event, for other trace types is an empty string. -- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of incremnt of profile event, for other trace types is 0. +- `increment` ([UInt64](../../sql-reference/data-types/int-uint.md)) - For trace type `ProfileEvent` is the amount of increment of profile event, for other trace types is 0. **Example** diff --git a/docs/en/operations/system-tables/zookeeper_log.md b/docs/en/operations/system-tables/zookeeper_log.md index 970ed192a48..b7cc4e22cd6 100644 --- a/docs/en/operations/system-tables/zookeeper_log.md +++ b/docs/en/operations/system-tables/zookeeper_log.md @@ -33,7 +33,7 @@ Columns with request response parameters: - `zxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — ZooKeeper transaction ID. The serial number issued by the ZooKeeper server in response to a successfully executed request (`0` if the request was not executed/returned an error/the client does not know whether the request was executed). - `error` ([Nullable(Enum)](../../sql-reference/data-types/nullable.md)) — Error code. Can have many values, here are just some of them: - - `ZOK` — The request was executed seccessfully. + - `ZOK` — The request was executed successfully. - `ZCONNECTIONLOSS` — The connection was lost. - `ZOPERATIONTIMEOUT` — The request execution timeout has expired. - `ZSESSIONEXPIRED` — The session has expired. @@ -43,7 +43,7 @@ Columns with request response parameters: - `path_created` ([String](../../sql-reference/data-types/string.md)) — The path to the created ZooKeeper node (for responses to the `CREATE` request), may differ from the `path` if the node is created as a `sequential`. - `stat_czxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that caused this ZooKeeper node to be created. - `stat_mzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The `zxid` of the change that last modified this ZooKeeper node. -- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified childern of this ZooKeeper node. +- `stat_pzxid` ([Int64](../../sql-reference/data-types/int-uint.md)) — The transaction ID of the change that last modified children of this ZooKeeper node. - `stat_version` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the data of this ZooKeeper node. - `stat_cversion` ([Int32](../../sql-reference/data-types/int-uint.md)) — The number of changes to the children of this ZooKeeper node. - `stat_dataLength` ([Int32](../../sql-reference/data-types/int-uint.md)) — The length of the data field of this ZooKeeper node. diff --git a/docs/en/operations/utilities/clickhouse-obfuscator.md b/docs/en/operations/utilities/clickhouse-obfuscator.md index 077489ba76e..ad51e9c7776 100644 --- a/docs/en/operations/utilities/clickhouse-obfuscator.md +++ b/docs/en/operations/utilities/clickhouse-obfuscator.md @@ -24,7 +24,7 @@ It is designed to retain the following properties of data: Most of the properties above are viable for performance testing: -reading data, filtering, aggregatio, and sorting will work at almost the same speed +reading data, filtering, aggregation, and sorting will work at almost the same speed as on original data due to saved cardinalities, magnitudes, compression ratios, etc. It works in a deterministic fashion: you define a seed value and the transformation is determined by input data and by seed. diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md index f412724ea08..e9e2c367610 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersv.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersv.md @@ -5,7 +5,7 @@ sidebar_position: 351 # cramersV -[Cramér's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramér's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. +[Cramer's V](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V) (sometimes referred to as Cramer's phi) is a measure of association between two columns in a table. The result of the `cramersV` function ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. It may be viewed as the association between two variables as a percentage of their maximum possible variation. **Syntax** @@ -69,4 +69,4 @@ Result: ┌─────cramersV(a, b)─┐ │ 0.8944271909999159 │ └────────────────────┘ -``` \ No newline at end of file +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md index 8e577efbc4d..f5ad3a8a937 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md +++ b/docs/en/sql-reference/aggregate-functions/reference/cramersvbiascorrected.md @@ -6,7 +6,7 @@ sidebar_position: 352 # cramersVBiasCorrected -Cramér's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramér's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). +Cramer's V is a measure of association between two columns in a table. The result of the [`cramersV` function](./cramersv.md) ranges from 0 (corresponding to no association between the variables) to 1 and can reach 1 only when each value is completely determined by the other. The function can be heavily biased, so this version of Cramer's V uses the [bias correction](https://en.wikipedia.org/wiki/Cram%C3%A9r%27s_V#Bias_correction). diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md index 5f57407a419..3d833555a43 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md @@ -5,7 +5,7 @@ sidebar_position: 125 # groupBitAnd -Applies bitwise `AND` for series of numbers. +Applies bit-wise `AND` for series of numbers. ``` sql groupBitAnd(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md index 59be69540b0..138ee998405 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md @@ -5,7 +5,7 @@ sidebar_position: 126 # groupBitOr -Applies bitwise `OR` for series of numbers. +Applies bit-wise `OR` for series of numbers. ``` sql groupBitOr(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md b/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md index b00876a2fdf..168335a010c 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md +++ b/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md @@ -5,7 +5,7 @@ sidebar_position: 127 # groupBitXor -Applies bitwise `XOR` for series of numbers. +Applies bit-wise `XOR` for series of numbers. ``` sql groupBitXor(expr) diff --git a/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md b/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md index 3da9645181e..d159eec7ce6 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md +++ b/docs/en/sql-reference/aggregate-functions/reference/kolmogorovsmirnovtest.md @@ -30,11 +30,11 @@ Samples must belong to continuous, one-dimensional probability distributions. The null hypothesis is that samples come from the same distribution, e.g. F(x) = G(x) for all x. And the alternative is that the distributions are not identical. - `'greater'` - The null hypothesis is that values in the first sample are *stohastically smaller* than those in the second one, + The null hypothesis is that values in the first sample are *stochastically smaller* than those in the second one, e.g. the CDF of first distribution lies above and hence to the left of that for the second one. Which in fact means that F(x) >= G(x) for all x. And the alternative in this case is that F(x) < G(x) for at least one x. - `'less'`. - The null hypothesis is that values in the first sample are *stohastically greater* than those in the second one, + The null hypothesis is that values in the first sample are *stochastically greater* than those in the second one, e.g. the CDF of first distribution lies below and hence to the right of that for the second one. Which in fact means that F(x) <= G(x) for all x. And the alternative in this case is that F(x) > G(x) for at least one x. - `computation_method` — the method used to compute p-value. (Optional, default: `'auto'`.) [String](../../../sql-reference/data-types/string.md). diff --git a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md index 7b9addbbdde..b3e21e0e69e 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md +++ b/docs/en/sql-reference/aggregate-functions/reference/quantiletdigestweighted.md @@ -14,7 +14,7 @@ The result depends on the order of running the query, and is nondeterministic. When using multiple `quantile*` functions with different levels in a query, the internal states are not combined (that is, the query works less efficiently than it could). In this case, use the [quantiles](../../../sql-reference/aggregate-functions/reference/quantiles.md#quantiles) function. :::note -Using `quantileTDigestWeighted` [is not recommended for tiny data sets](https://github.com/tdunning/t-digest/issues/167#issuecomment-828650275) and can lead to significat error. In this case, consider possibility of using [`quantileTDigest`](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) instead. +Using `quantileTDigestWeighted` [is not recommended for tiny data sets](https://github.com/tdunning/t-digest/issues/167#issuecomment-828650275) and can lead to significant error. In this case, consider possibility of using [`quantileTDigest`](../../../sql-reference/aggregate-functions/reference/quantiletdigest.md) instead. ::: **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md index 9481172c25b..f7615d90790 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md +++ b/docs/en/sql-reference/aggregate-functions/reference/stochasticlinearregression.md @@ -18,7 +18,7 @@ stochasticLinearRegression(1.0, 1.0, 10, 'SGD') 1. `learning rate` is the coefficient on step length, when gradient descent step is performed. Too big learning rate may cause infinite weights of the model. Default is `0.00001`. 2. `l2 regularization coefficient` which may help to prevent overfitting. Default is `0.1`. 3. `mini-batch size` sets the number of elements, which gradients will be computed and summed to perform one step of gradient descent. Pure stochastic descent uses one element, however having small batches(about 10 elements) make gradient steps more stable. Default is `15`. -4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergance and stability of stochastic gradient methods. +4. `method for updating weights`, they are: `Adam` (by default), `SGD`, `Momentum`, `Nesterov`. `Momentum` and `Nesterov` require little bit more computations and memory, however they happen to be useful in terms of speed of convergence and stability of stochastic gradient methods. ### Usage diff --git a/docs/en/sql-reference/data-types/datetime.md b/docs/en/sql-reference/data-types/datetime.md index 059c6acdb9e..0da273e01ad 100644 --- a/docs/en/sql-reference/data-types/datetime.md +++ b/docs/en/sql-reference/data-types/datetime.md @@ -22,7 +22,7 @@ Resolution: 1 second. The point in time is saved as a [Unix timestamp](https://en.wikipedia.org/wiki/Unix_time), regardless of the time zone or daylight saving time. The time zone affects how the values of the `DateTime` type values are displayed in text format and how the values specified as strings are parsed (‘2020-01-01 05:00:01’). -Timezone agnostic unix timestamp is stored in tables, and the timezone is used to transform it to text format or back during data import/export or to make calendar calculations on the values (example: `toDate`, `toHour` functions et cetera). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. +Timezone agnostic Unix timestamp is stored in tables, and the timezone is used to transform it to text format or back during data import/export or to make calendar calculations on the values (example: `toDate`, `toHour` functions etc.). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. A list of supported time zones can be found in the [IANA Time Zone Database](https://www.iana.org/time-zones) and also can be queried by `SELECT * FROM system.time_zones`. [The list](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones) is also available at Wikipedia. @@ -30,7 +30,7 @@ You can explicitly set a time zone for `DateTime`-type columns when creating a t The [clickhouse-client](../../interfaces/cli.md) applies the server time zone by default if a time zone isn’t explicitly set when initializing the data type. To use the client time zone, run `clickhouse-client` with the `--use_client_time_zone` parameter. -ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionaly you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. +ClickHouse outputs values depending on the value of the [date_time_output_format](../../operations/settings/settings.md#settings-date_time_output_format) setting. `YYYY-MM-DD hh:mm:ss` text format by default. Additionally, you can change the output with the [formatDateTime](../../sql-reference/functions/date-time-functions.md#formatdatetime) function. When inserting data into ClickHouse, you can use different formats of date and time strings, depending on the value of the [date_time_input_format](../../operations/settings/settings.md#settings-date_time_input_format) setting. @@ -120,9 +120,9 @@ FROM dt As timezone conversion only changes the metadata, the operation has no computation cost. -## Limitations on timezones support +## Limitations on time zones support -Some timezones may not be supported completely. There are a few cases: +Some time zones may not be supported completely. There are a few cases: If the offset from UTC is not a multiple of 15 minutes, the calculation of hours and minutes can be incorrect. For example, the time zone in Monrovia, Liberia has offset UTC -0:44:30 before 7 Jan 1972. If you are doing calculations on the historical time in Monrovia timezone, the time processing functions may give incorrect results. The results after 7 Jan 1972 will be correct nevertheless. diff --git a/docs/en/sql-reference/data-types/index.md b/docs/en/sql-reference/data-types/index.md index 88663968e50..508307a0543 100644 --- a/docs/en/sql-reference/data-types/index.md +++ b/docs/en/sql-reference/data-types/index.md @@ -27,7 +27,7 @@ ClickHouse data types include: - **Aggregation function types**: use [`SimpleAggregateFunction`](./simpleaggregatefunction.md) and [`AggregateFunction`](./aggregatefunction.md) for storing the intermediate status of aggregate function results - **Nested data structures**: A [`Nested` data structure](./nested-data-structures/index.md) is like a table inside a cell - **Tuples**: A [`Tuple` of elements](./tuple.md), each having an individual type. -- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column gettings its default value for the data type) +- **Nullable**: [`Nullable`](./nullable.md) allows you to store a value as `NULL` when a value is "missing" (instead of the column settings its default value for the data type) - **IP addresses**: use [`IPv4`](./domains/ipv4.md) and [`IPv6`](./domains/ipv6.md) to efficiently store IP addresses - **Geo types**: for [geographical data](./geo.md), including `Point`, `Ring`, `Polygon` and `MultiPolygon` - **Special data types**: including [`Expression`](./special-data-types/expression.md), [`Set`](./special-data-types/set.md), [`Nothing`](./special-data-types/nothing.md) and [`Interval`](./special-data-types/interval.md) diff --git a/docs/en/sql-reference/dictionaries/index.md b/docs/en/sql-reference/dictionaries/index.md index f7b4be64851..3a968992c13 100644 --- a/docs/en/sql-reference/dictionaries/index.md +++ b/docs/en/sql-reference/dictionaries/index.md @@ -984,7 +984,7 @@ SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source wher ... ``` -For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronious and asynchronous updates are supported. +For `Cache`, `ComplexKeyCache`, `SSDCache`, and `SSDComplexKeyCache` dictionaries both synchronous and asynchronous updates are supported. It is also possible for `Flat`, `Hashed`, `ComplexKeyHashed` dictionaries to only request data that was changed after the previous update. If `update_field` is specified as part of the dictionary source configuration, value of the previous update time in seconds will be added to the data request. Depends on source type (Executable, HTTP, MySQL, PostgreSQL, ClickHouse, or ODBC) different logic will be applied to `update_field` before request data from an external source. diff --git a/docs/en/sql-reference/functions/bit-functions.md b/docs/en/sql-reference/functions/bit-functions.md index 5b342fe4f24..3c07fe8bcbe 100644 --- a/docs/en/sql-reference/functions/bit-functions.md +++ b/docs/en/sql-reference/functions/bit-functions.md @@ -226,7 +226,7 @@ Result: Returns result of [logical conjuction](https://en.wikipedia.org/wiki/Logical_conjunction) (AND operator) of all bits at given positions. The countdown starts from 0 from the right to the left. -The conjuction for bitwise operations: +The conjuction for bit-wise operations: 0 AND 0 = 0 @@ -291,7 +291,7 @@ Result: Returns result of [logical disjunction](https://en.wikipedia.org/wiki/Logical_disjunction) (OR operator) of all bits at given positions. The countdown starts from 0 from the right to the left. -The disjunction for bitwise operations: +The disjunction for bit-wise operations: 0 OR 0 = 0 diff --git a/docs/en/sql-reference/functions/encryption-functions.md b/docs/en/sql-reference/functions/encryption-functions.md index 1224b7bc92b..b11bee83582 100644 --- a/docs/en/sql-reference/functions/encryption-functions.md +++ b/docs/en/sql-reference/functions/encryption-functions.md @@ -31,9 +31,9 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad]) **Arguments** - `mode` — Encryption mode. [String](../../sql-reference/data-types/string.md#string). -- `plaintext` — Text thats need to be encrypted. [String](../../sql-reference/data-types/string.md#string). +- `plaintext` — Text that need to be encrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Encryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Required for `-gcm` modes, optinal for others. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Required for `-gcm` modes, optional for others. [String](../../sql-reference/data-types/string.md#string). - `aad` — Additional authenticated data. It isn't encrypted, but it affects decryption. Works only in `-gcm` modes, for others would throw an exception. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -233,7 +233,7 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad]) - `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string). - `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Decryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Required for `-gcm` modes, optinal for others. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Required for `-gcm` modes, Optional for others. [String](../../sql-reference/data-types/string.md#string). - `aad` — Additional authenticated data. Won't decrypt if this value is incorrect. Works only in `-gcm` modes, for others would throw an exception. [String](../../sql-reference/data-types/string.md#string). **Returned value** @@ -364,7 +364,7 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) - `mode` — Decryption mode. [String](../../sql-reference/data-types/string.md#string). - `ciphertext` — Encrypted text that needs to be decrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Decryption key. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Optinal. [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Optional. [String](../../sql-reference/data-types/string.md#string). **Returned value** diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md index 1f695a13598..29486c58e6a 100644 --- a/docs/en/sql-reference/functions/geo/h3.md +++ b/docs/en/sql-reference/functions/geo/h3.md @@ -12,7 +12,7 @@ A latitude and longitude pair can be transformed to a 64-bit H3 index, identifyi The H3 index is used primarily for bucketing locations and other geospatial manipulations. -The full description of the H3 system is available at [the Uber Engeneering site](https://eng.uber.com/h3/). +The full description of the H3 system is available at [the Uber Engineering site](https://eng.uber.com/h3/). ## h3IsValid diff --git a/docs/en/sql-reference/functions/geo/s2.md b/docs/en/sql-reference/functions/geo/s2.md index 63fe5ca8530..f4702eff44b 100644 --- a/docs/en/sql-reference/functions/geo/s2.md +++ b/docs/en/sql-reference/functions/geo/s2.md @@ -249,7 +249,7 @@ s2RectAdd(s2pointLow, s2pointHigh, s2Point) **Returned values** - `s2PointLow` — Low S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/int-uint.md). -- `s2PointHigh` — Hight S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). +- `s2PointHigh` — Height S2 cell id corresponding to the grown rectangle. Type: [UInt64](../../../sql-reference/data-types/float.md). **Example** diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md index 89afcca3799..2cf3408534f 100644 --- a/docs/en/sql-reference/functions/hash-functions.md +++ b/docs/en/sql-reference/functions/hash-functions.md @@ -1161,7 +1161,7 @@ wordShingleSimHashUTF8(string[, shinglesize]) **Arguments** - `string` — String. [String](/docs/en/sql-reference/data-types/string.md). -- `shinglesize` — The size of a word shingle. Optinal. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](/docs/en/sql-reference/data-types/int-uint.md). +- `shinglesize` — The size of a word shingle. Optional. Possible values: any number from `1` to `25`. Default value: `3`. [UInt8](/docs/en/sql-reference/data-types/int-uint.md). **Returned value** diff --git a/docs/en/sql-reference/functions/logical-functions.md b/docs/en/sql-reference/functions/logical-functions.md index f5a1a6aac12..17a9fbb19fe 100644 --- a/docs/en/sql-reference/functions/logical-functions.md +++ b/docs/en/sql-reference/functions/logical-functions.md @@ -31,7 +31,7 @@ Alias: The [AND Operator](../../sql-reference/operators/index.md#logical-and-ope **Returned value** - `0`, if there at least one argument evaluates to `false`, -- `NULL`, if no argumetn evaluates to `false` and at least one argument is `NULL`, +- `NULL`, if no argument evaluates to `false` and at least one argument is `NULL`, - `1`, otherwise. Type: [UInt8](../../sql-reference/data-types/int-uint.md) or [Nullable](../../sql-reference/data-types/nullable.md)([UInt8](../../sql-reference/data-types/int-uint.md)). diff --git a/docs/en/sql-reference/functions/math-functions.md b/docs/en/sql-reference/functions/math-functions.md index 9851378d4fd..22492f2830b 100644 --- a/docs/en/sql-reference/functions/math-functions.md +++ b/docs/en/sql-reference/functions/math-functions.md @@ -52,7 +52,7 @@ Alias: `ln(x)` ## exp2 -Returns 2 to the power of the given argumetn +Returns 2 to the power of the given argument **Syntax** diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index efe1a77c285..41ccfe121a4 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -206,7 +206,7 @@ Type: [UInt64](../../sql-reference/data-types/int-uint.md). **Examples** -For [String](../../sql-reference/data-types/string.md) arguments the funtion returns the string length + 9 (terminating zero + length). +For [String](../../sql-reference/data-types/string.md) arguments the function returns the string length + 9 (terminating zero + length). Query: @@ -1352,7 +1352,7 @@ ORDER BY k ASC ClickHouse used the index in the same way as the previous time (`Processed 32.74 thousand rows`). The expression `k = '2017-09-15'` was not used when generating the result. -In examle the `indexHint` function allows to see adjacent dates. +In example the `indexHint` function allows to see adjacent dates. Result: diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 8662d08431c..5175bbf0615 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -393,7 +393,7 @@ Reverses a sequence of Unicode code points in a string. Assumes that the string ## format -Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitely given monotonically increasing numbers). +Format the `pattern` string with the strings listed in the arguments, similar to formatting in Python. The pattern string can contain replacement fields surrounded by curly braces `{}`. Anything not contained in braces is considered literal text and copied verbatim into the output. Literal brace character can be escaped by two braces: `{{ '{{' }}` and `{{ '}}' }}`. Field names can be numbers (starting from zero) or empty (then they are implicitly given monotonically increasing numbers). **Syntax** diff --git a/docs/en/sql-reference/functions/string-replace-functions.md b/docs/en/sql-reference/functions/string-replace-functions.md index 56c527d734e..74d5d747193 100644 --- a/docs/en/sql-reference/functions/string-replace-functions.md +++ b/docs/en/sql-reference/functions/string-replace-functions.md @@ -6,7 +6,7 @@ sidebar_label: Replacing in Strings # Functions for Replacing in Strings -[General strings functions](string-functions.md) and [functions for searchin in strings](string-search-functions.md) are described separately. +[General strings functions](string-functions.md) and [functions for searching in strings](string-search-functions.md) are described separately. ## replaceOne diff --git a/docs/en/sql-reference/functions/udf.md b/docs/en/sql-reference/functions/udf.md index a58c1364780..9c6b1b0c66b 100644 --- a/docs/en/sql-reference/functions/udf.md +++ b/docs/en/sql-reference/functions/udf.md @@ -19,7 +19,7 @@ A function configuration contains the following settings: - `argument` - argument description with the `type`, and optional `name` of an argument. Each argument is described in a separate setting. Specifying name is necessary if argument names are part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Default argument name value is `c` + argument_number. - `format` - a [format](../../interfaces/formats.md) in which arguments are passed to the command. - `return_type` - the type of a returned value. -- `return_name` - name of retuned value. Specifying return name is necessary if return name is part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Optional. Default value is `result`. +- `return_name` - name of returned value. Specifying return name is necessary if return name is part of serialization for user defined function format like [Native](../../interfaces/formats.md#native) or [JSONEachRow](../../interfaces/formats.md#jsoneachrow). Optional. Default value is `result`. - `type` - an executable type. If `type` is set to `executable` then single command is started. If it is set to `executable_pool` then a pool of commands is created. - `max_command_execution_time` - maximum execution time in seconds for processing block of data. This setting is valid for `executable_pool` commands only. Optional. Default value is `10`. - `command_termination_timeout` - time in seconds during which a command should finish after its pipe is closed. After that time `SIGTERM` is sent to the process executing the command. Optional. Default value is `10`. diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 8a8c86624d2..bfad16f8365 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -222,7 +222,7 @@ It also makes sense to specify a local table in the `GLOBAL IN` clause, in case ### Distributed Subqueries and max_rows_in_set -You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is tranferred during distributed queries. +You can use [`max_rows_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) and [`max_bytes_in_set`](../../operations/settings/query-complexity.md#max-rows-in-set) to control how much data is transferred during distributed queries. This is specially important if the `global in` query returns a large amount of data. Consider the following sql - ```sql diff --git a/docs/en/sql-reference/statements/alter/quota.md b/docs/en/sql-reference/statements/alter/quota.md index 74a184c1479..d41e2ff0f61 100644 --- a/docs/en/sql-reference/statements/alter/quota.md +++ b/docs/en/sql-reference/statements/alter/quota.md @@ -32,7 +32,7 @@ Limit the maximum number of queries for the current user with 123 queries in 15 ALTER QUOTA IF EXISTS qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER; ``` -For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters: +For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quarters: ``` sql ALTER QUOTA IF EXISTS qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default; diff --git a/docs/en/sql-reference/statements/create/quota.md b/docs/en/sql-reference/statements/create/quota.md index 7c31f93fff7..c69285171ab 100644 --- a/docs/en/sql-reference/statements/create/quota.md +++ b/docs/en/sql-reference/statements/create/quota.md @@ -32,7 +32,7 @@ Limit the maximum number of queries for the current user with 123 queries in 15 CREATE QUOTA qA FOR INTERVAL 15 month MAX queries = 123 TO CURRENT_USER; ``` -For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quaters: +For the default user limit the maximum execution time with half a second in 30 minutes, and limit the maximum number of queries with 321 and the maximum number of errors with 10 in 5 quarters: ``` sql CREATE QUOTA qB FOR INTERVAL 30 minute MAX execution_time = 0.5, FOR INTERVAL 5 quarter MAX queries = 321, errors = 10 TO default; diff --git a/docs/en/sql-reference/statements/explain.md b/docs/en/sql-reference/statements/explain.md index 1c93707402f..2d7204c2796 100644 --- a/docs/en/sql-reference/statements/explain.md +++ b/docs/en/sql-reference/statements/explain.md @@ -115,7 +115,7 @@ CROSS JOIN system.numbers AS c Settings: -- `run_passes` — Run all query tree passes before dumping the query tree. Defaul: `1`. +- `run_passes` — Run all query tree passes before dumping the query tree. Default: `1`. - `dump_passes` — Dump information about used passes before dumping the query tree. Default: `0`. - `passes` — Specifies how many passes to run. If set to `-1`, runs all the passes. Default: `-1`. @@ -463,5 +463,5 @@ Result: ``` :::note -The validation is not complete, so a successfull query does not guarantee that the override would not cause issues. +The validation is not complete, so a successful query does not guarantee that the override would not cause issues. ::: diff --git a/docs/en/sql-reference/statements/select/from.md b/docs/en/sql-reference/statements/select/from.md index 4ca8e8287c0..a4f449ad321 100644 --- a/docs/en/sql-reference/statements/select/from.md +++ b/docs/en/sql-reference/statements/select/from.md @@ -34,7 +34,7 @@ Queries that use `FINAL` are executed slightly slower than similar queries that - Data is merged during query execution. - Queries with `FINAL` read primary key columns in addition to the columns specified in the query. -**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine have’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). +**In most cases, avoid using `FINAL`.** The common approach is to use different queries that assume the background processes of the `MergeTree` engine haven’t happened yet and deal with it by applying aggregation (for example, to discard duplicates). `FINAL` can be applied automatically using [FINAL](../../../operations/settings/settings.md#final) setting to all tables in a query using a session or a user profile. diff --git a/docs/en/sql-reference/statements/select/order-by.md b/docs/en/sql-reference/statements/select/order-by.md index 712395a0357..3dfbd133364 100644 --- a/docs/en/sql-reference/statements/select/order-by.md +++ b/docs/en/sql-reference/statements/select/order-by.md @@ -289,7 +289,7 @@ When `FROM const_expr` not defined sequence of filling use minimal `expr` field When `TO const_expr` not defined sequence of filling use maximum `expr` field value from `ORDER BY`. When `STEP const_numeric_expr` defined then `const_numeric_expr` interprets `as is` for numeric types, as `days` for Date type, as `seconds` for DateTime type. It also supports [INTERVAL](https://clickhouse.com/docs/en/sql-reference/data-types/special-data-types/interval/) data type representing time and date intervals. When `STEP const_numeric_expr` omitted then sequence of filling use `1.0` for numeric type, `1 day` for Date type and `1 second` for DateTime type. -`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeate previous value. Omitted list will result in including all allowed columns. +`INTERPOLATE` can be applied to columns not participating in `ORDER BY WITH FILL`. Such columns are filled based on previous fields values by applying `expr`. If `expr` is not present will repeat previous value. Omitted list will result in including all allowed columns. Example of a query without `WITH FILL`: diff --git a/docs/en/sql-reference/window-functions/index.md b/docs/en/sql-reference/window-functions/index.md index 7ee2102c14d..a8f494a5afc 100644 --- a/docs/en/sql-reference/window-functions/index.md +++ b/docs/en/sql-reference/window-functions/index.md @@ -21,7 +21,7 @@ ClickHouse supports the standard grammar for defining windows and window functio | `lag/lead(value, offset)` | Not supported. Workarounds: | | | 1) replace with `any(value) over (.... rows between preceding and preceding)`, or `following` for `lead` | | | 2) use `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` | -| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unounded following). | +| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). | ## ClickHouse-specific Window Functions @@ -39,7 +39,7 @@ The computed value is the following for each row: The roadmap for the initial support of window functions is [in this issue](https://github.com/ClickHouse/ClickHouse/issues/18097). -All GitHub issues related to window funtions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. +All GitHub issues related to window functions have the [comp-window-functions](https://github.com/ClickHouse/ClickHouse/labels/comp-window-functions) tag. ### Tests diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index ded7a4643a9..583a49631a3 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1,376 +1,1159 @@ personal_ws-1.1 en 543 -AArch -ACLs -AMQP -ARMv -ASLR -ASan -Actian -AddressSanitizer -AppleClang -ArrowStream -AvroConfluent -BSON -BSONEachRow -Bool -BuilderBinAarch -BuilderBinAmd -CCTOOLS -CLion -CMake -CMakeLists -CPUs -CSVWithNames -CSVWithNamesAndTypes -CamelCase -CapnProto -CentOS -ClickHouse -ClickHouse's -ClickableSquare -CodeBlock -CodeLLDB -Compat -Config -ConnectionDetails -Contrib -Ctrl -CustomSeparated -CustomSeparatedWithNames -CustomSeparatedWithNamesAndTypes -DBMSs -DateTime -DateTimes -DockerHub -Doxygen -Encodings -Enum -Eoan -FixedString -FreeBSD -Fuzzer -Fuzzers -GTest -Gb -Gcc -GoogleTest -HDDs -Heredoc -Homebrew -Homebrew's -HorizontalDivide -Hostname -INSERTed -IPv -IntN -Integrations -JSONAsObject -JSONAsString -JSONColumns -JSONColumnsWithMetadata -JSONCompact -JSONCompactColumns -JSONCompactEachRow -JSONCompactEachRowWithNames -JSONCompactEachRowWithNamesAndTypes -JSONCompactStrings -JSONCompactStringsEachRow -JSONCompactStringsEachRowWithNames -JSONCompactStringsEachRowWithNamesAndTypes -JSONEachRow -JSONEachRowWithProgress -JSONObjectEachRow -JSONStrings -JSONStringsEachRow -JSONStringsEachRowWithProgress -JSONs -Jaeger -Jemalloc -Jepsen -KDevelop -LGPL -LLDB -LLVM's -LOCALTIME -LOCALTIMESTAMP -LibFuzzer -LineAsString -LinksDeployment -LowCardinality -MEMTABLE -MSan -MVCC -MacOS -Memcheck -MemorySanitizer -MergeTree -MessagePack -MiB -MsgPack -Multiline -Multithreading -MySQLDump -NEKUDOTAYIM -NULLIF -NVME -NYPD -NuRaft -OLAP -OLTP -ObjectId -Observability -Ok -OpenSSL -OpenSUSE -OpenStack -OpenTelemetry -PAAMAYIM -ParquetMetadata -Parsers -Postgres -Precompiled -PrettyCompact -PrettyCompactMonoBlock -PrettyCompactNoEscapes -PrettyCompactNoEscapesMonoBlock -PrettyJSONEachRow -PrettyMonoBlock -PrettyNoEscapes -PrettyNoEscapesMonoBlock -PrettySpace -PrettySpaceMonoBlock -PrettySpaceNoEscapes -PrettySpaceNoEscapesMonoBlock -Protobuf -ProtobufSingle -QEMU -QTCreator -QueryCacheHits -QueryCacheMisses -RBAC -RawBLOB -RedHat -ReplicatedMergeTree -RowBinary -RowBinaryWithNames -RowBinaryWithNamesAndTypes -Runtime -SATA -SELECTs -SERIALIZABLE -SIMD -SLES -SMALLINT -SQLInsert -SQLSTATE -SSSE -Schemas -SelfManaged -Stateful -Submodules -Subqueries -TSVRaw -TSan -TabItem -TabSeparated -TabSeparatedRaw -TabSeparatedRawWithNames -TabSeparatedRawWithNamesAndTypes -TabSeparatedWithNames -TabSeparatedWithNamesAndTypes -TargetSpecific -TemplateIgnoreSpaces -Testflows -Tgz -Toolset -Tradeoff -Transactional -TwoColumnList -UBSan -UInt -UIntN -UPDATEs -Uint -Updatable -Util -Valgrind -Vectorized -VideoContainer -ViewAllLink -VirtualBox -Werror -WithNamesAndTypes -Woboq -WriteBuffer -WriteBuffers -XCode -YAML -YYYY -Yasm -Zipkin -ZooKeeper -ZooKeeper's aarch +AArch +accurateCast +accurateCastOrDefault +accurateCastOrNull +ACLs +acos +acosh +Actian +ActionsMenu +activecube +activerecord +ActiveRecord +addDays +addHours +Additionaly +addMinutes +addMonths +addQuarters +addr +AddressSanitizer +addressToLine +addressToLineWithInlines +addressToSymbol +addSeconds +addWeeks +addYears +adhoc +adviced +aggregatefunction +Aggregatefunction +AggregateFunction +aggregatingmergetree +AggregatingMergeTree +aggregatio +AggregatorThreads +AggregatorThreadsActive +aggretate +aiochclient +Akka +alculates +AlertManager +Alexey allocator +alphaTokens +ALTERs +amplab +AMPLab +AMQP analytics +ANNIndex +ANNIndexes +anonymize anonymized ansi +AnyEvent +anyheavy +anyHeavy +anyIf +anylast +anyLast +AORM +APIs +appendTrailingCharIfAbsent +AppleClang +approximatly +argmax +argMax +argmin +argMin +arguments's +argumetn +arithmetics +ARMv +arrayAll +arrayAUC +arrayAvg +arrayCompact +arrayConcat +arrayCount +arrayCumSum +arrayCumSumNonNegative +arrayDifference +arrayDistinct +arrayElement +arrayEnumerate +arrayEnumerateDense +arrayEnumerateUniq +arrayExists +arrayFill +arrayFilter +arrayFirst +arrayFirstIndex +arrayFlatten +arrayIntersect +arrayJoin +ArrayJoin +arrayLast +arrayLastIndex +arrayMap +arrayMax +arrayMin +arrayPartialReverseSort +arrayPartialSort +arrayPopBack +arrayPopFront +arrayProduct +arrayPushBack +arrayPushFront +arrayReduce +arrayReduceInRanges +arrayResize +arrayReverse +arrayReverseFill +arrayReverseSort +arrayReverseSplit +arraySlice +arraySort +arraySplit +arrayStringConcat +arraySum +arrayUniq +arrayWithConstant +arrayZip +ArrowStream +ASan +ascii +asin +asinh +ASLR +ASOF +assumeNotNull +asterics async +asynch +AsynchronousHeavyMetricsCalculationTimeSpent +AsynchronousHeavyMetricsUpdateInterval +AsynchronousInsert +AsynchronousInsertThreads +AsynchronousInsertThreadsActive +AsynchronousMetricsCalculationTimeSpent +AsynchronousMetricsUpdateInterval +AsynchronousReadWait +AsyncInsertCacheSize +atan +atanh atomicity +auth +authenticator +Authenticator +authenticators +Authenticators +autocompletion +Autocompletion +autodetect +autodetected autogenerated autogeneration +AutoML autostart +avgweighted +avgWeighted avro +AvroConfluent avx aws backend +BackgroundBufferFlushSchedulePool +BackgroundBufferFlushSchedulePoolSize +BackgroundBufferFlushSchedulePoolTask +BackgroundCommonPoolSize +BackgroundCommonPoolTask +BackgroundDistributedSchedulePool +BackgroundDistributedSchedulePoolSize +BackgroundDistributedSchedulePoolTask +BackgroundFetchesPoolSize +BackgroundFetchesPoolTask +BackgroundMergesAndMutationsPoolSize +BackgroundMergesAndMutationsPoolTask +BackgroundMessageBrokerSchedulePoolSize +BackgroundMessageBrokerSchedulePoolTask +BackgroundMovePoolSize +BackgroundMovePoolTask +BackgroundProcessingPool +BackgroundSchedulePool +BackgroundSchedulePoolSize +BackgroundSchedulePoolTask backoff backticks +BackupsIO +BackupsIOThreads +BackupsIOThreadsActive +BackupsThreads +BackupsThreadsActive +balancer +basename +bcrypt benchmarking +BestEffort +BestEffortOrNull +BestEffortOrZero +BestEffortUS +BestEffortUSOrNull +BestEffortUSOrZero +bfloat +BIGINT +BIGSERIAL +binlog +bitAnd +bitCount +bitHammingDistance +bitmapAnd +bitmapAndCardinality +bitmapAndnot +bitmapAndnotCardinality +bitmapBuild +bitmapCardinality +bitmapContains +bitmapHasAll +bitmapHasAny +bitmapMax +bitmapMin +bitmapOr +bitmapOrCardinality +bitmapSubsetInRange +bitmapSubsetLimit +bitmapToArray +bitmapTransform +bitmapXor +bitmapXorCardinality +bitmask +bitmaskToArray +bitmaskToList +bitNot +bitOr +bitov +bitPositionsToArray +bitRotateLeft +bitRotateRight +bitShiftLeft +bitShiftRight +bitSlice +bitTest +bitTestAll +bitTestAny +bitXor blake +Blazingly +BlockActiveTime +BlockDiscardBytes +BlockDiscardMerges +BlockDiscardOps +BlockDiscardTime +BlockInFlightOps +blockinfo +blockNumber +BlockQueueTime +BlockReadBytes +blockreader +BlockReadMerges +BlockReadOps +BlockReadTime +blockSerializedSize +blocksize blockSize +BlockWriteBytes +BlockWriteMerges +BlockWriteOps +BlockWriteTime +bool +Bool boolean bools boringssl +BORO +bozerkins +broadcasted +BrokenDistributedFilesToInsert brotli bson +BSON bsoneachrow +BSONEachRow +buffersize buildable +BuilderBinAarch +BuilderBinAmd +buildId +BuildID +builtins +bytebase +Bytebase +byteSize +bytesToCutForIPv +CacheDetachedFileSegments +CacheDictionaries +CacheDictionary +CacheDictionaryThreads +CacheDictionaryThreadsActive +CacheDictionaryUpdateQueueBatches +CacheDictionaryUpdateQueueKeys +CacheFileSegments +cacheSessions +cachesize +caConfig camelCase +CamelCase +CapContains capn +Cap'n capnproto +CapnProto +CapUnion +cardinalities cardinality +cartesian cassandra +casted +catboost +CatBoost +catboostEvaluate +categoricalinformationvalue +categoricalInformationValue +cathetus cbindgen +cbrt ccache +CCTOOLS cctz +CDATA +CDFs +CDMA +ceil +CellAreaM +CellAreaRads +CellsIntersect +CentOS +centroid +certificateFile +CertificateHandler +CESU +cetera cfg +chadmin +Chadmin changelog changelogs +ChannelID charset charsets +chconn checkouting checksummed checksumming checksums +childern +chproxy +chunksize +cickhouse +Cidr +CIDR +CIDRToRange +cipherList +ciphertext +Ciphertext cityhash +cityHash +CityHash +CLang cli +ClickableSquare +clickcat +ClickCat clickhouse +ClickHouse +ClickHouseClient +clickhousedb +ClickHouseMigrator +ClickHouseNIO +ClickHouse's +ClickHouseVapor +clickhousex +clickmate clickstream +clickvisual +ClickVisual +CLion +CLOB +clockhour +cLoki +CloudDetails +clusterAllReplicas cmake +CMake +CMakeLists +CMPLNT codebase +CodeBlock codec +codecs +Codecs +CODECS +CodeLLDB +codepoint +codepoints +collapsingmergetree +CollapsingMergeTree +combinator +combinators +Combinators +commnents comparising +Compat +compatitalbe +CompiledExpressionCacheBytes +CompiledExpressionCacheCount +ComplexKeyCache +ComplexKeyDirect +ComplexKeyHashed +compressability +concat +concatAssumeInjective +concatWithSeparator +concatWithSeparatorAssumeInjective +concurenly +cond +conf config +Config configs +congruential +conjuction +conjuctive +ConnectionDetails +const +Const +ContextLockWait contrib +Contrib +convergance +convertCharset coroutines +cosineDistance +countDigits +countEqual +countMatches +countSubstrings +covariates +covarpop +covarPop +covarsamp +covarSamp +covid +Covid +COVID cpp cppkafka cpu +CPUFrequencyMHz +CPUs +Cramer's +cramersv +cramersV +cramersvbiascorrected +cramersVBiasCorrected +criteo +Criteo crlf croaring cronjob +Crotty +Crowdsourced +cryptocurrencies +cryptocurrency +cryptographic csv +CSVs csvwithnames +CSVWithNames csvwithnamesandtypes +CSVWithNamesAndTypes +CTEs +Ctrl +currentDatabase +CurrentMetrics +currentProfiles +currentRoles +currentUser +customizable +customizations customseparated +CustomSeparated customseparatedwithnames +CustomSeparatedWithNames customseparatedwithnamesandtypes +CustomSeparatedWithNamesAndTypes +cutFragment +cutIPv +cutQueryString +cutQueryStringAndFragment +cutToFirstSignificantSubdomain +cutToFirstSignificantSubdomainCustom +cutToFirstSignificantSubdomainCustomWithWWW +cutToFirstSignificantSubdomainWithWWW +cutURLParameter +cutWWW cyrus +DatabaseCatalog +DatabaseCatalogThreads +DatabaseCatalogThreadsActive +DatabaseOnDisk +DatabaseOnDiskThreads +DatabaseOnDiskThreadsActive +DatabaseOrdinaryThreads +DatabaseOrdinaryThreadsActive datacenter +datacenters datafiles +datagrip +DataGrip +datalens +DataLens +datanode dataset datasets +datasource +DataTime +datatypes +DataTypes +dateName datetime +dateTime +DateTime datetimes +DateTimes +dateTimeToSnowflake +dayofyear +dbal +DBAs +DbCL +dbeaver +DBeaver +dbgen dbms +DBMSs ddl +DDLWorker +DDLWORKER +DDLWorkerThreads +DDLWorkerThreadsActive deallocation +deallocations debian +decodeURLComponent +decodeURLFormComponent +decodeXMLComponent decompressor +decrypt +DECRYPT +decrypted +Decrypted +decrypts +deduplicate +Deduplicate +deduplicated +deduplicating +deduplication +Deduplication +defaultProfiles +defaultRoles +defaultValueOfArgumentType +defaultValueOfTypeName +DelayedInserts +DeliveryTag +deltalake +deltaLake +DeltaLake +deltasum +deltaSum +deltasumtimestamp +deltaSumTimestamp +demangle +denormalize +Denormalize +denormalized denormalizing denormals +DESC deserialization deserialized +deserializing +DestroyAggregatesThreads +DestroyAggregatesThreadsActive destructor destructors +detectCharset +detectLanguage +detectLanguageMixed +detectLanguageUnknown +determinator +deterministically +DictCacheRequests +dictGet +dictGetChildren +dictGetDescendant +dictGetHierarchy +dictGetOrDefault +dictGetOrNull +dictGetUUID +dictHas +dictIsIn +disableProtocols +disjunction +disjunctions +DiskAvailable +DiskObjectStorage +DiskObjectStorageAsyncThreads +DiskObjectStorageAsyncThreadsActive +DiskSpaceReservedForMerge +DiskTotal +DiskUnreserved +DiskUsed +displaySecretsInShowAndSelect +DistributedFilesToInsert +DistributedSend +distro +divideDecimal dmesg +DockerHub +DOGEFI +domainWithoutWWW dont +dotProduct +DoubleDelta +Doxygen +dplyr dragonbox +dropoff +dumpColumnStructure durations +ECMA +ecto +Ecto +EdgeAngle +EdgeLengthKm +EdgeLengthM +EmbeddedRocksDB +embeddings +Embeddings +emptyArray +emptyArrayDate +emptyArrayDateTime +emptyArrayFloat +emptyArrayInt +emptyArrayString +emptyArrayToSingle +emptyArrayUInt +enabledProfiles +enabledRoles +encodeURLComponent +encodeURLFormComponent +encodeXMLComponent encodings +Encodings +encryptions endian +endsWith +Engeneering enum +Enum +enum's +enums +Enums +Eoan +EphemeralNode +erfc +errorCodeToName +Ethereum +evalMLMethod +everytime +ExactEdgeLengthKm +ExactEdgeLengthM +ExactEdgeLengthRads +ExecutablePool exFAT +expiryMsec +exponentialmovingaverage +exponentialMovingAverage +expr +exprN +extendedVerification +extention +ExternalDistributed +extractAll +extractAllGroups +extractAllGroupsHorizontal +extractAllGroupsVertical +extractKeyValuePairs +extractKeyValuePairsWithEscaping +extractTextFromHTML +extractURLParameter +extractURLParameterNames +extractURLParameters +ExtType +failover +Failover +farmFingerprint +farmHash +FarmHash fastops fcoverage +FFFD filesystem +filesystemAvailable +FilesystemCacheBytes +FilesystemCacheElements +FilesystemCacheFiles +FilesystemCacheReadBuffers +FilesystemCacheSize +filesystemCapacity +filesystemFree +FilesystemLogsPathAvailableBytes +FilesystemLogsPathAvailableINodes +FilesystemLogsPathTotalBytes +FilesystemLogsPathTotalINodes +FilesystemLogsPathUsedBytes +FilesystemLogsPathUsedINodes +FilesystemMainPathAvailableBytes +FilesystemMainPathAvailableINodes +FilesystemMainPathTotalBytes +FilesystemMainPathTotalINodes +FilesystemMainPathUsedBytes +FilesystemMainPathUsedINodes filesystems +finalizeAggregation +fips +FIPS +firstSignificantSubdomain +firstSignificantSubdomainCustom +fixedstring +FixedString +flamegraph flatbuffers +flink +Flink +fluentd fmtlib +ForEach +formatDateTime +formatDateTimeInJoda +formatDateTimeInJodaSyntax +formated +formatReadableDecimalSize +formatReadableQuantity +formatReadableSize +formatReadableTimeDelta +formatRow +formatRowNoNewline formatschema formatter +FOSDEM +FQDN +FreeBSD +freezed +fromModifiedJulianDay +fromModifiedJulianDayOrNull +fromUnixTimestamp +fromUnixTimestampInJodaSyntax fsync +func +funtion +fuzzBits fuzzer +Fuzzer fuzzers -gRPC +Fuzzers +Gb +Gbit +Gcc +gccMurmurHash gcem +generateRandom +GenerateRandom +generateULID +generateUUIDv +geobase +geobases +Geobases +geocode +GeoCoord +geoDistance +geohash +Geohash +geohashDecode +geohashEncode +geohashesInBox +Geoid +geoip +geospatial +geoToH +geoToS +GetBaseCell +getblockinfo +GetDestinationIndexFromUnidirectionalEdge +getevents +GetFaces +GetIndexesFromUnidirectionalEdge +getMacro +GetNeighbors +GetOriginIndexFromUnidirectionalEdge +getOSKernelVersion +GetPentagonIndexes +GetRes +GetResolution +getServerPort +getSetting +getSizeOfEnumType +GetUnidirectionalEdge +GetUnidirectionalEdgeBoundary +GetUnidirectionalEdgesFromHexagon github +GitLab glibc +globalIn +globalNotIn +GlobalThread +GlobalThreadActive +glushkovds +GoLand +golang googletest +GoogleTest +grafana +Grafana +graphitemergetree +GraphiteMergeTree +graphouse +graphql +GraphQL +greatCircleAngle +greatCircleDistance +greaterOrEquals +greenspace +Greenwald +grouparray +groupArray +grouparrayinsertat +groupArrayInsertAt +grouparraylast +groupArrayLast +grouparraymovingavg +groupArrayMovingAvg +grouparraymovingsum +groupArrayMovingSum +grouparraysample +groupArraySample +groupbitand +groupBitAnd +groupbitmap +groupBitmap +groupbitmapand +groupBitmapAnd +groupbitmapor +groupBitmapOr +groupbitmapxor +groupBitmapXor +groupbitor +groupBitOr +groupbitxor +groupBitXor +groupuniqarray +groupUniqArray grpc +gRPC grpcio gtest +GTest +gtid +GTID +gzip +gzipped +hadoop +halfday +halfMD hardlinks +hasAll +hasAny +hasColumnInTable +HashedDictionary +HashedDictionaryThreads +HashedDictionaryThreadsActive +hashtables +hasSubstr +hasToken +hasTokenCaseInsensitive +hasTokenCaseInsensitiveOrNull +hasTokenOrNull +haversine +Haversine +have't +hdbc +HDDs hdfs +hdfsCluster heredoc +Heredoc heredocs +HexAreaKm +HexAreaM +HexRing +HHMM +Hight +hiveHash +HMAC +holistics +Holistics homebrew +Homebrew +Homebrew's +hopEnd +hopStart +horgh +HorizontalDivide +hostname +hostName +Hostname +hostnames +houseops +HouseOps +hsts +HSTS +html http +HTTPConnection https +HTTPThreads +hudi +Hudi +HyperLogLog hyperscan +hypot +Hypot +hyvor +IANA +icosahedron icudata +idempotency +identifiant +Identifiant +ifNotFinite +ifNull +iframe +ilike +IMDS +implicitely +incrementing +Incrementing +incremnt +IndexesAreNeighbors +indexHint +indexOf +infi +INFILE +InfluxDB +initializeAggregation +initialQueryID +injective +innogames +inodes +INSERTed +INSERTs +Instana instantiation +intDiv +intDivOrZero integrational integrations +Integrations +IntelliJ interserver +InterserverConnection +InterserverThreads +intervalLengthSum +intExp +intHash +IntN +introspections +invalidCertificateHandler invariants +invertedindexes +IOPrefetchThreads +IOPrefetchThreadsActive +IOThreads +IOThreadsActive +IOUringInFlightEvents +IOUringPendingEvents +IOWriterThreads +IOWriterThreadsActive +IPTrie +IPv +isConstant +isDecimalOverflow +isFinite +isInfinite +isIPAddressInRange +isIPv +isNaN +isNotNull +isNull +IsPentagon +IsResClassIII +IsValid +isValidJSON +isValidUTF +iteratively +Jaeger +Jannis +javaHash +JavaHash +javaHashUTF +jbod +JBOD jdbc jemalloc +Jemalloc +Jepsen +JetBrains +Jitter +Joda +JOINed +joinGet +JOINs json +JSONArrayLength +JSONAsObject jsonasstring +JSONAsString jsoncolumns +JSONColumns jsoncolumnsmonoblock +JSONColumnsWithMetadata jsoncompact +JSONCompact jsoncompactcolumns +JSONCompactColumns jsoncompacteachrow +JSONCompactEachRow jsoncompacteachrowwithnames +JSONCompactEachRowWithNames jsoncompacteachrowwithnamesandtypes +JSONCompactEachRowWithNamesAndTypes jsoncompactstrings +JSONCompactStrings jsoncompactstringseachrow +JSONCompactStringsEachRow jsoncompactstringseachrowwithnames +JSONCompactStringsEachRowWithNames jsoncompactstringseachrowwithnamesandtypes +JSONCompactStringsEachRowWithNamesAndTypes jsoneachrow +JSONEachRow jsoneachrowwithprogress +JSONEachRowWithProgress +JSONExtract +JSONExtractArrayRaw +JSONExtractBool +JSONExtractFloat +JSONExtractInt +JSONExtractKeys +JSONExtractKeysAndValues +JSONExtractKeysAndValuesRaw +JSONExtractRaw +JSONExtractString +JSONExtractUInt +JSONHas +JSONLength jsonobjecteachrow +JSONObjectEachRow +JSONs jsonstrings +JSONStrings jsonstringseachrow +JSONStringsEachRow jsonstringseachrowwithprogress +JSONStringsEachRowWithProgress +JSONType +jumpConsistentHash +JumpConsistentHash +Jupyter kafka +KafkaAssignedPartitions +KafkaBackgroundReads kafkacat +KafkaConsumers +KafkaConsumersInUse +KafkaConsumersWithAssignment +KafkaLibrdkafkaThreads +kafkaMurmurHash +KafkaProducers +KafkaWrites +Kahan +KDevelop +KeeperAliveConnections +keepermap +KeeperMap +KeeperOutstandingRequets +kerberized +kerberos +Kerberos +kernal +keyspace +keytab +Khanna +kittenhouse +KittenHouse +Klickhouse +Kolmogorov +kolmogorovsmirnovtest +kolmogorovSmirnovTest +kolya konsole +kRing +Kubernetes +kurtosis +kurtpop +kurtPop +kurtsamp +kurtSamp laion +lang +laravel latencies +ldap +LDAP +learing +leftPad +leftPadUTF +lemmatization +lemmatize +lemmatized +lengthUTF +lessOrEquals lexicographically -libFuzzer +lgamma +LGPL libc +libcatboost libcpuid libcxx libcxxabi libdivide libfarmhash libfuzzer +libFuzzer +LibFuzzer libgsasl libhdfs libmetrohash @@ -381,193 +1164,1405 @@ libs libunwind libuv libvirt +LightHouse linearizability linearizable +linearized lineasstring +LineAsString linefeeds lineorder +Linf +LinfDistance +LinfNorm +LinfNormalize +LinksDeployment +Linq linux +LLDB llvm +LLVM's +LoadAverage +loadDefaultCAFile localhost +localread +LocalThread +LocalThreadActive +LOCALTIME +LOCALTIMESTAMP +logagent +loghouse +LogQL +Logstash +logTrace +london +LONGLONG +LookML +lowcardinality +LowCardinality +lowercased +lowerUTF +LpDistance +LpNorm +LpNormalize +Luebbe +Lyft +lzma +MacBook +MACNumToString macOS +MacOS +MACStringToNum +MACStringToOUI +mailrugo +mailto +makeDate +makeDateTime +mannwhitneyutest +mannWhitneyUTest +mapAdd +mapAll +mapApply +mapConcat +mapContains +mapContainsKeyLike +mapExists +mapExtractKeyLike +mapFilter +mapFromArrays +mapKeys +mappedfile +mapPopulateSeries +mapReverseSort +mapSort +mapSubtract +mapUpdate +mapValues mariadb +MarkCacheBytes +MarkCacheFiles +MarksLoaderThreads +MarksLoaderThreadsActive +matcher +MaterializedMySQL +MaterializedPostgreSQL +materializedview +MaterializedView +MaxDDLEntryID +maxintersections +maxIntersections +maxintersectionsposition +maxIntersectionsPosition +maxmap +maxMap +maxmind +MaxMind +MaxPartCountForPartition +MaxPushedDDLEntryID +Mbps mdadm +meanztest +meanZTest +mebibytes +MEDIUMINT +Memcheck +MemoryCode +MemoryDataAndStack +MemoryResident +MemorySanitizer +MemoryShared +MemoryTracking +MemoryVirtual +MEMTABLE +mergeable +MergeJoin +MergeState +mergetree +MergeTree +MergeTreeAllRangesAnnouncementsSent +MergeTreeBackgroundExecutor +MergeTreeBackgroundExecutorThreads +MergeTreeBackgroundExecutorThreadsActive +MergeTreeDataSelectExecutor +MergeTreeDataSelectExecutorThreads +MergeTreeDataSelectExecutorThreadsActive +MergeTreePartsCleanerThreads +MergeTreePartsCleanerThreadsActive +MergeTreePartsLoaderThreads +MergeTreePartsLoaderThreadsActive +MergeTreeReadTaskRequestsSent +MergeTreeSettings +messageID +MessagePack +metacharacters +Metastore +metasymbols +metrica +metroHash +MetroHash +mfedotov +MiB +Milli +Milovidov +mindsdb +MindsDB +MinHash +minimalistic +mininum +MinIO miniselect +minmap +minMap +minmax +MinMax +mins +misconfiguration +mispredictions +mmap +MMapCacheCells +mmapped +MMappedAllocBytes +MMappedAllocs +MMappedFileBytes +MMappedFiles +moduloOrZero +mongodb +Mongodb +monthName +moscow +MSan msgpack +MsgPack msgpk +MSSQL +multibyte +multiFuzzyMatchAllIndices +multiFuzzyMatchAny +multiFuzzyMatchAnyIndex +multiIf multiline +Multiline +multiMatchAllIndices +multiMatchAny +multiMatchAnyIndex +multiplyDecimal +multipolygon +MultiPolygon +Multiqueries +multiSearchAllPositions +multiSearchAllPositionsUTF +multiSearchAny +multiSearchFirstIndex +multiSearchFirstPosition +multisets multithread +Multithreading +multiword +Multiword +munmap murmurhash +murmurHash +MurmurHash +musqldump mutex +MVCC +mydb +myfilter mysql +MySQLConnection mysqldump +MySQLDump mysqljs +MySQLThreads +mytable +Nagios +namedatabases +namenetworks +namenode +Namenode +namepassword +nameprofile +namequota +NamesAndTypesList +namespaces +Nano +NaNs natively +nats +NATS +NCHAR +negtive +NEKUDOTAYIM +Nesterov +nestjs +netloc +NetworkReceive +NetworkReceiveBytes +NetworkReceiveDrop +NetworkReceiveErrors +NetworkReceivePackets +NetworkSend +NetworkSendBytes +NetworkSendDrop +NetworkSendErrors +NetworkSendPackets +NEWDATE +NEWDECIMAL +NFKC +NFKD +ngram +ngrambf +ngramDistance +ngramMinHash +ngramMinHashArg +ngramMinHashArgCaseInsensitive +ngramMinHashArgCaseInsensitiveUTF +ngramMinHashArgUTF +ngramMinHashCaseInsensitive +ngramMinHashCaseInsensitiveUTF +ngramMinHashUTF +ngrams +ngramSearch +ngramSimHash +ngramSimHashCaseInsensitive +ngramSimHashCaseInsensitiveUTF +ngramSimHashUTF +NodeJs +nonNegativeDerivative noop +normalizedQueryHash +normalizeQuery +normalizeUTF +notEmpty +notEquals +notILike +notIn +notLike +notretry +nowInBlock +ntile nullability nullable +nullables +nullIf +NULLIF num +NumberOfDatabases +NumberOfDetachedByUserParts +NumberOfDetachedParts +NumberOfTables +numerics +NumHexagons +NumToString +NumToStringClassC +NuRaft +NVMe +NVME +nypd +NYPD obfuscator +ObjectId +observability +Observability +Octonica odbc +OFNS ok -openSUSE +Ok +OLAP +OLTP +omclickhouse +onstraints +ontime +OnTime +OpenCelliD +OpenFileForRead +OpenFileForWrite openldap +opensky +OpenSky +openssl +openSSL +OpenSSL +OpenStack +openSUSE +OpenSUSE opentelemetry +OpenTelemetry +optinal +Optinal +OrDefault +OrNull +OrZero +OSContextSwitches +OSGuestNiceTime +OSGuestNiceTimeCPU +OSGuestNiceTimeNormalized +OSGuestTime +OSGuestTimeCPU +OSGuestTimeNormalized +OSIdleTime +OSIdleTimeCPU +OSIdleTimeNormalized +OSInterrupts +OSIOWaitTime +OSIOWaitTimeCPU +OSIOWaitTimeNormalized +OSIrqTime +OSIrqTimeCPU +OSIrqTimeNormalized +OSMemoryAvailable +OSMemoryBuffers +OSMemoryCached +OSMemoryFreePlusCached +OSMemoryFreeWithoutCached +OSMemoryTotal +OSNiceTime +OSNiceTimeCPU +OSNiceTimeNormalized +OSOpenFiles +OSProcessesBlocked +OSProcessesCreated +OSProcessesRunning +OSSoftIrqTime +OSSoftIrqTimeCPU +OSSoftIrqTimeNormalized +OSStealTime +OSStealTimeCPU +OSStealTimeNormalized +OSSystemTime +OSSystemTimeCPU +OSSystemTimeNormalized +OSThreadsRunnable +OSThreadsTotal +OSUptime +OSUserTime +OSUserTimeCPU +OSUserTimeNormalized +OTLP +outfile +OUTFILE overcommit +overcommitted +OvercommitTracker +overfitting +PAAMAYIM +packetpool +packetsize +PagerDuty +pageviews +pandahouse +ParallelFormattingOutputFormatThreads +ParallelFormattingOutputFormatThreadsActive parallelization parallelize parallelized +ParallelParsingInputFormat +ParallelParsingInputFormatThreads +ParallelParsingInputFormatThreadsActive +Parametrized +params +paratemer +ParquetMetadata +parsable +parseable +parseDateTime +parseDateTimeBestEffort +parseDateTimeBestEffortOrNull +parseDateTimeBestEffortOrZero +parseDateTimeBestEffortUS +parseDateTimeBestEffortUSOrNull +parseDateTimeBestEffortUSOrZero +parseDateTimeInJodaSyntax +parseDateTimeInJodaSyntaxOrNull +parseDateTimeInJodaSyntaxOrZero +parseDateTimeOrNull +parseDateTimeOrZero parsers +Parsers +parseTimeDelta +Partitioner +PartMutation +PartsActive +PartsCommitted +PartsCompact +PartsDeleteOnDestroy +PartsDeleting +PartsInMemory +PartsOutdated +PartsPreActive +PartsPreCommitted +PartsTemporary +PartsWide +pathFull pclmulqdq +pcre +PCRE +PendingAsyncInsert +Percona performant +perl +persistency +phpclickhouse +PhpStorm +pipelining +plaintext +plantuml +PlantUML poco +PointDistKm +PointDistM +PointDistRads +pointInEllipses +pointInPolygon +polygonAreaCartesian +polygonAreaSpherical +polygonConvexHullCartesian +polygonPerimeterCartesian +polygonPerimeterSpherical +polygonsDistanceCartesian +polygonsDistanceSpherical +polygonsEqualsCartesian +polygonsIntersectionCartesian +polygonsIntersectionSpherical +polygonsSymDifferenceCartesian +polygonsSymDifferenceSpherical +polygonsUnionCartesian +polygonsUnionSpherical +polygonsWithinCartesian +polygonsWithinSpherical popcnt +porthttps +positionCaseInsensitive +positionCaseInsensitiveUTF +positionUTF +positiveModulo postfix postfixes +Postgres postgresql +PostgreSQLConnection +PostgreSQLThreads +PostgresSQL pre +pread +preallocate prebuild prebuilt +Precompiled preemptable +preferServerCiphers +prefertch +prefetch +prefetchsize preloaded +prepend +prepended +prepends +preprocess +Preprocess preprocessed +preprocessing preprocessor presentational prestable prettycompact +PrettyCompact prettycompactmonoblock +PrettyCompactMonoBlock prettycompactnoescapes +PrettyCompactNoEscapes prettycompactnoescapesmonoblock +PrettyCompactNoEscapesMonoBlock prettyjsoneachrow +PrettyJSONEachRow prettymonoblock +PrettyMonoBlock prettynoescapes +PrettyNoEscapes prettynoescapesmonoblock +PrettyNoEscapesMonoBlock prettyspace +PrettySpace prettyspacemonoblock +PrettySpaceMonoBlock prettyspacenoescapes +PrettySpaceNoEscapes prettyspacenoescapesmonoblock +PrettySpaceNoEscapesMonoBlock +prewhere +Prewhere +PREWHERE +privateKeyFile +privateKeyPassphraseHandler +PrivateKeyPassphraseHandler prlimit +PROCESSLIST +procfs +ProfileEvents +profiler +Profiler +profuct +Proleptic prometheus +PromHouse +Promql +PromQL +Promtail proto protobuf +Protobuf protobufsingle +ProtobufSingle +proxied +ProxySQL +pseudorandom +pseudorandomize psql ptrs +publsh +pushdown +pwrite py +PyCharm +QEMU +qouta +qryn +QTCreator +quantile +Quantile +quantilebfloat +quantileBFloat +quantiledeterministic +quantileDeterministic +quantileexact +quantileExact +quantileExactExclusive +quantileExactHigh +quantileExactInclusive +quantileExactLow +quantileexactweighted +quantileExactWeighted +quantileGK +quantileInterpolatedWeighted +quantiles +quantilesExactExclusive +quantilesExactInclusive +quantilesGK +quantilesTimingWeighted +quantiletdigest +quantileTDigest +quantiletdigestweighted +quantileTDigestWeighted +quantiletiming +quantileTiming +quantiletimingweighted +quantileTimingWeighted +quartile +quaters +QueryCacheHits +QueryCacheMisses +queryID +QueryPreempted +queryString +queryStringAndFragment +QueryThread +QuoteMeta +rabbitmq +RabbitMQ +raduis +randBernoulli +randBinomial +randCanonical +randChiSquared +randConstant +randExponential +randFisherF +randLogNormal +randNegativeBinomial +randNormal +randomFixedString +randomPrintableASCII +randomString +randomStringUTF +randPoisson +randStudentT +randUniform +RangeHashed +rankCorr rapidjson rawblob +RawBLOB +RBAC +RClickHouse readahead readline readme readonly +ReadonlyReplica +ReadTaskRequestsSent +readWKTMultiPolygon +readWKTPolygon +reate +rebalance rebalanced +recency +RecipeNLG +recompress +recompressed +recompressing +Recompressing +recompression +Recompression +reconnection +RectAdd +RectContains +RectIntersection +RectUnion +recurse +redash +Redash +reddit +Reddit +RedHat +redisstreams +ReDoS +Refactorings +refcounter +ReferenceKeyed +regexpExtract +regexpQuoteMeta +RegexpTree +regionHierarchy +regionIn +regionToArea +regionToCity +regionToContinent +regionToCountry +regionToDistrict +regionToName +regionToPopulation +regionToTopContinent +reinitialization +reinitializing +reinterpretAs +reinterpretAsDate +reinterpretAsDateTime +reinterpretAsFixedString +reinterpretAsFloat +reinterpretAsInt +reinterpretAsString +reinterpretAsUInt +reinterpretAsUUID +RemoteRead +remoteSecure +replaceAll +replaceOne +replaceRegexpAll +replaceRegexpOne +replacingmergetree +ReplacingMergeTree +ReplicasMaxAbsoluteDelay +ReplicasMaxInsertsInQueue +ReplicasMaxMergesInQueue +ReplicasMaxQueueSize +ReplicasMaxRelativeDelay +ReplicasSumInsertsInQueue +ReplicasSumMergesInQueue +ReplicasSumQueueSize +replicatable +ReplicatedAggregatingMergeTree +ReplicatedChecks +ReplicatedCollapsingMergeTree +ReplicatedFetch +ReplicatedGraphiteMergeTree +replicatedmergetree +ReplicatedMergeTree +ReplicatedReplacingMergeTree +ReplicatedSend +ReplicatedSummingMergeTree +ReplicatedVersionedCollapsingMergeTree replxx repo representable requestor +requireTLSv +Resample +resharding +reshards +RestartReplicaThreads +RestartReplicaThreadsActive +RestoreThreads +RestoreThreadsActive resultset +retentions rethrow +retransmit retriable +retuned +Returnes +reult +reverseDNSQuery +reverseUTF +RHEL +rightPad +rightPadUTF risc riscv ro +roadmap +RoaringBitmap rocksdb -rowNumberInBlock +RocksDB +rollup +Rollup +ROLLUP +roundAge +roundBankers +roundDown +roundDuration +roundToExp +routineley rowbinary +RowBinary rowbinarywithnames +RowBinaryWithNames rowbinarywithnamesandtypes +RowBinaryWithNamesAndTypes +rowNumberInAllBlocks +rowNumberInBlock rsync +rsyslog runnable runningAccumulate +runningConcurrency +runningDifference +runningDifferenceStartingWithFirstValue runtime +Runtime russian rw +RWLock +RWLockActiveReaders +RWLockActiveWriters +RWLockWaitingReaders +RWLockWaitingWriters +SaaS +Sanjeev +Sankey sasl +SATA +satisfiable +scala +Scalable +Scatterplot +Schaefer schemas +Schemas +Schwartzian +searchin +SeasClick +seccessfully +seekable +seektable +SeekTable +SELECTs +SelfManaged +Sematext +SendExternalTables +SendScalars +separatelly +sequenceCount +sequenceMatch +sequenceNextNode +SERIALIZABLE +serverUUID +sessionCacheSize +sessionIdContext +sessionTimeout +seva +shardCount +sharded +sharding +shardNum +ShareAlike +shortcircuit +shoutout +SIGTERM +SIMD simdjson +Simhash +SimHash +simpleaggregatefunction +SimpleAggregateFunction +simplelinearregression +simpleLinearRegression +SimpleState +simpliest +simpod +singlepart +sinh +siphash +sipHash +SipHash +skewness +skewpop +skewPop +skewsamp +skewSamp skippingerrors +sleepEachRow +SLES +SLRU +SMALLINT +Smirnov's +Smirnov'test +snowflakeToDateTime +socketcache +soundex +Soundex +SpanKind +sparkbar sparsehash +Spearman's +speedscope +splitByChar +splitByNonAlpha +splitByRegexp +splitByString +splitByWhitespace +SPNEGO +SQEs sql +sqlalchemy +SQLAlchemy +SQLConsoleDetail sqlinsert +SQLInsert +sqlite +SQLSTATE +sqrt src +SSDCache +SSDComplexKeyCache +SSDs +SSLManager +SSRF +SSSE +stacktrace stacktraces +startsWith +StartTime +StartTLS +StartupSystemTables +StartupSystemTablesThreads +StartupSystemTablesThreadsActive statbox stateful +Stateful +stddev +stddevpop +stddevPop +stddevsamp +stddevSamp stderr stdin stdout +stochastically +stochasticlinearregression +stochasticLinearRegression +stochasticlogisticregression +stochasticLogisticRegression +StorageBufferBytes +StorageBufferRows +StorageDistributed +StorageDistributedThreads +StorageDistributedThreadsActive +StorageHive +StorageHiveThreads +StorageHiveThreadsActive +StorageODBC +storages +StorageS +storig +stringToH +StringToNum +StringToNumOrDefault +StringToNumOrNull +stripelog +Stripelog +StripeLog +Strohmeier strtod strtoll strtoull +struct structs +studentttest +studentTTest +subarray +subarrays +subBitmap +subcolumn +subcolumns +Subcolumns +subcribe subdirectories +subdirectory +subexpression +Subexpression subexpressions +subfolder +subinterval +subintervals +subkey +submatch submodule submodules +Submodules +subnet +subnetwork subpattern subpatterns subqueries +Subqueries subquery +subranges +subreddits subseconds +substracted substring +substrings +Substrings +substringUTF +subtitiles +subtractDays +subtractHours +subtractMinutes +subtractMonths +subtractQuarters +subtractSeconds +subtractWeeks +subtractYears subtree subtype sudo +sumcount +sumCount +sumkahan +sumKahan +summap +sumMap +sumMapFiltered +summingmergetree +SummingMergeTree +sumwithoverflow +sumWithOverflow +superaggregates +Superset +SuperSet +SupersetDocker +supremum symlink symlinks +synchronious syntaxes +syscall +syscalls +syslog +syslogd systemd +SystemReplicasThreads +SystemReplicasThreadsActive +TabItem +tabix +Tabix +TablesLoaderThreads +TablesLoaderThreadsActive +TablesToDropQueueSize +tablum +TABLUM tabseparated +TabSeparated tabseparatedraw +TabSeparatedRaw tabseparatedrawwithnames +TabSeparatedRawWithNames tabseparatedrawwithnamesandtypes +TabSeparatedRawWithNamesAndTypes tabseparatedwithnames +TabSeparatedWithNames tabseparatedwithnamesandtypes +TabSeparatedWithNamesAndTypes +TargetSpecific tcp +TCPConnection +tcpnodelay +tcpPort +TCPThreads +Telegraf templateignorespaces +TemplateIgnoreSpaces +TemporaryFilesForAggregation +TemporaryFilesForJoin +TemporaryFilesForSort +TemporaryFilesUnknown +Testflows +tgamma tgz +Tgz th +thats +Theil's +theilsu +theilsU +themself +threadpool +ThreadPoolFSReaderThreads +ThreadPoolFSReaderThreadsActive +ThreadPoolRemoteFSReaderThreads +ThreadPoolRemoteFSReaderThreadsActive +ThreadsActive +ThreadsInOvercommitTracker +throwIf +timeSlot +timeSlots +Timeunit +timeZone +timeZoneOf +timeZoneOffset +timezones +TINYINT +tinylog +TinyLog +Tkachenko +TKSV +TLSv tmp +ToCenterChild +ToChildren +toColumnTypeName +toDate +toDateOrDefault +toDateOrNull +toDateOrZero +toDateTime +toDateTimeOrDefault +toDateTimeOrNull +toDateTimeOrZero +toDayOfMonth +toDayOfWeek +toDayOfYear +toDecimal +toDecimalString +toFixedString +toFloat +ToGeo +ToGeoBoundary +toHour +toInt +toInterval +toIPv +ToIPv +toISOWeek +toISOYear +toJSONString +tokenbf tokenization +tokenized +tokenizer +toLastDayOfMonth +toLowCardinality +toMinute toml +toModifiedJulianDay +toModifiedJulianDayOrNull +toMonday +toMonth +toNullable toolchain toolset +Toolset +ToParent +topk +topK +TopK +topkweighted +topKWeighted +topLevelDomain +toQuarter +toRelativeDayNum +toRelativeHourNum +toRelativeMinuteNum +toRelativeMonthNum +toRelativeQuarterNum +toRelativeSecondNum +toRelativeWeekNum +toRelativeYearNum +toSecond +ToSnowflake +toStartOfDay +toStartOfFifteenMinutes +toStartOfFiveMinutes +toStartOfHour +toStartOfInterval +toStartOfISOYear +toStartOfMinute +toStartOfMonth +toStartOfQuarter +toStartOfSecond +toStartOfTenMinutes +toStartOfWeek +toStartOfYear +toString +ToString +toStringCutToZero +TotalBytesOfMergeTreeTables +TotalPartsOfMergeTreeTables +TotalRowsOfMergeTreeTables +TotalTemporaryFiles +toTime +toTimeZone +toType +toTypeName +toUInt +toUnixTimestamp +toUUID +toUUIDOrDefault +toUUIDOrNull +toUUIDOrZero +toValidUTF +toWeek +toYear +toYearWeek +toYYYYMM +toYYYYMMDD +toYYYYMMDDhhmmss +TPCH +Tradeoff transactional +Transactional transactionally +translateUTF +translocality +trie +trimBoth +trimLeft +trimRight +trunc +tryBase +tryDecrypt +TSan +TSDB tskv tsv +TSVRaw +TSVs +TThe tui +tumbleEnd +tumbleStart +tupleDivide +tupleDivideByNumber +tupleElement +tupleHammingDistance +tupleMinus +tupleMultiply +tupleMultiplyByNumber +tupleNegate +tuplePlus +tupleToNameValuePairs turbostat +TwoColumnList txt +typename +Uber +UBSan ubuntu +UDFs uint +Uint +UInt +UIntN +ulid +ULID +ULIDStringToDateTime +UMTS unary +unbin +uncomment +UncompressedCacheBytes +UncompressedCacheCells +underying +undrop +UNDROP +unencoded unencrypted +unescaped +unescaping +unhex +unicode +unidimensional +UnidirectionalEdgeIsValid +uniq +uniqcombined +uniqCombined +uniqexact +uniqExact +uniqhll +uniqHLL +uniqtheta +uniqTheta +uniqThetaIntersect +uniqThetaNot +uniqthetasketch +uniqThetaSketch +UniqThetaSketch +uniqThetaUnion +uniqUpTo +unix unixodbc +unixODBC unoptimized +unparsed +unrealiable +unreplicated +unresolvable +unrounded +untracked +untrusted +untuple +Updatable +UPDATEs +uploaders +Uppercased +upperUTF +uptime +Uptime +uptrace +Uptrace +uring +URIs url +urlCluster +URLHash +URLHierarchy +URLPathHierarchy +urls +URL's +UserID userspace userver +Util utils uuid +UUid +UUIDNumToString +UUIDs +UUIDStringToNum +Vadim +Valgrind +VARCHAR variadic varint +varpop +varPop +varsamp +varSamp vectorized +Vectorized vectorscan +verificationDepth +verificationMode +versionedcollapsingmergetree +VersionedCollapsingMergeTree +VersionInteger +vhost +VideoContainer +ViewAllLink +VIEWs +VirtualBox +virtualized +visibleWidth +visitParam +visitParamExtractBool +visitParamExtractFloat +visitParamExtractInt +visitParamExtractRaw +visitParamExtractString +visitParamExtractUInt +visitParamHas +WALs wchc wchs webpage webserver +weekyear +Welch's +welchttest +welchTTest +Werror +Wether wget +which's whitespace whitespaces +wikistat +WikiStat +windowFunnel +WindowView +WithNames +WithNamesAndTypes +Woboq +WordNet +wordshingleMinHash +wordShingleMinHash +wordShingleMinHashArg +wordShingleMinHashArgCaseInsensitive +wordShingleMinHashArgCaseInsensitiveUTF +wordShingleMinHashArgUTF +wordShingleMinHashCaseInsensitive +wordShingleMinHashCaseInsensitiveUTF +wordShingleMinHashUTF +wordShingleSimHash +wordShingleSimHashCaseInsensitive +wordShingleSimHashCaseInsensitiveUTF +wordShingleSimHashUTF +WriteBuffer +WriteBuffers wrt xcode +XCode +Xeon +xeus +XHTML +xkcd +xlarge xml +XORs +xxHash xz -zLib -zLinux +YAML +YAMLRegExpTree +yandex +Yandex +Yasm +youtube +YYYY +zabbix +Zabbix +Zipkin zkcopy zlib +zLib +zLinux +znode znodes +ZooKeeper +ZooKeeperRequest +ZooKeeper's +ZooKeepers +ZooKeeperSession +zookeeperSessionUptime +ZooKeeperWatch zstd diff --git a/utils/check-style/check-doc-aspell b/utils/check-style/check-doc-aspell index d39769aa930..952dbd5b507 100755 --- a/utils/check-style/check-doc-aspell +++ b/utils/check-style/check-doc-aspell @@ -1,5 +1,8 @@ #!/usr/bin/env bash +# force-enable double star globbing +shopt -s globstar + # Perform spell checking on the docs if [[ ${1:-} == "--help" ]] || [[ ${1:-} == "-h" ]]; then From c3a888a47bfd68c3737dee55fe1a0a2c92fa9341 Mon Sep 17 00:00:00 2001 From: Roman Vlasenko Date: Fri, 2 Jun 2023 15:41:43 +0300 Subject: [PATCH 21/26] Remove needless minus sign --- docs/ru/faq/integration/json-import.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/faq/integration/json-import.md b/docs/ru/faq/integration/json-import.md index bc65b5a614a..a3c89aed429 100644 --- a/docs/ru/faq/integration/json-import.md +++ b/docs/ru/faq/integration/json-import.md @@ -19,7 +19,7 @@ $ echo '{"foo":"bar"}' | curl 'http://localhost:8123/?query=INSERT%20INTO%20test При помощи [интефейса CLI](../../interfaces/cli.md): ``` bash -$ echo '{"foo":"bar"}' | clickhouse-client ---query="INSERT INTO test FORMAT JSONEachRow" +$ echo '{"foo":"bar"}' | clickhouse-client --query="INSERT INTO test FORMAT JSONEachRow" ``` Чтобы не вставлять данные вручную, используйте одну из [готовых библиотек](../../interfaces/index.md). @@ -31,4 +31,4 @@ $ echo '{"foo":"bar"}' | clickhouse-client ---query="INSERT INTO test FORMAT JS :::note "Примечание" В HTTP-интерфейсе настройки передаются через параметры `GET` запроса, в `CLI` interface — как дополнительные аргументы командной строки, начинающиеся с `--`. - ::: \ No newline at end of file + ::: From 079008058adbefea63edc8afffa0dd20d694e5e5 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Fri, 2 Jun 2023 08:49:26 -0400 Subject: [PATCH 22/26] move from server to user settings --- .../settings.md | 51 ++----------------- docs/en/operations/settings/settings.md | 42 +++++++++++++++ 2 files changed, 46 insertions(+), 47 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 82be1c10dcc..d07fb80d1da 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -275,9 +275,9 @@ Type: UInt64 Default: 1000 -## max_concurrent_insert_queries +## max_concurrent_queries -Limit on total number of concurrent insert queries. Zero means Unlimited. +Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. :::note These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -287,9 +287,9 @@ Type: UInt64 Default: 0 -## max_concurrent_queries +## max_concurrent_insert_queries -Limit on total number of concurrently executed queries. Zero means Unlimited. Note that limits on insert and select queries, and on the maximum number of queries for users must also be considered. See also max_concurrent_insert_queries, max_concurrent_select_queries, max_concurrent_queries_for_all_users. Zero means unlimited. +Limit on total number of concurrent insert queries. Zero means Unlimited. :::note These settings can be modified at runtime and will take effect immediately. Queries that are already running will remain unchanged. @@ -1277,49 +1277,6 @@ For more information, see the section [Creating replicated tables](../../engines ``` - -## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} - -The maximum number of simultaneously processed queries related to MergeTree table per user. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -5 -``` - -## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} - -Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. - -Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. - -Modifying the setting for one query or user does not affect other queries. - -Possible values: - -- Positive integer. -- 0 — No limit. - -Default value: `0`. - -**Example** - -``` xml -99 -``` - -**See Also** - -- [max_concurrent_queries](#max-concurrent-queries) - ## max_open_files {#max-open-files} The maximum number of open files. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index b868299aeff..374afb6bed7 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -646,6 +646,48 @@ Used for the same purpose as `max_block_size`, but it sets the recommended block However, the block size cannot be more than `max_block_size` rows. By default: 1,000,000. It only works when reading from MergeTree engines. +## max_concurrent_queries_for_user {#max-concurrent-queries-for-user} + +The maximum number of simultaneously processed queries related to MergeTree table per user. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + +**Example** + +``` xml +5 +``` + +## max_concurrent_queries_for_all_users {#max-concurrent-queries-for-all-users} + +Throw exception if the value of this setting is less or equal than the current number of simultaneously processed queries. + +Example: `max_concurrent_queries_for_all_users` can be set to 99 for all users and database administrator can set it to 100 for itself to run queries for investigation even when the server is overloaded. + +Modifying the setting for one query or user does not affect other queries. + +Possible values: + +- Positive integer. +- 0 — No limit. + +Default value: `0`. + +**Example** + +``` xml +99 +``` + +**See Also** + +- [max_concurrent_queries](/docs/en/operations/server-configuration-parameters/settings.md/#max_concurrent_queries) + ## merge_tree_min_rows_for_concurrent_read {#setting-merge-tree-min-rows-for-concurrent-read} If the number of rows to be read from a file of a [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table exceeds `merge_tree_min_rows_for_concurrent_read` then ClickHouse tries to perform a concurrent reading from this file on several threads. From e148c60d5a308b2ea86128021beba0e8321fef9b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 16:18:32 +0300 Subject: [PATCH 23/26] Fixes for MergeTree with readonly disks (#50244) * fixes for MergeTree with readonly disks * Automatic style fix * Update test.py * Automatic style fix * Update test.py * Update test.py * Automatic style fix * Update test.py --------- Co-authored-by: robot-clickhouse Co-authored-by: alesapin --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++ src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageMergeTree.cpp | 5 +- .../test_disk_over_web_server/test.py | 50 ++++++++++++++++--- ...02435_rollback_cancelled_queries.reference | 1 - .../02435_rollback_cancelled_queries.sh | 11 ++-- 6 files changed, 61 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 338a221e45e..32665429051 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4854,6 +4854,9 @@ void MergeTreeData::checkAlterPartitionIsPossible( void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, ContextPtr local_context) { + if (!supportsReplication() && isStaticStorage()) + return; + DataPartsVector parts_to_remove; const auto * partition_ast = partition->as(); if (partition_ast && partition_ast->all) @@ -4874,6 +4877,9 @@ void MergeTreeData::checkPartitionCanBeDropped(const ASTPtr & partition, Context void MergeTreeData::checkPartCanBeDropped(const String & part_name) { + if (!supportsReplication() && isStaticStorage()) + return; + auto part = getPartIfExists(part_name, {MergeTreeDataPartState::Active}); if (!part) throw Exception(ErrorCodes::NO_SUCH_DATA_PART, "No part {} in committed state", part_name); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index fce7d989a2f..1c41de6fa19 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -865,7 +865,7 @@ public: DiskPtr tryGetDiskForDetachedPart(const String & part_name) const; DiskPtr getDiskForDetachedPart(const String & part_name) const; - bool storesDataOnDisk() const override { return true; } + bool storesDataOnDisk() const override { return !isStaticStorage(); } Strings getDataPaths() const override; /// Reserves space at least 1MB. diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index a721dd30cd7..3da4724471d 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -114,7 +114,7 @@ StorageMergeTree::StorageMergeTree( loadDataParts(has_force_restore_data_flag); - if (!attach && !getDataPartsForInternalUsage().empty()) + if (!attach && !getDataPartsForInternalUsage().empty() && !isStaticStorage()) throw Exception(ErrorCodes::INCORRECT_DATA, "Data directory for table already containing data parts - probably " "it was unclean DROP table or manual intervention. " @@ -283,6 +283,9 @@ StorageMergeTree::write(const ASTPtr & /*query*/, const StorageMetadataPtr & met void StorageMergeTree::checkTableCanBeDropped() const { + if (!supportsReplication() && isStaticStorage()) + return; + auto table_id = getStorageID(); getContext()->checkTableCanBeDropped(table_id.database_name, table_id.table_name, getTotalActiveSizeInBytes()); } diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index fd71389f71a..719de5e8bef 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -10,16 +10,22 @@ def cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance( - "node1", main_configs=["configs/storage_conf.xml"], with_nginx=True + "node1", + main_configs=["configs/storage_conf.xml"], + with_nginx=True, ) cluster.add_instance( "node2", main_configs=["configs/storage_conf_web.xml"], with_nginx=True, stay_alive=True, + with_zookeeper=True, ) cluster.add_instance( - "node3", main_configs=["configs/storage_conf_web.xml"], with_nginx=True + "node3", + main_configs=["configs/storage_conf_web.xml"], + with_nginx=True, + with_zookeeper=True, ) cluster.add_instance( @@ -95,7 +101,7 @@ def test_usage(cluster, node_name): for i in range(3): node2.query( """ - ATTACH TABLE test{} UUID '{}' + CREATE TABLE test{} UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -140,7 +146,7 @@ def test_incorrect_usage(cluster): global uuids node2.query( """ - ATTACH TABLE test0 UUID '{}' + CREATE TABLE test0 UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -173,7 +179,7 @@ def test_cache(cluster, node_name): for i in range(3): node2.query( """ - ATTACH TABLE test{} UUID '{}' + CREATE TABLE test{} UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'cached_web'; """.format( @@ -238,7 +244,7 @@ def test_unavailable_server(cluster): global uuids node2.query( """ - ATTACH TABLE test0 UUID '{}' + CREATE TABLE test0 UUID '{}' (id Int32) ENGINE = MergeTree() ORDER BY id SETTINGS storage_policy = 'web'; """.format( @@ -276,3 +282,35 @@ def test_unavailable_server(cluster): ) node2.start_clickhouse() node2.query("DROP TABLE test0 SYNC") + + +def test_replicated_database(cluster): + node1 = cluster.instances["node3"] + node1.query( + "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r1')", + settings={"allow_experimental_database_replicated": 1}, + ) + + global uuids + node1.query( + """ + CREATE TABLE rdb.table0 UUID '{}' + (id Int32) ENGINE = MergeTree() ORDER BY id + SETTINGS storage_policy = 'web'; + """.format( + uuids[0] + ) + ) + + node2 = cluster.instances["node2"] + node2.query( + "CREATE DATABASE rdb ENGINE=Replicated('/test/rdb', 's1', 'r2')", + settings={"allow_experimental_database_replicated": 1}, + ) + node2.query("SYSTEM SYNC DATABASE REPLICA rdb") + + assert node1.query("SELECT count() FROM rdb.table0") == "5000000\n" + assert node2.query("SELECT count() FROM rdb.table0") == "5000000\n" + + node1.query("DROP DATABASE rdb SYNC") + node2.query("DROP DATABASE rdb SYNC") diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference index 2d32c17ec7c..38ff81b2371 100644 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.reference @@ -1,3 +1,2 @@ 1000000 0 -1 diff --git a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh index 8f8e8cc7ee0..776d1f850b0 100755 --- a/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh +++ b/tests/queries/0_stateless/02435_rollback_cancelled_queries.sh @@ -110,11 +110,12 @@ insert_data 1 $CLICKHOUSE_CLIENT --implicit_transaction=1 -q 'select throwIf(count() % 1000000 != 0 or count() = 0) from dedup_test' \ || $CLICKHOUSE_CLIENT -q "select name, rows, active, visible, creation_tid, creation_csn from system.parts where database=currentDatabase();" -# Ensure that thread_cancel actually did something -$CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( - message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', - 'Query was cancelled or a client has unexpectedly dropped the connection') or - message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" +# Ensure that thread_cancel actually did something (useful when editing this test) +# We cannot check it in the CI, because sometimes it fails due to randomization +# $CLICKHOUSE_CLIENT -q "select count() > 0 from system.text_log where event_date >= yesterday() and query_id like '$TEST_MARK%' and ( +# message_format_string in ('Unexpected end of file while reading chunk header of HTTP chunked data', 'Unexpected EOF, got {} of {} bytes', +# 'Query was cancelled or a client has unexpectedly dropped the connection') or +# message like '%Connection reset by peer%' or message like '%Broken pipe, while writing to socket%')" wait_for_queries_to_finish 30 $CLICKHOUSE_CLIENT --database_atomic_wait_for_drop_and_detach_synchronously=0 -q "drop table dedup_test" From 54872f9e7ea314da7f766c8929c212fd3d07ec21 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 2 Jun 2023 13:27:56 +0000 Subject: [PATCH 24/26] Typos: Follow-up to #50476 --- .../mergetree-family/mergetree.md | 2 +- .../settings.md | 2 +- .../operations/settings/settings-formats.md | 2 +- docs/en/operations/settings/settings.md | 8 ++-- docs/en/operations/system-tables/quotas.md | 4 +- .../parametric-functions.md | 2 +- .../reference/exponentialmovingaverage.md | 4 +- docs/en/sql-reference/dictionaries/index.md | 6 +-- .../functions/array-functions.md | 4 +- .../functions/distance-functions.md | 2 +- .../functions/encryption-functions.md | 2 +- .../sql-reference/functions/hash-functions.md | 2 +- .../sql-reference/functions/math-functions.md | 2 +- .../functions/other-functions.md | 4 +- .../functions/type-conversion-functions.md | 2 +- docs/en/sql-reference/operators/index.md | 2 +- .../table-functions/urlCluster.md | 4 +- .../aspell-ignore/en/aspell-dict.txt | 45 ------------------- 18 files changed, 27 insertions(+), 72 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 1ab0f4057ff..07f706af91d 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -779,7 +779,7 @@ Disks, volumes and storage policies should be declared inside the ` Date: Fri, 2 Jun 2023 14:00:57 +0000 Subject: [PATCH 25/26] Fix typos --- docs/en/sql-reference/functions/other-functions.md | 6 +----- docs/en/sql-reference/functions/random-functions.md | 2 +- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md index 6d49527fd0d..254297451f7 100644 --- a/docs/en/sql-reference/functions/other-functions.md +++ b/docs/en/sql-reference/functions/other-functions.md @@ -2006,11 +2006,7 @@ isDecimalOverflow(d, [p]) **Arguments** - `d` — value. [Decimal](../../sql-reference/data-types/decimal.md). -<<<<<<< HEAD -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This paratemer can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). -======= -- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. Using of this parameter could be helpful for data extraction to another DBMS or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). ->>>>>>> rschu1ze/master +- `p` — precision. Optional. If omitted, the initial precision of the first argument is used. This parameter can be helpful to migrate data from/to another database or file. [UInt8](../../sql-reference/data-types/int-uint.md#uint-ranges). **Returned values** diff --git a/docs/en/sql-reference/functions/random-functions.md b/docs/en/sql-reference/functions/random-functions.md index 63d5174b494..e593d9458f0 100644 --- a/docs/en/sql-reference/functions/random-functions.md +++ b/docs/en/sql-reference/functions/random-functions.md @@ -67,7 +67,7 @@ randUniform(min, max) **Arguments** - `min` - `Float64` - left boundary of the range, -- `max` - `Float64` - reight boundary of the range. +- `max` - `Float64` - right boundary of the range. **Returned value** From 963641b20ffd09c01107fe8b210d38218ca85161 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 2 Jun 2023 18:24:56 +0300 Subject: [PATCH 26/26] disable 00534_functions_bad_arguments with msan (#50481) --- tests/queries/0_stateless/00534_functions_bad_arguments10.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments5.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments6.sh | 2 +- tests/queries/0_stateless/00534_functions_bad_arguments9.sh | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments10.sh b/tests/queries/0_stateless/00534_functions_bad_arguments10.sh index 8525b63a989..b9733f92812 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments10.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments10.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments5.sh b/tests/queries/0_stateless/00534_functions_bad_arguments5.sh index a8b0ce77677..812ba9f97fa 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments5.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments5.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug, no-fasttest +# Tags: no-tsan, no-debug, no-fasttest, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments6.sh b/tests/queries/0_stateless/00534_functions_bad_arguments6.sh index b0080c3b418..6626a6dfe55 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments6.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments6.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016 diff --git a/tests/queries/0_stateless/00534_functions_bad_arguments9.sh b/tests/queries/0_stateless/00534_functions_bad_arguments9.sh index 2975643020b..c7659db8621 100755 --- a/tests/queries/0_stateless/00534_functions_bad_arguments9.sh +++ b/tests/queries/0_stateless/00534_functions_bad_arguments9.sh @@ -1,5 +1,5 @@ #!/usr/bin/env bash -# Tags: no-tsan, no-debug +# Tags: no-tsan, no-debug, no-msan # Tag no-tsan: Too long for TSan # shellcheck disable=SC2016