From 97b5179e55496b42f93db35cc6b8957be5a6b0bf Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 19 Jan 2021 22:21:06 +0300 Subject: [PATCH 001/510] Implement HedgedRequests --- src/Client/Connection.cpp | 115 ++-- src/Client/Connection.h | 17 +- src/Client/ConnectionPoolWithFailover.cpp | 235 +++++++- src/Client/ConnectionPoolWithFailover.h | 61 +++ src/Client/GetHedgedConnections.cpp | 512 ++++++++++++++++++ src/Client/GetHedgedConnections.h | 154 ++++++ src/Client/HedgedConnections.cpp | 389 +++++++++++++ src/Client/HedgedConnections.h | 93 ++++ src/Client/IConnections.h | 57 ++ src/Client/MultiplexedConnections.cpp | 2 +- src/Client/MultiplexedConnections.h | 39 +- src/Client/ya.make | 2 + src/Common/Epoll.cpp | 82 +++ src/Common/Epoll.h | 44 ++ src/Common/ErrorCodes.cpp | 1 + src/Common/PoolWithFailoverBase.h | 94 ++-- src/Common/TimerDescriptor.cpp | 2 +- src/Common/TimerDescriptor.h | 16 +- src/Common/ya.make | 1 + src/Core/Defines.h | 4 + src/Core/Settings.h | 9 + src/DataStreams/RemoteQueryExecutor.cpp | 77 +-- src/DataStreams/RemoteQueryExecutor.h | 9 +- .../RemoteQueryExecutorReadContext.cpp | 71 +-- .../RemoteQueryExecutorReadContext.h | 19 +- src/IO/ConnectionTimeouts.h | 45 +- src/IO/ConnectionTimeoutsContext.h | 11 +- src/IO/ReadBufferFromPocoSocket.cpp | 2 +- src/IO/ReadBufferFromPocoSocket.h | 6 +- src/Processors/Executors/PollingQueue.cpp | 35 +- src/Processors/Executors/PollingQueue.h | 3 +- src/Server/TCPHandler.cpp | 25 + .../configs/remote_servers.xml | 18 + .../test_hedged_requests/configs/users.xml | 10 + .../test_hedged_requests/configs/users1.xml | 7 + .../integration/test_hedged_requests/test.py | 76 +++ 36 files changed, 2054 insertions(+), 289 deletions(-) create mode 100644 src/Client/GetHedgedConnections.cpp create mode 100644 src/Client/GetHedgedConnections.h create mode 100644 src/Client/HedgedConnections.cpp create mode 100644 src/Client/HedgedConnections.h create mode 100644 src/Client/IConnections.h create mode 100644 src/Common/Epoll.cpp create mode 100644 src/Common/Epoll.h create mode 100644 tests/integration/test_hedged_requests/configs/remote_servers.xml create mode 100644 tests/integration/test_hedged_requests/configs/users.xml create mode 100644 tests/integration/test_hedged_requests/configs/users1.xml create mode 100644 tests/integration/test_hedged_requests/test.py diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index ef114490c51..15f530f4085 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -64,53 +64,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) if (connected) disconnect(); - LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", - default_database.empty() ? "(not specified)" : default_database, - user, - static_cast(secure) ? ". Secure" : "", - static_cast(compression) ? "" : ". Uncompressed"); - - if (static_cast(secure)) - { -#if USE_SSL - socket = std::make_unique(); - - /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) - /// work we need to pass host name separately. It will be send into TLS Hello packet to let - /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). - static_cast(socket.get())->setPeerHostName(host); -#else - throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - else - { - socket = std::make_unique(); - } - - current_resolved_address = DNSResolver::instance().resolveAddress(host, port); - - const auto & connection_timeout = static_cast(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout; - socket->connect(*current_resolved_address, connection_timeout); - socket->setReceiveTimeout(timeouts.receive_timeout); - socket->setSendTimeout(timeouts.send_timeout); - socket->setNoDelay(true); - if (timeouts.tcp_keep_alive_timeout.totalSeconds()) - { - socket->setKeepAlive(true); - socket->setOption(IPPROTO_TCP, -#if defined(TCP_KEEPALIVE) - TCP_KEEPALIVE -#else - TCP_KEEPIDLE // __APPLE__ -#endif - , timeouts.tcp_keep_alive_timeout); - } - - in = std::make_shared(*socket); - out = std::make_shared(*socket); - - connected = true; + prepare(timeouts); sendHello(); receiveHello(); @@ -146,6 +100,57 @@ void Connection::disconnect() connected = false; } +void Connection::prepare(const ConnectionTimeouts & timeouts) +{ + LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", + default_database.empty() ? "(not specified)" : default_database, + user, + static_cast(secure) ? ". Secure" : "", + static_cast(compression) ? "" : ". Uncompressed"); + + if (static_cast(secure)) + { +#if USE_SSL + socket = std::make_unique(); + + /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) + /// work we need to pass host name separately. It will be send into TLS Hello packet to let + /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). + static_cast(socket.get())->setPeerHostName(host); +#else + throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + } + else + { + socket = std::make_unique(); + } + + current_resolved_address = DNSResolver::instance().resolveAddress(host, port); + + const auto & connection_timeout = static_cast(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout; + socket->connect(*current_resolved_address, connection_timeout); + socket->setReceiveTimeout(timeouts.receive_timeout); + socket->setSendTimeout(timeouts.send_timeout); + socket->setNoDelay(true); + if (timeouts.tcp_keep_alive_timeout.totalSeconds()) + { + socket->setKeepAlive(true); + socket->setOption(IPPROTO_TCP, +#if defined(TCP_KEEPALIVE) + TCP_KEEPALIVE +#else + TCP_KEEPIDLE // __APPLE__ +#endif + , timeouts.tcp_keep_alive_timeout); + } + + in = std::make_shared(*socket); + out = std::make_shared(*socket); + + connected = true; +} + void Connection::sendHello() { @@ -334,8 +339,6 @@ void Connection::sendClusterNameAndSalt() bool Connection::ping() { - // LOG_TRACE(log_wrapper.get(), "Ping"); - TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); try { @@ -379,10 +382,21 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); + sendTablesStatusRequest(request); + TablesStatusResponse response = receiveTablesStatusResponse(); + + return response; +} + +void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) +{ writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); out->next(); +} +TablesStatusResponse Connection::receiveTablesStatusResponse() +{ UInt64 response_type = 0; readVarUInt(response_type, *in); @@ -396,7 +410,6 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time return response; } - void Connection::sendQuery( const ConnectionTimeouts & timeouts, const String & query, @@ -742,7 +755,7 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) } -Packet Connection::receivePacket(std::function async_callback) +Packet Connection::receivePacket(AsyncCallback async_callback) { in->setAsyncCallback(std::move(async_callback)); SCOPE_EXIT(in->setAsyncCallback({})); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 83e8f3ba206..7c96634c21a 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -173,7 +173,7 @@ public: /// Receive packet from server. /// Each time read blocks and async_callback is set, it will be called. You can poll socket inside it. - Packet receivePacket(std::function async_callback = {}); + Packet receivePacket(AsyncCallback async_callback = {}); /// If not connected yet, or if connection is broken - then connect. If cannot connect - throw an exception. void forceConnected(const ConnectionTimeouts & timeouts); @@ -192,6 +192,19 @@ public: size_t outBytesCount() const { return out ? out->count() : 0; } size_t inBytesCount() const { return in ? in->count() : 0; } + /// Make preparation before sending Hello in connect + void prepare(const ConnectionTimeouts & timeouts); + + void sendHello(); + + void receiveHello(); + + void sendTablesStatusRequest(const TablesStatusRequest & request); + + TablesStatusResponse receiveTablesStatusResponse(); + + Poco::Net::Socket * getSocket() { return socket.get(); } + private: String host; UInt16 port; @@ -280,8 +293,6 @@ private: LoggerWrapper log_wrapper; void connect(const ConnectionTimeouts & timeouts); - void sendHello(); - void receiveHello(); #if USE_SSL void sendClusterNameAndSalt(); diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 1ca61dc8059..df4541ecf7e 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -132,6 +132,8 @@ std::vector ConnectionPoolWithFailover::getMany(const Co const Settings * settings, PoolMode pool_mode) { + LOG_DEBUG(log, "ConnectionPoolWithFailover getMany"); + TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) { return tryGetEntry(pool, timeouts, fail_message, settings); @@ -164,6 +166,9 @@ std::vector ConnectionPoolWithFailover::g const Settings * settings, PoolMode pool_mode, const QualifiedTableName & table_to_check) { + + LOG_DEBUG(log, "ConnectionPoolWithFailover getManyChecked"); + TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) { return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check); @@ -172,11 +177,49 @@ std::vector ConnectionPoolWithFailover::g return getManyImpl(settings, pool_mode, try_get_entry); } +ConnectionPoolWithFailover::Base::GetPriorityFunc ConnectionPoolWithFailover::makeGetPriorityFunc(const Settings * settings) +{ + size_t offset = 0; + if (settings) + offset = settings->load_balancing_first_offset % nested_pools.size(); + + GetPriorityFunc get_priority; + switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing) + { + case LoadBalancing::NEAREST_HOSTNAME: + get_priority = [&](size_t i) { return hostname_differences[i]; }; + break; + case LoadBalancing::IN_ORDER: + get_priority = [](size_t i) { return i; }; + break; + case LoadBalancing::RANDOM: + break; + case LoadBalancing::FIRST_OR_RANDOM: + get_priority = [offset](size_t i) -> size_t { return i != offset; }; + break; + case LoadBalancing::ROUND_ROBIN: + if (last_used >= nested_pools.size()) + last_used = 0; + ++last_used; + /* Consider nested_pools.size() equals to 5 + * last_used = 1 -> get_priority: 0 1 2 3 4 + * last_used = 2 -> get_priority: 5 0 1 2 3 + * last_used = 3 -> get_priority: 5 4 0 1 2 + * ... + * */ + get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; }; + break; + } + + return get_priority; +} + std::vector ConnectionPoolWithFailover::getManyImpl( const Settings * settings, PoolMode pool_mode, const TryGetEntryFunc & try_get_entry) { + LOG_DEBUG(log, "ConnectionPoolWithFailover getManyImpl"); size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; size_t max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : @@ -194,36 +237,7 @@ std::vector ConnectionPoolWithFailover::g else throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR); - size_t offset = 0; - if (settings) - offset = settings->load_balancing_first_offset % nested_pools.size(); - GetPriorityFunc get_priority; - switch (settings ? LoadBalancing(settings->load_balancing) : default_load_balancing) - { - case LoadBalancing::NEAREST_HOSTNAME: - get_priority = [&](size_t i) { return hostname_differences[i]; }; - break; - case LoadBalancing::IN_ORDER: - get_priority = [](size_t i) { return i; }; - break; - case LoadBalancing::RANDOM: - break; - case LoadBalancing::FIRST_OR_RANDOM: - get_priority = [offset](size_t i) -> size_t { return i != offset; }; - break; - case LoadBalancing::ROUND_ROBIN: - if (last_used >= nested_pools.size()) - last_used = 0; - ++last_used; - /* Consider nested_pools.size() equals to 5 - * last_used = 1 -> get_priority: 0 1 2 3 4 - * last_used = 2 -> get_priority: 5 0 1 2 3 - * last_used = 3 -> get_priority: 5 4 0 1 2 - * ... - * */ - get_priority = [&](size_t i) { ++i; return i < last_used ? nested_pools.size() - i : i - last_used; }; - break; - } + GetPriorityFunc get_priority = makeGetPriorityFunc(settings); UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true; @@ -244,8 +258,11 @@ ConnectionPoolWithFailover::tryGetEntry( TryResult result; try { + LOG_DEBUG(log, "ConnectionPoolWithFailover tryGetEntry"); result.entry = pool.get(timeouts, settings, /* force_connected = */ false); + LOG_DEBUG(log, "ConnectionPoolWithFailover isConnected {}", result.entry->isConnected()); + UInt64 server_revision = 0; if (table_to_check) server_revision = result.entry->getServerRevision(timeouts); @@ -314,4 +331,162 @@ ConnectionPoolWithFailover::tryGetEntry( return result; } +std::vector ConnectionPoolWithFailover::getShuffledPools(const Settings * settings) +{ + GetPriorityFunc get_priority = makeGetPriorityFunc(settings); + UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0; + return Base::getShuffledPools(max_ignored_errors, get_priority); +} + +TryGetConnection::TryGetConnection( + IConnectionPool * pool_, + const ConnectionTimeouts * timeouts_, + const Settings * settings_, + std::shared_ptr table_to_check_, + Poco::Logger * log_) : + pool(pool_), timeouts(timeouts_), settings(settings_), + table_to_check(table_to_check_), log(log_), stage(Stage::CONNECT), socket_fd(-1) +{ +} + +void TryGetConnection::reset() +{ + resetResult(); + stage = Stage::CONNECT; + epoll = nullptr; + socket_fd = -1; + fail_message.clear(); +} + +void TryGetConnection::resetResult() +{ + if (!result.entry.isNull()) + { + result.entry->disconnect(); + result.reset(); + } +} + +void TryGetConnection::processFail(bool add_description) +{ + if (epoll) + epoll->remove(socket_fd); + + fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); + if (add_description) + fail_message += " (" + result.entry->getDescription() + ")"; + resetResult(); + socket_fd = -1; + stage = Stage::FAILED; +} + +void TryGetConnection::run() +{ + try + { + if (stage == Stage::CONNECT) + { + result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); + + if (!result.entry->isConnected()) + { + result.entry->prepare(*timeouts); + socket_fd = result.entry->getSocket()->impl()->sockfd(); + result.entry->sendHello(); + stage = Stage::RECEIVE_HELLO; + /// We are waiting for hello from replica. + return; + } + + socket_fd = result.entry->getSocket()->impl()->sockfd(); + stage = Stage::START_CHECK_TABLE; + } + + if (stage == Stage::RECEIVE_HELLO) + { + result.entry->receiveHello(); + stage = Stage::START_CHECK_TABLE; + } + + if (stage == Stage::START_CHECK_TABLE) + { + UInt64 server_revision = 0; + if (table_to_check) + server_revision = result.entry->getServerRevision(*timeouts); + + if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) + { + result.entry->forceConnected(*timeouts); + result.is_usable = true; + result.is_up_to_date = true; + stage = FINISHED; + return; + } + + TablesStatusRequest status_request; + status_request.tables.emplace(*table_to_check); + + result.entry->sendTablesStatusRequest(status_request); + stage = Stage::RECEIVE_TABLES_STATUS; + /// We are waiting for tables status response. + return; + } + + if (stage == Stage::RECEIVE_TABLES_STATUS) + { + TablesStatusResponse status_response = result.entry->receiveTablesStatusResponse(); + auto table_status_it = status_response.table_states_by_id.find(*table_to_check); + if (table_status_it == status_response.table_states_by_id.end()) + { + const char * message_pattern = "There is no table {}.{} on server: {}"; + fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); + LOG_WARNING(log, fail_message); + ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); + stage = Stage::FINISHED; + return; + } + + result.is_usable = true; + + UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; + if (!max_allowed_delay) + { + result.is_up_to_date = true; + stage = Stage::FINISHED; + return; + } + + UInt32 delay = table_status_it->second.absolute_delay; + + if (delay < max_allowed_delay) + result.is_up_to_date = true; + else + { + result.is_up_to_date = false; + result.staleness = delay; + + LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); + ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); + } + } + + stage = Stage::FINISHED; + } + catch (Poco::Net::NetException & e) + { + processFail(true); + } + catch (Poco::TimeoutException & e) + { + processFail(true); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throw; + + processFail(false); + } +} + } diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 7d5f713f6a9..c57a7bb984a 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -31,6 +32,55 @@ enum class PoolMode GET_ALL }; +/// Class for establishing connection with replica without blocking. +class TryGetConnection +{ +public: + enum Stage + { + CONNECT = 0, + RECEIVE_HELLO = 1, + START_CHECK_TABLE = 2, + RECEIVE_TABLES_STATUS = 3, + FINISHED = 4, + FAILED = 5, + }; + + using TryResult = PoolWithFailoverBase::TryResult; + + TryGetConnection(IConnectionPool * pool_, + const ConnectionTimeouts * timeouts_, + const Settings * settings_, + std::shared_ptr table_to_check = nullptr, + Poco::Logger * log_ = nullptr); + + /// Continue connecting to replica from previous stage. Initial stage is CONNECT. + void run(); + + void resetResult(); + + /// Reset class to initial stage. + void reset(); + + /// If connection is failed and epoll is set, before disconnecting + /// socket will be removed from epoll. + void setEpoll(Epoll * epoll_) { epoll = epoll_; } + + /// Process fail connection. + void processFail(bool add_description = false); + + IConnectionPool * pool; + const ConnectionTimeouts * timeouts; + std::string fail_message; + const Settings * settings; + std::shared_ptr table_to_check; + Poco::Logger * log; + TryResult result; + Stage stage; + int socket_fd; + Epoll * epoll = nullptr; +}; + class ConnectionPoolWithFailover : public IConnectionPool, private PoolWithFailoverBase { public: @@ -80,6 +130,15 @@ public: using Status = std::vector; Status getStatus() const; + std::vector getShuffledPools(const Settings * settings); + + size_t getMaxErrorCup() const { return Base::max_error_cap; } + + void updateSharedError(std::vector & shuffled_pools) + { + Base::updateSharedErrorCounts(shuffled_pools); + } + private: /// Get the values of relevant settings and call Base::getMany() std::vector getManyImpl( @@ -97,6 +156,8 @@ private: const Settings * settings, const QualifiedTableName * table_to_check = nullptr); + GetPriorityFunc makeGetPriorityFunc(const Settings * settings); + private: std::vector hostname_differences; /// Distances from name of this host to the names of hosts of pools. size_t last_used = 0; /// Last used for round_robin policy. diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp new file mode 100644 index 00000000000..3b30650e6e5 --- /dev/null +++ b/src/Client/GetHedgedConnections.cpp @@ -0,0 +1,512 @@ +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int ALL_CONNECTION_TRIES_FAILED; +} + +GetHedgedConnections::GetHedgedConnections( + const ConnectionPoolWithFailoverPtr & pool_, + const Settings * settings_, + const ConnectionTimeouts & timeouts_, + std::shared_ptr table_to_check_) + : pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_) +{ + log = &Poco::Logger::get("GetHedgedConnections"); + shuffled_pools = pool->getShuffledPools(settings); + for (size_t i = 0; i != shuffled_pools.size(); ++i) + try_get_connections.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check, log); + + max_tries + = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); + + fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries : false; +} + +GetHedgedConnections::~GetHedgedConnections() +{ + pool->updateSharedError(shuffled_pools); +} + +GetHedgedConnections::Replicas GetHedgedConnections::getConnections() +{ + entries_count = 0; + usable_count = 0; + failed_pools_count = 0; + + ReplicaStatePtr replica = &first_replica; + int index = 0; + + while (index != -1 || epoll.size() != 0) + { + if (index != -1) + { + Action action = startTryGetConnection(index, replica); + if (action == Action::TRY_NEXT_REPLICA) + { + index = getNextIndex(index); + continue; + } + + if (action == Action::FINISH) + { + swapReplicasIfNeeded(); + return {&first_replica, &second_replica}; + } + } + + /// Process epoll events + replica = processEpollEvents(); + if (replica->isReady()) + { + swapReplicasIfNeeded(); + return {&first_replica, &second_replica}; + } + + index = getNextIndex(index); + } + + /// We reach this point only if there was no up to date replica + + if (usable_count == 0) + { + if (settings && settings->skip_unavailable_shards) + { + first_replica.state = State::CANNOT_CHOOSE; + second_replica.state = State::CANNOT_CHOOSE; + return {&first_replica, &second_replica}; + } + + throw NetException("All connection tries failed. Log: \n\n" + fail_messages + "\n", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + } + if (!fallback_to_stale_replicas) + throw DB::Exception("Could not find connection to up-to-date replica.", DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); + + setBestUsableReplica(first_replica); + return {&first_replica, &second_replica}; +} + +void GetHedgedConnections::chooseSecondReplica() +{ + LOG_DEBUG(log, "choose second replica"); + + if (second_replica.isCannotChoose() || second_replica.isReady()) + return; + + int index; + if (second_replica.isNotReady()) + index = second_replica.index; + else + index = first_replica.index; + + while (true) + { + if (second_replica.isEmpty()) + { + + index = getNextIndex(index); + if (index == -1) + break; + + Action action = startTryGetConnection(index, &second_replica); + + if (action == Action::TRY_NEXT_REPLICA) + continue; + + /// Second replica is ready or we are waiting for response from it + return; + } + + if (!second_replica.isNotReady()) + throw Exception("Second replica state must be 'NOT_READY' before process epoll events", ErrorCodes::LOGICAL_ERROR); + + ReplicaStatePtr replica = processEpollEvents( true); + + if (replica != &second_replica) + throw Exception("Epoll could return only second replica here", ErrorCodes::LOGICAL_ERROR); + + /// If replica is not empty than it is ready or we are waiting for a response from it + if (!second_replica.isEmpty()) + return; + } + + /// There is no up to date replica + + LOG_DEBUG(log, "there is no up to date replica for second replica"); + + if (!fallback_to_stale_replicas || usable_count <= 1) + second_replica.state = State::CANNOT_CHOOSE; + else + setBestUsableReplica(second_replica, first_replica.index); +} + +void GetHedgedConnections::stopChoosingSecondReplica() +{ + LOG_DEBUG(log, "stop choosing second replica"); + + if (!second_replica.isNotReady()) + throw Exception("Can't stop choosing second replica, because it's not in process of choosing", ErrorCodes::LOGICAL_ERROR); + + removeTimeoutsFromReplica(&second_replica, epoll); + epoll.remove(second_replica.fd); + + try_get_connections[second_replica.index].reset(); + second_replica.reset(); +} + +int GetHedgedConnections::getNextIndex(int cur_index) +{ + /// Check if there is no more available replicas + if (cur_index == -1 || entries_count + failed_pools_count >= shuffled_pools.size()) + return -1; + + /// We can work with two replicas simultaneously and they must have different indexes + int skip_index = -1; + if (!first_replica.isEmpty()) + skip_index = first_replica.index; + else if (!second_replica.isEmpty()) + skip_index = second_replica.index; + + bool finish = false; + int next_index = cur_index; + while (!finish) + { + next_index = (next_index + 1) % shuffled_pools.size(); + + /// Check if we can try this replica + if (next_index != skip_index && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) + && try_get_connections[next_index].stage != TryGetConnection::Stage::FINISHED) + finish = true; + + /// If we made a complete round, there is no replica to connect + else if (next_index == cur_index) + return -1; + } + + LOG_DEBUG(log, "get next index: {}", next_index); + + return next_index; +} + +GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr replica) +{ + LOG_DEBUG(log, "start try get connection with {} replica", index); + TryGetConnection & try_get_connection = try_get_connections[index]; + + replica->state = State::NOT_READY; + replica->index = index; + + try_get_connection.reset(); + try_get_connection.run(); + + if (try_get_connection.stage != TryGetConnection::Stage::FAILED) + { + replica->fd = try_get_connection.socket_fd; + replica->connection = &*try_get_connection.result.entry; + } + + Action action = processTryGetConnectionStage(replica); + + if (action == Action::PROCESS_EPOLL_EVENTS) + { + epoll.add(try_get_connection.socket_fd); + try_get_connection.setEpoll(&epoll); + addTimeouts(replica); + } + + return action; +} + +GetHedgedConnections::Action +GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr replica, bool remove_from_epoll) +{ + LOG_DEBUG(log, "process get connection stage for {} replica", replica->index); + TryGetConnection & try_get_connection = try_get_connections[replica->index]; + + if (try_get_connection.stage == TryGetConnection::Stage::FINISHED) + { + LOG_DEBUG(log, "stage: FINISHED"); + ++entries_count; + + if (remove_from_epoll) + epoll.remove(try_get_connection.socket_fd); + + if (try_get_connection.result.is_usable) + { + LOG_DEBUG(log, "replica is usable"); + ++usable_count; + if (try_get_connection.result.is_up_to_date) + { + LOG_DEBUG(log, "replica is up to date, finish get hedged connections"); + replica->state = State::READY; + return Action::FINISH; + } + + /// This replica is not up to date, we will try to find up to date + replica->reset(); + return Action::TRY_NEXT_REPLICA; + } + } + else if (try_get_connection.stage == TryGetConnection::Stage::FAILED) + { + LOG_DEBUG(log, "stage: FAILED"); + processFailedConnection(replica); + return Action::TRY_NEXT_REPLICA; + } + + LOG_DEBUG(log, "middle stage, process epoll events"); + + /// Get connection process is not finished + return Action::PROCESS_EPOLL_EVENTS; +} + +void GetHedgedConnections::processFailedConnection(ReplicaStatePtr replica) +{ + LOG_DEBUG(log, "failed connection with {} replica", replica->index); + + ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; + LOG_WARNING( + log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), try_get_connections[replica->index].fail_message); + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); + + shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); + + if (shuffled_pool.error_count >= max_tries) + { + ++failed_pools_count; + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); + } + + std::string & fail_message = try_get_connections[replica->index].fail_message; + if (!fail_message.empty()) + fail_messages += fail_message + "\n"; + + replica->reset(); +} + +void GetHedgedConnections::addTimeouts(ReplicaState * replica) +{ + LOG_DEBUG(log, "add timeouts for {} replica", replica->index); + + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeouts); + + /// If we haven't connected to second replica yet, set special timeout for it + if (second_replica.isEmpty()) + { + auto stage = try_get_connections[replica->index].stage; + if (stage == TryGetConnection::Stage::RECEIVE_HELLO) + addTimeoutToReplica(TimerTypes::RECEIVE_HELLO_TIMEOUT, replica, epoll, timeouts); + else if (stage == TryGetConnection::Stage::RECEIVE_TABLES_STATUS) + addTimeoutToReplica(TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT, replica, epoll, timeouts); + } +} + +void GetHedgedConnections::swapReplicasIfNeeded() +{ + if ((!first_replica.isReady() && second_replica.isReady())) + { + LOG_DEBUG(log, "swap replicas"); + swapReplicas(); + } +} + +GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(bool non_blocking) +{ + LOG_DEBUG(log, "process epoll events"); + int event_fd; + ReplicaStatePtr replica; + bool finish = false; + while (!finish) + { + event_fd = getReadyFileDescriptor(epoll); + + if ((replica = isEventReplica(event_fd))) + finish = processReplicaEvent(replica, non_blocking); + + else if (auto * timeout_descriptor = isEventTimeout(event_fd, replica)) + { + processTimeoutEvent(replica, timeout_descriptor); + finish = true; + } + else + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); + } + + LOG_DEBUG(log, "cancel process epoll events"); + + return replica; +} + +GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::isEventReplica(int event_fd) +{ + if (event_fd == first_replica.fd) + return &first_replica; + + if (event_fd == second_replica.fd) + return &second_replica; + + return nullptr; +} + +TimerDescriptorPtr GetHedgedConnections::isEventTimeout(int event_fd, ReplicaStatePtr & replica_out) +{ + if (first_replica.active_timeouts.find(event_fd) != first_replica.active_timeouts.end()) + { + replica_out = &first_replica; + return first_replica.active_timeouts[event_fd].get(); + } + + if (second_replica.active_timeouts.find(event_fd) != second_replica.active_timeouts.end()) + { + replica_out = &second_replica; + return second_replica.active_timeouts[event_fd].get(); + } + + return nullptr; +} + +int GetHedgedConnections::getReadyFileDescriptor(Epoll & epoll_, AsyncCallback async_callback) +{ + if (first_replica.connection && first_replica.connection->hasReadPendingData()) + return first_replica.fd; + + if (second_replica.connection && second_replica.connection->hasReadPendingData()) + return second_replica.fd; + + return epoll_.getReady(std::move(async_callback)).data.fd; +} + +bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr replica, bool non_blocking) +{ + LOG_DEBUG(log, "epoll event is {} replica", replica->index); + removeTimeoutsFromReplica(replica, epoll); + try_get_connections[replica->index].run(); + Action action = processTryGetConnectionStage(replica, true); + if (action == Action::PROCESS_EPOLL_EVENTS) + { + addTimeouts(replica); + return non_blocking; + } + + return true; +} + +void GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor) +{ + LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index); + + epoll.remove(timeout_descriptor->getDescriptor()); + replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); + + if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) + { + LOG_DEBUG(log, "process receive timeout for {} replica", replica->index); + removeTimeoutsFromReplica(replica, epoll); + epoll.remove(replica->fd); + + TryGetConnection & try_get_connection = try_get_connections[replica->index]; + try_get_connection.fail_message = "Receive timeout expired (" + try_get_connection.result.entry->getDescription() + ")"; + try_get_connection.resetResult(); + try_get_connection.stage = TryGetConnection::Stage::FAILED; + processFailedConnection(replica); + } + + else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT + || timeout_descriptor->getType() == TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT) + { + if (replica->index == second_replica.index || !second_replica.isEmpty()) + throw Exception( + "Received timeout to connect with second replica, but current replica is second or second replica is not empty", + ErrorCodes::LOGICAL_ERROR); + replica = &second_replica; + } +} + +void GetHedgedConnections::setBestUsableReplica(ReplicaState & replica, int skip_index) +{ + LOG_DEBUG(log, "set best usable replica"); + + std::vector indexes(try_get_connections.size()); + for (size_t i = 0; i != indexes.size(); ++i) + indexes[i] = i; + + /// Remove unusable and failed replicas, skip the replica with skip_index index + indexes.erase( + std::remove_if( + indexes.begin(), + indexes.end(), + [&](int i) { + return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable || i == skip_index; + }), + indexes.end()); + + if (indexes.empty()) + throw Exception("There is no usable replica to choose", ErrorCodes::LOGICAL_ERROR); + + /// Sort replicas by staleness + std::stable_sort(indexes.begin(), indexes.end(), [&](size_t lhs, size_t rhs) { + return try_get_connections[lhs].result.staleness < try_get_connections[rhs].result.staleness; + }); + + replica.index = indexes[0]; + replica.connection = &*try_get_connections[indexes[0]].result.entry; + replica.state = State::READY; + replica.fd = replica.connection->getSocket()->impl()->sockfd(); +} + +void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll, const ConnectionTimeouts & timeouts) +{ + Poco::Timespan timeout; + switch (type) + { + case TimerTypes::RECEIVE_HELLO_TIMEOUT: + timeout = timeouts.receive_hello_timeout; + break; + case TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT: + timeout = timeouts.receive_tables_status_timeout; + break; + case TimerTypes::RECEIVE_DATA_TIMEOUT: + timeout = timeouts.receive_data_timeout; + break; + case TimerTypes::RECEIVE_TIMEOUT: + timeout = timeouts.receive_timeout; + break; + default: + throw Exception("Unknown timeout type", ErrorCodes::BAD_ARGUMENTS); + } + + std::unique_ptr timeout_descriptor = std::make_unique(); + timeout_descriptor->setType(type); + timeout_descriptor->setRelative(timeout); + epoll.add(timeout_descriptor->getDescriptor()); + replica->active_timeouts[timeout_descriptor->getDescriptor()] = std::move(timeout_descriptor); +} + +void removeTimeoutsFromReplica(GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll) +{ + for (auto & [fd, _] : replica->active_timeouts) + epoll.remove(fd); + replica->active_timeouts.clear(); +} + +void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll) +{ + auto it = std::find_if( + replica->active_timeouts.begin(), + replica->active_timeouts.end(), + [type](auto & value){ return value.second->getType() == type; } + ); + + if (it != replica->active_timeouts.end()) + { + epoll.remove(it->first); + replica->active_timeouts.erase(it); + } +} + +} diff --git a/src/Client/GetHedgedConnections.h b/src/Client/GetHedgedConnections.h new file mode 100644 index 00000000000..c42dc24ddc7 --- /dev/null +++ b/src/Client/GetHedgedConnections.h @@ -0,0 +1,154 @@ +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Class for establishing hedged connections with replicas. +/// It works with multiple replicas simultaneously without blocking +/// (in current implementation only with 2 replicas) by using epoll. +class GetHedgedConnections +{ +public: + using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool; + + enum State + { + EMPTY = 0, + READY = 1, + NOT_READY = 2, + CANNOT_CHOOSE = 3, + }; + + struct ReplicaState + { + Connection * connection = nullptr; + State state = State::EMPTY; + int index = -1; + int fd = -1; + std::unordered_map> active_timeouts; + + void reset() + { + connection = nullptr; + state = State::EMPTY; + index = -1; + fd = -1; + active_timeouts.clear(); + } + + bool isReady() const { return state == State::READY; }; + bool isNotReady() const { return state == State::NOT_READY; }; + bool isEmpty() const { return state == State::EMPTY; }; + bool isCannotChoose() const { return state == State::CANNOT_CHOOSE; }; + }; + + using ReplicaStatePtr = ReplicaState *; + + struct Replicas + { + ReplicaStatePtr first_replica; + ReplicaStatePtr second_replica; + }; + + GetHedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, + const Settings * settings_, + const ConnectionTimeouts & timeouts_, + std::shared_ptr table_to_check_ = nullptr); + + /// Establish connection with replicas. Return replicas as soon as connection with one of them is finished. + /// The first replica is always has state FINISHED and ready for sending query, the second replica + /// may have any state. To continue working with second replica call chooseSecondReplica(). + Replicas getConnections(); + + /// Continue choosing second replica, this function is not blocking. Second replica will be ready + /// for sending query when it has state FINISHED. + void chooseSecondReplica(); + + void stopChoosingSecondReplica(); + + void swapReplicas() { std::swap(first_replica, second_replica); } + + /// Move ready replica to the first place. + void swapReplicasIfNeeded(); + + /// Check if the file descriptor is belong to one of replicas. If yes, return this replica, if no, return nullptr. + ReplicaStatePtr isEventReplica(int event_fd); + + /// Check if the file descriptor is belong to timeout to any replica. + /// If yes, return corresponding TimerDescriptor and set timeout owner to replica, + /// if no, return nullptr. + TimerDescriptorPtr isEventTimeout(int event_fd, ReplicaStatePtr & replica); + + /// Get file rescriptor that ready for reading. + int getReadyFileDescriptor(Epoll & epoll_, AsyncCallback async_callback = {}); + + int getFileDescriptor() const { return epoll.getFileDescriptor(); } + + const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; } + + ~GetHedgedConnections(); + +private: + + enum Action + { + FINISH = 0, + PROCESS_EPOLL_EVENTS = 1, + TRY_NEXT_REPLICA = 2, + }; + + Action startTryGetConnection(int index, ReplicaStatePtr replica); + + Action processTryGetConnectionStage(ReplicaStatePtr replica, bool remove_from_epoll = false); + + int getNextIndex(int cur_index = -1); + + void addTimeouts(ReplicaStatePtr replica); + + void processFailedConnection(ReplicaStatePtr replica); + + void processReceiveTimeout(ReplicaStatePtr replica); + + bool processReplicaEvent(ReplicaStatePtr replica, bool non_blocking); + + void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor); + + ReplicaStatePtr processEpollEvents(bool non_blocking = false); + + void setBestUsableReplica(ReplicaState & replica, int skip_index = -1); + + const ConnectionPoolWithFailoverPtr pool; + const Settings * settings; + const ConnectionTimeouts timeouts; + std::shared_ptr table_to_check; + std::vector try_get_connections; + std::vector shuffled_pools; + ReplicaState first_replica; + ReplicaState second_replica; + bool fallback_to_stale_replicas; + Epoll epoll; + Poco::Logger * log; + std::string fail_messages; + size_t entries_count; + size_t usable_count; + size_t failed_pools_count; + size_t max_tries; + +}; + +/// Add timeout with particular type to replica and add it to epoll. +void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll, const ConnectionTimeouts & timeouts); + +/// Remove timeout with particular type from replica and epoll. +void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll); + +/// Remove all timeouts from replica and epoll. +void removeTimeoutsFromReplica(GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll); + +} diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp new file mode 100644 index 00000000000..57315bcd6fe --- /dev/null +++ b/src/Client/HedgedConnections.cpp @@ -0,0 +1,389 @@ +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int MISMATCH_REPLICAS_DATA_SOURCES; + extern const int LOGICAL_ERROR; + extern const int SOCKET_TIMEOUT; +} + +HedgedConnections::HedgedConnections( + const ConnectionPoolWithFailoverPtr & pool_, + const Settings & settings_, + const ConnectionTimeouts & timeouts_, + const ThrottlerPtr & throttler_, + std::shared_ptr table_to_check_) + : get_hedged_connections(pool_, &settings_, timeouts_, table_to_check_), settings(settings_), throttler(throttler_), log(&Poco::Logger::get("HedgedConnections")) +{ + replicas = get_hedged_connections.getConnections(); + + /// First replica may have state CANNOT_CHOOSE if setting skip_unavailable_shards is enabled + if (replicas.first_replica->isReady()) + replicas.first_replica->connection->setThrottler(throttler); + + if (!replicas.second_replica->isCannotChoose()) + { + if (replicas.second_replica->isNotReady()) + epoll.add(get_hedged_connections.getFileDescriptor()); + + auto set_throttler = [this, throttler_](ReplicaStatePtr replica) + { + replica->connection->setThrottler(throttler_); + }; + second_replica_pipeline.add(std::function(set_throttler)); + } +} + +void HedgedConnections::Pipeline::add(std::function send_function) +{ + pipeline.push_back(send_function); +} + +void HedgedConnections::Pipeline::run(ReplicaStatePtr replica) +{ + for (auto & send_func : pipeline) + send_func(replica); + + pipeline.clear(); +} + +size_t HedgedConnections::size() const +{ + if (replicas.first_replica->isReady() || replicas.second_replica->isReady()) + return 1; + + return 0; +} + +bool HedgedConnections::hasActiveConnections() const +{ + return replicas.first_replica->isReady() || replicas.second_replica->isReady(); +} + +void HedgedConnections::sendScalarsData(Scalars & data) +{ + std::lock_guard lock(cancel_mutex); + + if (!sent_query) + throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); + + auto send_scalars_data = [&data](ReplicaStatePtr replica) { replica->connection->sendScalarsData(data); }; + + if (replicas.first_replica->isReady()) + send_scalars_data(replicas.first_replica); + + if (replicas.second_replica->isReady()) + send_scalars_data(replicas.second_replica); + else if (!replicas.second_replica->isCannotChoose()) + second_replica_pipeline.add(std::function(send_scalars_data)); +} + +void HedgedConnections::sendExternalTablesData(std::vector & data) +{ + std::lock_guard lock(cancel_mutex); + + if (!sent_query) + throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); + + if (data.size() != size()) + throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES); + + auto send_external_tables_data = [&data](ReplicaStatePtr replica) { replica->connection->sendExternalTablesData(data[0]); }; + + if (replicas.first_replica->isReady()) + send_external_tables_data(replicas.first_replica); + + if (replicas.second_replica->isReady()) + send_external_tables_data(replicas.second_replica); + else if (!replicas.second_replica->isCannotChoose()) + second_replica_pipeline.add(send_external_tables_data); +} + +void HedgedConnections::sendQuery( + const ConnectionTimeouts & timeouts, + const String & query, + const String & query_id, + UInt64 stage, + const ClientInfo & client_info, + bool with_pending_data) +{ + std::lock_guard lock(cancel_mutex); + + if (sent_query) + throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR); + + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr replica) + { + Settings modified_settings = settings; + if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + { + modified_settings.group_by_two_level_threshold = 0; + modified_settings.group_by_two_level_threshold_bytes = 0; + } + + replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); + this->epoll.add(replica->fd); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, timeouts); + }; + + if (replicas.first_replica->isReady()) + { + send_query(replicas.first_replica); + if (replicas.second_replica->isEmpty()) + addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replicas.first_replica, epoll, timeouts); + } + + if (replicas.second_replica->isReady()) + send_query(replicas.second_replica); + else if (!replicas.second_replica->isCannotChoose()) + second_replica_pipeline.add(send_query); + + sent_query = true; +} + +void HedgedConnections::disconnect() +{ + std::lock_guard lock(cancel_mutex); + + if (replicas.first_replica->isReady()) + { + replicas.first_replica->connection->disconnect(); + replicas.first_replica->reset(); + } + + if (replicas.second_replica->isReady()) + { + replicas.second_replica->connection->disconnect(); + replicas.second_replica->reset(); + } + else if (replicas.second_replica->isNotReady()) + get_hedged_connections.stopChoosingSecondReplica(); +} + +std::string HedgedConnections::dumpAddresses() const +{ + std::lock_guard lock(cancel_mutex); + + std::string addresses = ""; + + if (replicas.first_replica->isReady()) + addresses += replicas.first_replica->connection->getDescription(); + + if (replicas.second_replica->isReady()) + addresses += "; " + replicas.second_replica->connection->getDescription(); + + return addresses; +} + +void HedgedConnections::sendCancel() +{ + std::lock_guard lock(cancel_mutex); + + if (!sent_query || cancelled) + throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR); + + if (replicas.first_replica->isReady()) + replicas.first_replica->connection->sendCancel(); + + if (replicas.second_replica->isReady()) + replicas.second_replica->connection->sendCancel(); + + cancelled = true; +} + + +Packet HedgedConnections::drain() +{ + std::lock_guard lock(cancel_mutex); + + if (!cancelled) + throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR); + + Packet res; + res.type = Protocol::Server::EndOfStream; + + while (epoll.size() != 0) + { + Packet packet = receivePacketImpl(); + switch (packet.type) + { + case Protocol::Server::Data: + case Protocol::Server::Progress: + case Protocol::Server::ProfileInfo: + case Protocol::Server::Totals: + case Protocol::Server::Extremes: + case Protocol::Server::EndOfStream: + break; + + case Protocol::Server::Exception: + default: + /// If we receive an exception or an unknown packet, we save it. + res = std::move(packet); + break; + } + } + + return res; +} + +Packet HedgedConnections::receivePacket() +{ + std::lock_guard lock(cancel_mutex); + return receivePacketUnlocked(); +} + +Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) +{ + if (!sent_query) + throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR); + if (!hasActiveConnections()) + throw Exception("No more packets are available.", ErrorCodes::LOGICAL_ERROR); + + if (epoll.size() == 0) + throw Exception("No pending events in epoll.", ErrorCodes::LOGICAL_ERROR); + + return receivePacketImpl(std::move(async_callback)); +} + +Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) +{ + int event_fd; + ReplicaStatePtr replica; + Packet packet; + bool finish = false; + while (!finish) + { + event_fd = get_hedged_connections.getReadyFileDescriptor(epoll, async_callback); + + if (auto timeout_descriptor = get_hedged_connections.isEventTimeout(event_fd, replica)) + processTimeoutEvent(replica, timeout_descriptor); + else if ((replica = get_hedged_connections.isEventReplica(event_fd))) + { + packet = receivePacketFromReplica(replica, async_callback); + finish = true; + } + else if (event_fd == get_hedged_connections.getFileDescriptor()) + processGetHedgedConnectionsEvent(); + else + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); + } + + return packet; +}; + +Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback) +{ + Packet packet = replica->connection->receivePacket(std::move(async_callback)); + switch (packet.type) + { + case Protocol::Server::Data: + removeTimeoutsFromReplica(replica, epoll); + processReceiveData(replica); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, get_hedged_connections.getConnectionTimeouts()); + break; + case Protocol::Server::Progress: + case Protocol::Server::ProfileInfo: + case Protocol::Server::Totals: + case Protocol::Server::Extremes: + case Protocol::Server::Log: + removeTimeoutFromReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, get_hedged_connections.getConnectionTimeouts()); + break; + + case Protocol::Server::EndOfStream: + finishProcessReplica(replica, false); + break; + + case Protocol::Server::Exception: + default: + finishProcessReplica(replica, true); + break; + } + + return packet; +} + +void HedgedConnections::processReceiveData(ReplicaStatePtr replica) +{ + /// When we receive first packet of data from any replica, we continue working with this replica + /// and stop working with another replica (if there is another replica). If current replica is + /// second, move it to the first place. + if (replica == replicas.second_replica) + get_hedged_connections.swapReplicas(); + + if (replicas.second_replica->isCannotChoose() || replicas.second_replica->isEmpty()) + return; + + if (replicas.second_replica->isNotReady()) + { + get_hedged_connections.stopChoosingSecondReplica(); + epoll.remove(get_hedged_connections.getFileDescriptor()); + } + else if (replicas.second_replica->isReady()) + { + replicas.second_replica->connection->sendCancel(); + finishProcessReplica(replicas.second_replica, true); + } +} + +void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor) +{ + epoll.remove(timeout_descriptor->getDescriptor()); + replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); + + if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) + { + finishProcessReplica(replica, true); + + if (!replicas.first_replica->isReady() && !replicas.second_replica->isNotReady()) + throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); + } + else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_DATA_TIMEOUT) + { + if (!replicas.second_replica->isEmpty()) + throw Exception("Cannot start choosing second replica, it's not empty", ErrorCodes::LOGICAL_ERROR); + + get_hedged_connections.chooseSecondReplica(); + + if (replicas.second_replica->isReady()) + processChosenSecondReplica(); + else if (replicas.second_replica->isNotReady()) + epoll.add(get_hedged_connections.getFileDescriptor()); + } +} + +void HedgedConnections::processGetHedgedConnectionsEvent() +{ + get_hedged_connections.chooseSecondReplica(); + if (replicas.second_replica->isReady()) + processChosenSecondReplica(); + + if (!replicas.second_replica->isNotReady()) + epoll.remove(get_hedged_connections.getFileDescriptor()); +} + +void HedgedConnections::processChosenSecondReplica() +{ + second_replica_pipeline.run(replicas.second_replica); + + /// In case when the first replica get receive timeout before the second is chosen, + /// we need to move the second replica to the first place + get_hedged_connections.swapReplicasIfNeeded(); +} + +void HedgedConnections::finishProcessReplica(ReplicaStatePtr replica, bool disconnect) +{ + removeTimeoutsFromReplica(replica, epoll); + epoll.remove(replica->fd); + if (disconnect) + replica->connection->disconnect(); + replica->reset(); + + /// Move active connection to the first replica if it exists + get_hedged_connections.swapReplicasIfNeeded(); +} + +} diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h new file mode 100644 index 00000000000..b84ad89658f --- /dev/null +++ b/src/Client/HedgedConnections.h @@ -0,0 +1,93 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class HedgedConnections : public IConnections +{ +public: + using ReplicaStatePtr = GetHedgedConnections::ReplicaStatePtr; + using Replicas = GetHedgedConnections::Replicas; + + HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, + const Settings & settings_, + const ConnectionTimeouts & timeouts_, + const ThrottlerPtr & throttler, + std::shared_ptr table_to_check_ = nullptr); + + void sendScalarsData(Scalars & data) override; + + void sendExternalTablesData(std::vector & data) override; + + void sendQuery( + const ConnectionTimeouts & timeouts, + const String & query, + const String & query_id, + UInt64 stage, + const ClientInfo & client_info, + bool with_pending_data) override; + + Packet receivePacket() override; + + Packet receivePacketUnlocked(AsyncCallback async_callback = {}) override; + + void disconnect() override; + + void sendCancel() override; + + Packet drain() override; + + std::string dumpAddresses() const override; + + size_t size() const override; + + bool hasActiveConnections() const override; + +private: + class Pipeline + { + public: + void add(std::function send_function); + + void run(ReplicaStatePtr replica); + + bool empty() const { return pipeline.empty(); } + + private: + std::vector> pipeline; + }; + + void processChosenSecondReplica(); + + Packet receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback = {}); + + Packet receivePacketImpl(AsyncCallback async_callback = {}); + + void processReceiveData(ReplicaStatePtr replica); + + void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor); + + void processGetHedgedConnectionsEvent(); + + void removeReceiveTimeout(ReplicaStatePtr replica); + + void finishProcessReplica(ReplicaStatePtr replica, bool disconnect); + + GetHedgedConnections get_hedged_connections; + Replicas replicas; + Epoll epoll; + const Settings & settings; + ThrottlerPtr throttler; + Poco::Logger * log; + Pipeline second_replica_pipeline; + bool sent_query = false; + bool cancelled = false; + + mutable std::mutex cancel_mutex; +}; + +} diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h new file mode 100644 index 00000000000..85d1e29c243 --- /dev/null +++ b/src/Client/IConnections.h @@ -0,0 +1,57 @@ +#pragma once + +#include + +namespace DB +{ + +/// Base class for working with multiple replicas (connections) +/// from one shard within a single thread +class IConnections : boost::noncopyable +{ +public: + /// Send all scalars to replicas. + virtual void sendScalarsData(Scalars & data) = 0; + /// Send all content of external tables to replicas. + virtual void sendExternalTablesData(std::vector & data) = 0; + + /// Send request to replicas. + virtual void sendQuery( + const ConnectionTimeouts & timeouts, + const String & query, + const String & query_id, + UInt64 stage, + const ClientInfo & client_info, + bool with_pending_data) = 0; + + /// Get packet from any replica. + virtual Packet receivePacket() = 0; + + /// Version of `receivePacket` function without locking. + virtual Packet receivePacketUnlocked(AsyncCallback async_callback) = 0; + + /// Break all active connections. + virtual void disconnect() = 0; + + /// Send a request to replicas to cancel the request + virtual void sendCancel() = 0; + + /** On each replica, read and skip all packets to EndOfStream or Exception. + * Returns EndOfStream if no exception has been received. Otherwise + * returns the last received packet of type Exception. + */ + virtual Packet drain() = 0; + + /// Get the replica addresses as a string. + virtual std::string dumpAddresses() const = 0; + + /// Returns the number of replicas. + virtual size_t size() const = 0; + + /// Check if there are any valid replicas. + virtual bool hasActiveConnections() const = 0; + + virtual ~IConnections() = default; +}; + +} diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index ed7aad0a515..fbf8c9aa172 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -237,7 +237,7 @@ std::string MultiplexedConnections::dumpAddressesUnlocked() const return buf.str(); } -Packet MultiplexedConnections::receivePacketUnlocked(std::function async_callback) +Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callback) { if (!sent_query) throw Exception("Cannot receive packets: no query sent.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 2ab2b60570e..720add1ba81 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -16,7 +17,7 @@ namespace DB * * The interface is almost the same as Connection. */ -class MultiplexedConnections final : private boost::noncopyable +class MultiplexedConnections final : public IConnections { public: /// Accepts ready connection. @@ -27,49 +28,35 @@ public: std::vector && connections, const Settings & settings_, const ThrottlerPtr & throttler_); - /// Send all scalars to replicas. - void sendScalarsData(Scalars & data); - /// Send all content of external tables to replicas. - void sendExternalTablesData(std::vector & data); + void sendScalarsData(Scalars & data) override; + void sendExternalTablesData(std::vector & data) override; - /// Send request to replicas. void sendQuery( const ConnectionTimeouts & timeouts, const String & query, const String & query_id, UInt64 stage, const ClientInfo & client_info, - bool with_pending_data); + bool with_pending_data) override; - /// Get packet from any replica. - Packet receivePacket(); + Packet receivePacket() override; - /// Break all active connections. - void disconnect(); + void disconnect() override; - /// Send a request to the replica to cancel the request - void sendCancel(); + void sendCancel() override; - /** On each replica, read and skip all packets to EndOfStream or Exception. - * Returns EndOfStream if no exception has been received. Otherwise - * returns the last received packet of type Exception. - */ - Packet drain(); + Packet drain() override; - /// Get the replica addresses as a string. - std::string dumpAddresses() const; + std::string dumpAddresses() const override; - /// Returns the number of replicas. /// Without locking, because sendCancel() does not change this number. - size_t size() const { return replica_states.size(); } + size_t size() const override { return replica_states.size(); } - /// Check if there are any valid replicas. /// Without locking, because sendCancel() does not change the state of the replicas. - bool hasActiveConnections() const { return active_connection_count > 0; } + bool hasActiveConnections() const override { return active_connection_count > 0; } private: - /// Internal version of `receivePacket` function without locking. - Packet receivePacketUnlocked(std::function async_callback = {}); + Packet receivePacketUnlocked(AsyncCallback async_callback = {}) override; /// Internal version of `dumpAddresses` function without locking. std::string dumpAddressesUnlocked() const; diff --git a/src/Client/ya.make b/src/Client/ya.make index 87a0cea102a..603e8290350 100644 --- a/src/Client/ya.make +++ b/src/Client/ya.make @@ -12,6 +12,8 @@ PEERDIR( SRCS( Connection.cpp ConnectionPoolWithFailover.cpp + GetHedgedConnections.cpp + HedgedConnections.cpp MultiplexedConnections.cpp TimeoutSetter.cpp diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp new file mode 100644 index 00000000000..8ce100c7834 --- /dev/null +++ b/src/Common/Epoll.cpp @@ -0,0 +1,82 @@ +#include "Epoll.h" +#include +#include +#include + +namespace DB +{ + + +/// TODO: add appropriate error codes +namespace ErrorCodes +{ + extern const int EPOLL_ERROR; + extern const int LOGICAL_ERROR; +} + +Epoll::Epoll() : events_count(0) +{ + epoll_fd = epoll_create1(0); + if (epoll_fd == -1) + throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR); +} + +void Epoll::add(int fd, void * ptr) +{ + epoll_event event; + event.events = EPOLLIN | EPOLLPRI; + if (ptr) + event.data.ptr = ptr; + else + event.data.fd = fd; + + if (epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &event) == -1) + throwFromErrno("Cannot add new descriptor to epoll", DB::ErrorCodes::EPOLL_ERROR); + + ++events_count; +} + +void Epoll::remove(int fd) +{ + if (epoll_ctl(epoll_fd, EPOLL_CTL_DEL, fd, nullptr) == -1) + throwFromErrno("Cannot remove descriptor from epoll", DB::ErrorCodes::EPOLL_ERROR); + + --events_count; +} + +epoll_event Epoll::getReady(AsyncCallback async_callback) const +{ + std::vector events = getManyReady(1, true, std::move(async_callback)); + if (events.empty()) + throw Exception("Vector of ready events is empty", ErrorCodes::LOGICAL_ERROR); + + return events[0]; +} + +std::vector Epoll::getManyReady(int max_events, bool blocking, AsyncCallback async_callback) const +{ + std::vector events(max_events); + + int ready_size = 0; + int timeout = blocking && !async_callback ? -1 : 0; + while (ready_size <= 0 && (ready_size != 0 || blocking)) + { + ready_size = epoll_wait(epoll_fd, events.data(), max_events, timeout); + + if (ready_size == -1 && errno != EINTR) + throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR); + + if (ready_size == 0 && blocking && async_callback) + async_callback(epoll_fd, 0, "epoll"); + } + + events.resize(ready_size); + return events; +} + +Epoll::~Epoll() +{ + close(epoll_fd); +} + +} diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h new file mode 100644 index 00000000000..0e04d666af0 --- /dev/null +++ b/src/Common/Epoll.h @@ -0,0 +1,44 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +using AsyncCallback = std::function; + +class Epoll : boost::noncopyable +{ +public: + Epoll(); + + /// Add new file descriptor to epoll. + void add(int fd, void * ptr = nullptr); + + /// Remove file descriptor to epoll. + void remove(int fd); + + /// Get events from epoll. If blocking is false and there are no ready events, + /// return empty vector, otherwise wait for ready events. If blocking is true, + /// async_callback is given and there is no ready events, async_callback is called + /// with epoll file descriptor. + std::vector getManyReady(int max_events, bool blocking, AsyncCallback async_callback = {}) const; + + /// Get only one ready event, this function is always blocking. + epoll_event getReady(AsyncCallback async_callback = {}) const; + + int getFileDescriptor() const { return epoll_fd; } + + int size() const { return events_count; } + + ~Epoll(); + +private: + int epoll_fd; + int events_count; +}; + +} diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index a2cd65137c0..fe0c0533adc 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -533,6 +533,7 @@ M(564, INTERSERVER_SCHEME_DOESNT_MATCH) \ M(565, TOO_MANY_PARTITIONS) \ M(566, CANNOT_RMDIR) \ + M(567, EPOLL_ERROR) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Common/PoolWithFailoverBase.h b/src/Common/PoolWithFailoverBase.h index 7779d18d969..6bb6f4a94dd 100644 --- a/src/Common/PoolWithFailoverBase.h +++ b/src/Common/PoolWithFailoverBase.h @@ -93,6 +93,18 @@ public: double staleness = 0.0; /// Helps choosing the "least stale" option when all replicas are stale. }; + struct PoolState; + + using PoolStates = std::vector; + + struct ShuffledPool + { + NestedPool * pool{}; + const PoolState * state{}; + size_t index = 0; + size_t error_count = 0; + }; + /// This functor must be provided by a client. It must perform a single try that takes a connection /// from the provided pool and checks that it is good. using TryGetEntryFunc = std::function; @@ -113,9 +125,6 @@ public: const GetPriorityFunc & get_priority = GetPriorityFunc()); protected: - struct PoolState; - - using PoolStates = std::vector; /// Returns a single connection. Entry get(size_t max_ignored_errors, bool fallback_to_stale_replicas, @@ -124,6 +133,10 @@ protected: /// This function returns a copy of pool states to avoid race conditions when modifying shared pool states. PoolStates updatePoolStates(size_t max_ignored_errors); + std::vector getShuffledPools(size_t max_ignored_errors, const GetPriorityFunc & get_priority); + + inline void updateSharedErrorCounts(std::vector & shuffled_pools); + auto getPoolExtendedStates() const { std::lock_guard lock(pool_states_mutex); @@ -143,6 +156,46 @@ protected: Poco::Logger * log; }; + +template +std::vector::ShuffledPool> +PoolWithFailoverBase::getShuffledPools( + size_t max_ignored_errors, const PoolWithFailoverBase::GetPriorityFunc & get_priority) +{ + /// Update random numbers and error counts. + PoolStates pool_states = updatePoolStates(max_ignored_errors); + if (get_priority) + { + for (size_t i = 0; i < pool_states.size(); ++i) + pool_states[i].priority = get_priority(i); + } + + /// Sort the pools into order in which they will be tried (based on respective PoolStates). + std::vector shuffled_pools; + shuffled_pools.reserve(nested_pools.size()); + for (size_t i = 0; i < nested_pools.size(); ++i) + shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0}); + std::sort( + shuffled_pools.begin(), shuffled_pools.end(), + [](const ShuffledPool & lhs, const ShuffledPool & rhs) + { + return PoolState::compare(*lhs.state, *rhs.state); + }); + + return shuffled_pools; +} + +template +inline void PoolWithFailoverBase::updateSharedErrorCounts(std::vector & shuffled_pools) +{ + std::lock_guard lock(pool_states_mutex); + for (const ShuffledPool & pool: shuffled_pools) + { + auto & pool_state = shared_pool_states[pool.index]; + pool_state.error_count = std::min(max_error_cap, pool_state.error_count + pool.error_count); + } +} + template typename TNestedPool::Entry PoolWithFailoverBase::get(size_t max_ignored_errors, bool fallback_to_stale_replicas, @@ -168,33 +221,7 @@ PoolWithFailoverBase::getMany( const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority) { - /// Update random numbers and error counts. - PoolStates pool_states = updatePoolStates(max_ignored_errors); - if (get_priority) - { - for (size_t i = 0; i < pool_states.size(); ++i) - pool_states[i].priority = get_priority(i); - } - - struct ShuffledPool - { - NestedPool * pool{}; - const PoolState * state{}; - size_t index = 0; - size_t error_count = 0; - }; - - /// Sort the pools into order in which they will be tried (based on respective PoolStates). - std::vector shuffled_pools; - shuffled_pools.reserve(nested_pools.size()); - for (size_t i = 0; i < nested_pools.size(); ++i) - shuffled_pools.push_back(ShuffledPool{nested_pools[i].get(), &pool_states[i], i, 0}); - std::sort( - shuffled_pools.begin(), shuffled_pools.end(), - [](const ShuffledPool & lhs, const ShuffledPool & rhs) - { - return PoolState::compare(*lhs.state, *rhs.state); - }); + std::vector shuffled_pools = getShuffledPools(max_ignored_errors, get_priority); /// We will try to get a connection from each pool until a connection is produced or max_tries is reached. std::vector try_results(shuffled_pools.size()); @@ -206,12 +233,7 @@ PoolWithFailoverBase::getMany( /// At exit update shared error counts with error counts occurred during this call. SCOPE_EXIT( { - std::lock_guard lock(pool_states_mutex); - for (const ShuffledPool & pool: shuffled_pools) - { - auto & pool_state = shared_pool_states[pool.index]; - pool_state.error_count = std::min(max_error_cap, pool_state.error_count + pool.error_count); - } + updateSharedErrorCounts(shuffled_pools); }); std::string fail_messages; diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index f4c3ec35588..e2b8a0ec928 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -74,7 +74,7 @@ void TimerDescriptor::setRelative(const Poco::Timespan & timespan) const spec.it_interval.tv_nsec = 0; spec.it_interval.tv_sec = 0; spec.it_value.tv_sec = timespan.totalSeconds(); - spec.it_value.tv_nsec = timespan.useconds(); + spec.it_value.tv_nsec = timespan.useconds() * 1000; if (-1 == timerfd_settime(timer_fd, 0 /*relative timer */, &spec, nullptr)) throwFromErrno("Cannot set time for timer_fd", ErrorCodes::CANNOT_SET_TIMER_PERIOD); diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index ddb8f2a1367..fa49189abfc 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -5,14 +5,24 @@ namespace DB { +enum TimerTypes +{ + DEFAULT, + RECEIVE_HELLO_TIMEOUT, + RECEIVE_TABLES_STATUS_TIMEOUT, + RECEIVE_DATA_TIMEOUT, + RECEIVE_TIMEOUT, +}; + /// Wrapper over timerfd. class TimerDescriptor { private: int timer_fd; + int type = TimerTypes::DEFAULT; public: - explicit TimerDescriptor(int clockid, int flags); + explicit TimerDescriptor(int clockid = CLOCK_MONOTONIC, int flags = 0); ~TimerDescriptor(); TimerDescriptor(const TimerDescriptor &) = delete; @@ -21,11 +31,15 @@ public: TimerDescriptor & operator=(TimerDescriptor &&) = default; int getDescriptor() const { return timer_fd; } + int getType() const { return type; } void reset() const; void drain() const; void setRelative(const Poco::Timespan & timespan) const; + void setType(int type_) { type = type_; } }; +using TimerDescriptorPtr = TimerDescriptor *; + } #endif diff --git a/src/Common/ya.make b/src/Common/ya.make index 5b5da618bbe..9097736c32c 100644 --- a/src/Common/ya.make +++ b/src/Common/ya.make @@ -40,6 +40,7 @@ SRCS( DirectorySyncGuard.cpp Dwarf.cpp Elf.cpp + Epoll.cpp ErrorCodes.cpp Exception.cpp ExternalLoaderStatus.cpp diff --git a/src/Core/Defines.h b/src/Core/Defines.h index f1cd1a8a708..d8e8b526600 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -11,6 +11,10 @@ #define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS 100 #define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300 #define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300 +/// Timeouts for hedged requests +#define DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_SEC 1 +#define DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_SEC 1 +#define DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC 1 /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus). #define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5 #define DBMS_DEFAULT_POLL_INTERVAL 10 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1e4b07997ab..46da24aca80 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,6 +55,10 @@ class IColumn; M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ + M(Seconds, receive_hello_timeout, DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_SEC, "Connection timeout for receiving hello from replica", 0) \ + M(Seconds, receive_tables_status_timeout, DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_SEC, "Connection timeout for receiving tables status from replica", 0) \ + M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data from replica", 0) \ + M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \ M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ M(Milliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \ M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \ @@ -215,6 +219,11 @@ class IColumn; M(Milliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.", 0) \ M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ + /** Settings for testing hedged requests */ \ + M(Int64, sleep_before_send_hello, 0, "Time to sleep before sending hello in TCPHandler", 0) \ + M(Int64, sleep_before_send_tables_status, 0, "Time to sleep before sending tables status response in TCPHandler", 0) \ + M(Int64, sleep_before_send_data, 0, "Time to sleep before sending data in TCPHandler", 0) \ + \ M(Bool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ M(Seconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout", 0) \ diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 14e51ffefdf..49654b51199 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -13,6 +13,8 @@ #include #include #include +#include +#include namespace DB { @@ -29,23 +31,23 @@ RemoteQueryExecutor::RemoteQueryExecutor( : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) { - create_multiplexed_connections = [this, &connection, throttler]() + create_connections = [this, &connection, throttler]() { return std::make_unique(connection, context.getSettingsRef(), throttler); }; } RemoteQueryExecutor::RemoteQueryExecutor( - std::vector && connections, + std::vector && connections_, const String & query_, const Block & header_, const Context & context_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_, QueryProcessingStage::Enum stage_) : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) { - create_multiplexed_connections = [this, connections, throttler]() mutable + create_connections = [this, connections_, throttler]() mutable { return std::make_unique( - std::move(connections), context.getSettingsRef(), throttler); + std::move(connections_), context.getSettingsRef(), throttler); }; } @@ -56,23 +58,34 @@ RemoteQueryExecutor::RemoteQueryExecutor( : header(header_), query(query_), context(context_) , scalars(scalars_), external_tables(external_tables_), stage(stage_) { - create_multiplexed_connections = [this, pool, throttler]() + create_connections = [this, pool, throttler]()->std::unique_ptr { const Settings & current_settings = context.getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - std::vector connections; - if (main_table) + + if (current_settings.use_hedged_requests && current_settings.max_parallel_replicas <= 1) { - auto try_results = pool->getManyChecked(timeouts, ¤t_settings, pool_mode, main_table.getQualifiedName()); - connections.reserve(try_results.size()); - for (auto & try_result : try_results) - connections.emplace_back(std::move(try_result.entry)); + std::shared_ptr table_to_check = nullptr; + if (main_table) + table_to_check = std::make_shared(main_table.getQualifiedName()); + + return std::make_unique(pool, current_settings, timeouts, throttler, table_to_check); } else - connections = pool->getMany(timeouts, ¤t_settings, pool_mode); + { + std::vector connection_entries; + if (main_table) + { + auto try_results = pool->getManyChecked(timeouts, ¤t_settings, pool_mode, main_table.getQualifiedName()); + connection_entries.reserve(try_results.size()); + for (auto & try_result : try_results) + connection_entries.emplace_back(std::move(try_result.entry)); + } + else + connection_entries = pool->getMany(timeouts, ¤t_settings, pool_mode); - return std::make_unique( - std::move(connections), current_settings, throttler); + return std::make_unique(std::move(connection_entries), current_settings, throttler); + } }; } @@ -83,7 +96,7 @@ RemoteQueryExecutor::~RemoteQueryExecutor() * these connections did not remain hanging in the out-of-sync state. */ if (established || isQueryPending()) - multiplexed_connections->disconnect(); + connections->disconnect(); } /** If we receive a block with slightly different column types, or with excessive columns, @@ -140,10 +153,10 @@ void RemoteQueryExecutor::sendQuery() if (sent_query) return; - multiplexed_connections = create_multiplexed_connections(); + connections = create_connections(); const auto & settings = context.getSettingsRef(); - if (settings.skip_unavailable_shards && 0 == multiplexed_connections->size()) + if (settings.skip_unavailable_shards && 0 == connections->size()) return; /// Query cannot be canceled in the middle of the send query, @@ -167,7 +180,7 @@ void RemoteQueryExecutor::sendQuery() modified_client_info.client_trace_context = CurrentThread::get().thread_trace_context; } - multiplexed_connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true); + connections->sendQuery(timeouts, query, query_id, stage, modified_client_info, true); established = false; sent_query = true; @@ -183,7 +196,7 @@ Block RemoteQueryExecutor::read() { sendQuery(); - if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size())) + if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size())) return {}; } @@ -192,7 +205,7 @@ Block RemoteQueryExecutor::read() if (was_cancelled) return Block(); - Packet packet = multiplexed_connections->receivePacket(); + Packet packet = connections->receivePacket(); if (auto block = processPacket(std::move(packet))) return *block; @@ -207,7 +220,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr { sendQuery(); - if (context.getSettingsRef().skip_unavailable_shards && (0 == multiplexed_connections->size())) + if (context.getSettingsRef().skip_unavailable_shards && (0 == connections->size())) return Block(); } @@ -217,7 +230,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr if (was_cancelled) return Block(); - read_context = std::make_unique(*multiplexed_connections); + read_context = std::make_unique(*connections); } do @@ -228,7 +241,7 @@ std::variant RemoteQueryExecutor::read(std::unique_ptr if (read_context->is_read_in_progress.load(std::memory_order_relaxed)) { read_context->setTimer(); - return read_context->epoll_fd; + return read_context->epoll.getFileDescriptor(); } else { @@ -258,7 +271,7 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) break; case Protocol::Server::EndOfStream: - if (!multiplexed_connections->hasActiveConnections()) + if (!connections->hasActiveConnections()) { finished = true; return Block(); @@ -300,7 +313,7 @@ std::optional RemoteQueryExecutor::processPacket(Packet packet) got_unknown_packet_from_replica = true; throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from one of the following replicas: {}", toString(packet.type), - multiplexed_connections->dumpAddresses()); + connections->dumpAddresses()); } return {}; @@ -326,7 +339,7 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) tryCancel("Cancelling query because enough data has been read", read_context); /// Get the remaining packets so that there is no out of sync in the connections to the replicas. - Packet packet = multiplexed_connections->drain(); + Packet packet = connections->drain(); switch (packet.type) { case Protocol::Server::EndOfStream: @@ -348,7 +361,7 @@ void RemoteQueryExecutor::finish(std::unique_ptr * read_context) got_unknown_packet_from_replica = true; throw Exception(ErrorCodes::UNKNOWN_PACKET_FROM_SERVER, "Unknown packet {} from one of the following replicas: {}", toString(packet.type), - multiplexed_connections->dumpAddresses()); + connections->dumpAddresses()); } } @@ -371,14 +384,14 @@ void RemoteQueryExecutor::cancel(std::unique_ptr * read_context) void RemoteQueryExecutor::sendScalars() { - multiplexed_connections->sendScalarsData(scalars); + connections->sendScalarsData(scalars); } void RemoteQueryExecutor::sendExternalTables() { SelectQueryInfo query_info; - size_t count = multiplexed_connections->size(); + size_t count = connections->size(); { std::lock_guard lock(external_tables_mutex); @@ -415,7 +428,7 @@ void RemoteQueryExecutor::sendExternalTables() } } - multiplexed_connections->sendExternalTablesData(external_tables_data); + connections->sendExternalTablesData(external_tables_data); } void RemoteQueryExecutor::tryCancel(const char * reason, std::unique_ptr * read_context) @@ -432,11 +445,11 @@ void RemoteQueryExecutor::tryCancel(const char * reason, std::unique_ptrcancel(); - multiplexed_connections->sendCancel(); + connections->sendCancel(); } if (log) - LOG_TRACE(log, "({}) {}", multiplexed_connections->dumpAddresses(), reason); + LOG_TRACE(log, "({}) {}", connections->dumpAddresses(), reason); } bool RemoteQueryExecutor::isQueryPending() const diff --git a/src/DataStreams/RemoteQueryExecutor.h b/src/DataStreams/RemoteQueryExecutor.h index 46d9d067563..2a1f2687e59 100644 --- a/src/DataStreams/RemoteQueryExecutor.h +++ b/src/DataStreams/RemoteQueryExecutor.h @@ -1,7 +1,8 @@ #pragma once #include -#include +#include +#include #include #include #include @@ -40,7 +41,7 @@ public: /// Accepts several connections already taken from pool. RemoteQueryExecutor( - std::vector && connections, + std::vector && connections_, const String & query_, const Block & header_, const Context & context_, const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); @@ -100,8 +101,8 @@ private: Block totals; Block extremes; - std::function()> create_multiplexed_connections; - std::unique_ptr multiplexed_connections; + std::function()> create_connections; + std::unique_ptr connections; const String query; String query_id = ""; diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp index bc47b049407..c854794cd27 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -3,7 +3,7 @@ #include #include #include -#include +#include #include namespace DB @@ -11,7 +11,7 @@ namespace DB struct RemoteQueryExecutorRoutine { - MultiplexedConnections & connections; + IConnections & connections; RemoteQueryExecutorReadContext & read_context; struct ReadCallback @@ -19,15 +19,15 @@ struct RemoteQueryExecutorRoutine RemoteQueryExecutorReadContext & read_context; Fiber & fiber; - void operator()(Poco::Net::Socket & socket) + void operator()(int fd, const Poco::Timespan & timeout = 0, const std::string fd_description = "") { try { - read_context.setSocket(socket); + read_context.setConnectionFD(fd, timeout, fd_description); } catch (DB::Exception & e) { - e.addMessage(" while reading from socket ({})", socket.peerAddress().toString()); + e.addMessage(" while reading from {}", fd_description); throw; } @@ -70,60 +70,38 @@ namespace ErrorCodes extern const int SOCKET_TIMEOUT; } -RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(MultiplexedConnections & connections_) +RemoteQueryExecutorReadContext::RemoteQueryExecutorReadContext(IConnections & connections_) : connections(connections_) { - epoll_fd = epoll_create(2); - if (-1 == epoll_fd) - throwFromErrno("Cannot create epoll descriptor", ErrorCodes::CANNOT_OPEN_FILE); if (-1 == pipe2(pipe_fd, O_NONBLOCK)) throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE); { - epoll_event socket_event; - socket_event.events = EPOLLIN | EPOLLPRI; - socket_event.data.fd = pipe_fd[0]; - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, pipe_fd[0], &socket_event)) - throwFromErrno("Cannot add pipe descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + epoll.add(pipe_fd[0]); } { - epoll_event timer_event; - timer_event.events = EPOLLIN | EPOLLPRI; - timer_event.data.fd = timer.getDescriptor(); - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, timer_event.data.fd, &timer_event)) - throwFromErrno("Cannot add timer descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + epoll.add(timer.getDescriptor()); } auto routine = RemoteQueryExecutorRoutine{connections, *this}; fiber = boost::context::fiber(std::allocator_arg_t(), stack, std::move(routine)); } -void RemoteQueryExecutorReadContext::setSocket(Poco::Net::Socket & socket) +void RemoteQueryExecutorReadContext::setConnectionFD(int fd, const Poco::Timespan & timeout, const std::string & fd_description) { - int fd = socket.impl()->sockfd(); - if (fd == socket_fd) + if (fd == connection_fd) return; - epoll_event socket_event; - socket_event.events = EPOLLIN | EPOLLPRI; - socket_event.data.fd = fd; + if (connection_fd != -1) + epoll.remove(connection_fd); - if (socket_fd != -1) - { - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_DEL, socket_fd, &socket_event)) - throwFromErrno("Cannot remove socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); - } + connection_fd = fd; + epoll.add(connection_fd); - socket_fd = fd; - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, socket_fd, &socket_event)) - throwFromErrno("Cannot add socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); - - receive_timeout = socket.impl()->getReceiveTimeout(); + receive_timeout = timeout; + connection_fd_description = fd_description; } bool RemoteQueryExecutorReadContext::checkTimeout() const @@ -142,21 +120,16 @@ bool RemoteQueryExecutorReadContext::checkTimeout() const bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const { - epoll_event events[3]; - events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - - /// Wait for epoll_fd will not block if it was polled externally. - int num_events = epoll_wait(epoll_fd, events, 3, 0); - if (num_events == -1) - throwFromErrno("Failed to epoll_wait", ErrorCodes::CANNOT_READ_FROM_SOCKET); + /// Wait for epoll will not block if it was polled externally. + std::vector events = epoll.getManyReady(epoll.size(), /* blocking = */ false); bool is_socket_ready = false; bool is_pipe_alarmed = false; bool has_timer_alarm = false; - for (int i = 0; i < num_events; ++i) + for (size_t i = 0; i < events.size(); ++i) { - if (events[i].data.fd == socket_fd) + if (events[i].data.fd == connection_fd) is_socket_ready = true; if (events[i].data.fd == timer.getDescriptor()) has_timer_alarm = true; @@ -225,9 +198,7 @@ void RemoteQueryExecutorReadContext::cancel() RemoteQueryExecutorReadContext::~RemoteQueryExecutorReadContext() { - /// socket_fd is closed by Poco::Net::Socket - if (epoll_fd != -1) - close(epoll_fd); + /// connection_fd is closed by Poco::Net::Socket if (pipe_fd[0] != -1) close(pipe_fd[0]); if (pipe_fd[1] != -1) diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.h b/src/DataStreams/RemoteQueryExecutorReadContext.h index 89dc2813a9a..cb6421f78d0 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.h +++ b/src/DataStreams/RemoteQueryExecutorReadContext.h @@ -7,7 +7,9 @@ #include #include #include +#include #include +#include #include namespace Poco::Net @@ -33,26 +35,29 @@ public: std::mutex fiber_lock; Poco::Timespan receive_timeout; - MultiplexedConnections & connections; + IConnections & connections; Poco::Net::Socket * last_used_socket = nullptr; /// Here we have three descriptors we are going to wait: - /// * socket_fd is a descriptor of connection. It may be changed in case of reading from several replicas. + /// * connection_fd is a descriptor of connection. It may be changed in case of reading from several replicas. /// * timer is a timerfd descriptor to manually check socket timeout /// * pipe_fd is a pipe we use to cancel query and socket polling by executor. - /// We put those descriptors into our own epoll_fd which is used by external executor. + /// We put those descriptors into our own epoll which is used by external executor. TimerDescriptor timer{CLOCK_MONOTONIC, 0}; - int socket_fd = -1; - int epoll_fd = -1; + int connection_fd = -1; int pipe_fd[2] = { -1, -1 }; - explicit RemoteQueryExecutorReadContext(MultiplexedConnections & connections_); + Epoll epoll; + + std::string connection_fd_description; + + explicit RemoteQueryExecutorReadContext(IConnections & connections_); ~RemoteQueryExecutorReadContext(); bool checkTimeout() const; bool checkTimeoutImpl() const; - void setSocket(Poco::Net::Socket & socket); + void setConnectionFD(int fd, const Poco::Timespan & timeout = 0, const std::string & fd_description = ""); void setTimer() const; bool resumeRoutine(); diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index e5efabee6e2..01f31d6efa8 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -17,6 +17,11 @@ struct ConnectionTimeouts Poco::Timespan http_keep_alive_timeout; Poco::Timespan secure_connection_timeout; + /// Timeouts for HedgedConnections + Poco::Timespan receive_hello_timeout; + Poco::Timespan receive_tables_status_timeout; + Poco::Timespan receive_data_timeout; + ConnectionTimeouts() = default; ConnectionTimeouts(const Poco::Timespan & connection_timeout_, @@ -27,7 +32,10 @@ struct ConnectionTimeouts receive_timeout(receive_timeout_), tcp_keep_alive_timeout(0), http_keep_alive_timeout(0), - secure_connection_timeout(connection_timeout) + secure_connection_timeout(connection_timeout), + receive_hello_timeout(0), + receive_tables_status_timeout(0), + receive_data_timeout(0) { } @@ -40,7 +48,10 @@ struct ConnectionTimeouts receive_timeout(receive_timeout_), tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(0), - secure_connection_timeout(connection_timeout) + secure_connection_timeout(connection_timeout), + receive_hello_timeout(0), + receive_tables_status_timeout(0), + receive_data_timeout(0) { } ConnectionTimeouts(const Poco::Timespan & connection_timeout_, @@ -53,7 +64,10 @@ struct ConnectionTimeouts receive_timeout(receive_timeout_), tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(http_keep_alive_timeout_), - secure_connection_timeout(connection_timeout) + secure_connection_timeout(connection_timeout), + receive_hello_timeout(0), + receive_tables_status_timeout(0), + receive_data_timeout(0) { } @@ -62,13 +76,19 @@ struct ConnectionTimeouts const Poco::Timespan & receive_timeout_, const Poco::Timespan & tcp_keep_alive_timeout_, const Poco::Timespan & http_keep_alive_timeout_, - const Poco::Timespan & secure_connection_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_) + const Poco::Timespan & secure_connection_timeout_, + const Poco::Timespan & receive_hello_timeout_, + const Poco::Timespan & receive_tables_status_timeout_, + const Poco::Timespan & receive_data_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_), + receive_hello_timeout(receive_hello_timeout_), + receive_tables_status_timeout(receive_tables_status_timeout_), + receive_data_timeout(receive_data_timeout_) { } @@ -87,7 +107,10 @@ struct ConnectionTimeouts saturate(receive_timeout, limit), saturate(tcp_keep_alive_timeout, limit), saturate(http_keep_alive_timeout, limit), - saturate(secure_connection_timeout, limit)); + saturate(secure_connection_timeout, limit), + saturate(receive_hello_timeout, limit), + saturate(receive_tables_status_timeout, limit), + saturate(receive_data_timeout, limit)); } /// Timeouts for the case when we have just single attempt to connect. diff --git a/src/IO/ConnectionTimeoutsContext.h b/src/IO/ConnectionTimeoutsContext.h index ce19738f507..c6daae39e7a 100644 --- a/src/IO/ConnectionTimeoutsContext.h +++ b/src/IO/ConnectionTimeoutsContext.h @@ -16,7 +16,16 @@ inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(cons /// Timeouts for the case when we will try many addresses in a loop. inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings & settings) { - return ConnectionTimeouts(settings.connect_timeout_with_failover_ms, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout, 0, settings.connect_timeout_with_failover_secure_ms); + return ConnectionTimeouts( + settings.connect_timeout_with_failover_ms, + settings.send_timeout, + settings.receive_timeout, + settings.tcp_keep_alive_timeout, + 0, + settings.connect_timeout_with_failover_secure_ms, + settings.receive_hello_timeout, + settings.receive_tables_status_timeout, + settings.receive_data_timeout); } inline ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Context & context) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 2c13446e693..e08b9e7c8fb 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -42,7 +42,7 @@ bool ReadBufferFromPocoSocket::nextImpl() /// Note that receive timeout is not checked here. External code should check it while polling. while (bytes_read < 0 && async_callback && errno == EAGAIN) { - async_callback(socket); + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), "socket (" + socket.peerAddress().toString() + ")"); bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), internal_buffer.size(), flags); } } diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 8064cd39246..7fd1b646846 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -8,6 +8,8 @@ namespace DB { +using AsyncCallback = std::function; + /** Works with the ready Poco::Net::Socket. Blocking operations. */ class ReadBufferFromPocoSocket : public BufferWithOwnMemory @@ -28,10 +30,10 @@ public: bool poll(size_t timeout_microseconds); - void setAsyncCallback(std::function async_callback_) { async_callback = std::move(async_callback_); } + void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } private: - std::function async_callback; + AsyncCallback async_callback; }; } diff --git a/src/Processors/Executors/PollingQueue.cpp b/src/Processors/Executors/PollingQueue.cpp index 93edfe53987..44941ae788a 100644 --- a/src/Processors/Executors/PollingQueue.cpp +++ b/src/Processors/Executors/PollingQueue.cpp @@ -23,24 +23,14 @@ namespace ErrorCodes PollingQueue::PollingQueue() { - epoll_fd = epoll_create(1); - if (-1 == epoll_fd) - throwFromErrno("Cannot create epoll descriptor", ErrorCodes::CANNOT_OPEN_FILE); - if (-1 == pipe2(pipe_fd, O_NONBLOCK)) throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE); - epoll_event socket_event; - socket_event.events = EPOLLIN | EPOLLPRI; - socket_event.data.ptr = pipe_fd; - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, pipe_fd[0], &socket_event)) - throwFromErrno("Cannot add pipe descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + epoll.add(pipe_fd[0], pipe_fd); } PollingQueue::~PollingQueue() { - close(epoll_fd); close(pipe_fd[0]); close(pipe_fd[1]); } @@ -52,13 +42,7 @@ void PollingQueue::addTask(size_t thread_number, void * data, int fd) throw Exception(ErrorCodes::LOGICAL_ERROR, "Task {} was already added to task queue", key); tasks[key] = TaskData{thread_number, data, fd}; - - epoll_event socket_event; - socket_event.events = EPOLLIN | EPOLLPRI; - socket_event.data.ptr = data; - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &socket_event)) - throwFromErrno("Cannot add socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + epoll.add(fd, data); } static std::string dumpTasks(const std::unordered_map & tasks) @@ -84,16 +68,7 @@ PollingQueue::TaskData PollingQueue::wait(std::unique_lock & lock) lock.unlock(); - epoll_event event; - event.data.ptr = nullptr; - int num_events = 0; - - while (num_events == 0) - { - num_events = epoll_wait(epoll_fd, &event, 1, 0); - if (num_events == -1) - throwFromErrno("Failed to epoll_wait", ErrorCodes::CANNOT_READ_FROM_SOCKET); - } + epoll_event event = epoll.getReady(); lock.lock(); @@ -111,9 +86,7 @@ PollingQueue::TaskData PollingQueue::wait(std::unique_lock & lock) auto res = it->second; tasks.erase(it); - - if (-1 == epoll_ctl(epoll_fd, EPOLL_CTL_DEL, res.fd, &event)) - throwFromErrno("Cannot remove socket descriptor to epoll", ErrorCodes::CANNOT_OPEN_FILE); + epoll.remove(res.fd); return res; } diff --git a/src/Processors/Executors/PollingQueue.h b/src/Processors/Executors/PollingQueue.h index 9d37bf0a2cc..0d306ddf2f7 100644 --- a/src/Processors/Executors/PollingQueue.h +++ b/src/Processors/Executors/PollingQueue.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -25,7 +26,7 @@ public: }; private: - int epoll_fd; + Epoll epoll; int pipe_fd[2]; std::atomic_bool is_finished = false; std::unordered_map tasks; diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 12d1a0249b7..b6298f46dc7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -681,6 +681,15 @@ void TCPHandler::processTablesStatusRequest() response.table_states_by_id.emplace(table_name, std::move(status)); } + /// For testing hedged requests + const Settings & settings = query_context->getSettingsRef(); + if (settings.sleep_before_send_tables_status) + { + std::chrono::seconds sec(settings.sleep_before_send_tables_status); + std::this_thread::sleep_for(sec); + } + + writeVarUInt(Protocol::Server::TablesStatusResponse, *out); response.write(*out, client_tcp_protocol_version); } @@ -881,6 +890,14 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { + /// For testing hedged requests + const Settings & settings = query_context->getSettingsRef(); + if (settings.sleep_before_send_hello) + { + std::chrono::seconds sec(settings.sleep_before_send_hello); + std::this_thread::sleep_for(sec); + } + writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); @@ -1313,6 +1330,14 @@ bool TCPHandler::isQueryCancelled() void TCPHandler::sendData(const Block & block) { + /// For testing hedged requests + const Settings & settings = query_context->getSettingsRef(); + if (settings.sleep_before_send_data) + { + std::chrono::seconds sec(settings.sleep_before_send_data); + std::this_thread::sleep_for(sec); + } + initBlockOutput(block); writeVarUInt(Protocol::Server::Data, *out); diff --git a/tests/integration/test_hedged_requests/configs/remote_servers.xml b/tests/integration/test_hedged_requests/configs/remote_servers.xml new file mode 100644 index 00000000000..60d2f5891d7 --- /dev/null +++ b/tests/integration/test_hedged_requests/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + + true + + node_1 + 9000 + + + node_2 + 9000 + + + + + + diff --git a/tests/integration/test_hedged_requests/configs/users.xml b/tests/integration/test_hedged_requests/configs/users.xml new file mode 100644 index 00000000000..0cf32bf9e1a --- /dev/null +++ b/tests/integration/test_hedged_requests/configs/users.xml @@ -0,0 +1,10 @@ + + + + + in_order + 1 + 1 + + + diff --git a/tests/integration/test_hedged_requests/configs/users1.xml b/tests/integration/test_hedged_requests/configs/users1.xml new file mode 100644 index 00000000000..2a54396feca --- /dev/null +++ b/tests/integration/test_hedged_requests/configs/users1.xml @@ -0,0 +1,7 @@ + + + + + + + diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py new file mode 100644 index 00000000000..719477d9c7f --- /dev/null +++ b/tests/integration/test_hedged_requests/test.py @@ -0,0 +1,76 @@ +import os +import sys +import time + +import pytest + +sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +cluster = ClickHouseCluster(__file__) + +# Cluster with 1 shard of 2 replicas. node is the instance with Distributed table. +node = cluster.add_instance( + 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) +node_1 = cluster.add_instance('node_1', with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) +node_2 = cluster.add_instance('node_2', with_zookeeper=True) + +sleep_timeout = 5 +receive_timeout = 1 + +config = ''' + + + <{setting}>{sleep} + + +''' + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node_1.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_1') ORDER BY id PARTITION BY toYYYYMM(date)''') + + node_2.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_2') ORDER BY id PARTITION BY toYYYYMM(date)''') + + node.query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = + Distributed('test_cluster', 'default', 'replicated')''') + + yield cluster + + finally: + cluster.shutdown() + +def process_test(sleep_setting_name, receive_timeout_name): + node_1.replace_config('/etc/clickhouse-server/users.d/users1.xml', config.format(setting=sleep_setting_name, sleep=sleep_timeout)) + + # Restart node to make new config relevant + node_1.restart_clickhouse(sleep_timeout + 1) + + # Without hedged requests select query will last more than sleep_timeout seconds, + # with hedged requests it will last just over receive_timeout seconds + + node.query("SET {setting}={value}".format(setting=receive_timeout_name, value=receive_timeout)) + + start = time.time() + node.query("SELECT * FROM distributed"); + query_time = time.time() - start + + # Check that query time is not long + assert query_time < sleep_timeout + + +def test_change_replica_on_receive_hello(started_cluster): + node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") + + process_test("sleep_before_send_hello", "receive_hello_timeout") + process_test("sleep_before_send_tables_status", "receive_tables_status_timeout") + process_test("sleep_before_send_data", "receive_data_timeout") + From 507695cbcbeaf08af3c3d240b62a2a73000001c6 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 20 Jan 2021 02:15:13 +0300 Subject: [PATCH 002/510] Fix build --- src/Client/HedgedConnections.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 57315bcd6fe..4963c74c327 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -29,7 +29,7 @@ HedgedConnections::HedgedConnections( if (replicas.second_replica->isNotReady()) epoll.add(get_hedged_connections.getFileDescriptor()); - auto set_throttler = [this, throttler_](ReplicaStatePtr replica) + auto set_throttler = [throttler_](ReplicaStatePtr replica) { replica->connection->setThrottler(throttler_); }; From 2aa29e1bc7f9fe1a9ccfa96ed96db87ac8a6cd95 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 20 Jan 2021 13:29:31 +0300 Subject: [PATCH 003/510] Fix build 2 --- src/Client/ConnectionPoolWithFailover.cpp | 4 ++-- src/Client/GetHedgedConnections.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index df4541ecf7e..00ec1e30f10 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -472,11 +472,11 @@ void TryGetConnection::run() stage = Stage::FINISHED; } - catch (Poco::Net::NetException & e) + catch (Poco::Net::NetException &) { processFail(true); } - catch (Poco::TimeoutException & e) + catch (Poco::TimeoutException &) { processFail(true); } diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 3b30650e6e5..839d6bf37c2 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -319,7 +319,7 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(b { LOG_DEBUG(log, "process epoll events"); int event_fd; - ReplicaStatePtr replica; + ReplicaStatePtr replica = nullptr; bool finish = false; while (!finish) { From 4f6c880232cac8e9d9c6e0c79111de5a9fed8c91 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Mon, 25 Jan 2021 18:31:59 +0400 Subject: [PATCH 004/510] Pass and handle a chain of multiple prewhere infos --- src/Interpreters/InterpreterSelectQuery.cpp | 72 ++++++---- .../getHeaderForProcessingStage.cpp | 11 +- .../MergeTreeBaseSelectProcessor.cpp | 41 +++--- .../MergeTree/MergeTreeBaseSelectProcessor.h | 8 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 19 +-- .../MergeTree/MergeTreeBlockReadUtils.h | 7 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 30 ++-- .../MergeTree/MergeTreeRangeReader.cpp | 136 ++++++++++++------ src/Storages/MergeTree/MergeTreeRangeReader.h | 13 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 8 +- src/Storages/MergeTree/MergeTreeReadPool.h | 9 +- .../MergeTreeReverseSelectProcessor.cpp | 11 +- .../MergeTreeReverseSelectProcessor.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 11 +- .../MergeTree/MergeTreeSelectProcessor.h | 2 +- ...rgeTreeThreadSelectBlockInputProcessor.cpp | 8 +- ...MergeTreeThreadSelectBlockInputProcessor.h | 2 +- src/Storages/SelectQueryInfo.h | 6 +- src/Storages/StorageBuffer.cpp | 25 ++-- 19 files changed, 247 insertions(+), 174 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 4f6b76e9b53..9dd63362dbd 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1186,36 +1186,40 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { Pipe pipe(std::make_shared(source_header)); - if (query_info.prewhere_info) + if (query_info.prewhere_info_list) { - if (query_info.prewhere_info->alias_actions) + for (const auto & prewhere_info : *query_info.prewhere_info_list) { + if (prewhere_info.alias_actions) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, prewhere_info.alias_actions); + }); + } + pipe.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->alias_actions); + return std::make_shared( + header, + prewhere_info.prewhere_actions, + prewhere_info.prewhere_column_name, + prewhere_info.remove_prewhere_column); }); - } - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - query_info.prewhere_info->prewhere_actions, - query_info.prewhere_info->prewhere_column_name, - query_info.prewhere_info->remove_prewhere_column); - }); - - // To remove additional columns - // In some cases, we did not read any marks so that the pipeline.streams is empty - // Thus, some columns in prewhere are not removed as expected - // This leads to mismatched header in distributed table - if (query_info.prewhere_info->remove_columns_actions) - { - pipe.addSimpleTransform([&](const Block & header) + // To remove additional columns + // In some cases, we did not read any marks so that the pipeline.streams is empty + // Thus, some columns in prewhere are not removed as expected + // This leads to mismatched header in distributed table + if (prewhere_info.remove_columns_actions) { - return std::make_shared( - header, query_info.prewhere_info->remove_columns_actions); - }); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, prewhere_info.remove_columns_actions); + }); + } } } @@ -1552,17 +1556,23 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { - query_info.prewhere_info = std::make_shared( - std::make_shared(prewhere_info->prewhere_actions), - prewhere_info->prewhere_column_name); + if (!query_info.prewhere_info_list) + query_info.prewhere_info_list = std::make_shared(); + + query_info.prewhere_info_list->emplace_back( + std::make_shared(prewhere_info->prewhere_actions), + prewhere_info->prewhere_column_name); + + auto & new_prewhere_info = query_info.prewhere_info_list->back(); if (prewhere_info->alias_actions) - query_info.prewhere_info->alias_actions = std::make_shared(prewhere_info->alias_actions); - if (prewhere_info->remove_columns_actions) - query_info.prewhere_info->remove_columns_actions = std::make_shared(prewhere_info->remove_columns_actions); + new_prewhere_info.alias_actions = std::make_shared(prewhere_info->alias_actions); - query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column; - query_info.prewhere_info->need_filter = prewhere_info->need_filter; + if (prewhere_info->remove_columns_actions) + new_prewhere_info.remove_columns_actions = std::make_shared(prewhere_info->remove_columns_actions); + + new_prewhere_info.remove_prewhere_column = prewhere_info->remove_prewhere_column; + new_prewhere_info.need_filter = prewhere_info->need_filter; } /// Create optimizer with prepared actions. diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index e341a5637f4..761f04e81ee 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -42,11 +42,14 @@ Block getHeaderForProcessingStage( case QueryProcessingStage::FetchColumns: { Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); - if (query_info.prewhere_info) + if (query_info.prewhere_info_list) { - query_info.prewhere_info->prewhere_actions->execute(header); - if (query_info.prewhere_info->remove_prewhere_column) - header.erase(query_info.prewhere_info->prewhere_column_name); + for (const auto & prewhere_info : *query_info.prewhere_info_list) + { + prewhere_info.prewhere_actions->execute(header); + if (prewhere_info.remove_prewhere_column) + header.erase(prewhere_info.prewhere_column_name); + } } return header; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index c852151f27d..3405a211c98 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -22,17 +22,17 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( Block header, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, UInt64 preferred_max_column_in_block_size_bytes_, const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, const Names & virt_column_names_) - : SourceWithProgress(getHeader(std::move(header), prewhere_info_, virt_column_names_)) + : SourceWithProgress(getHeader(std::move(header), prewhere_info_list_, virt_column_names_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) - , prewhere_info(prewhere_info_) + , prewhere_info_list(prewhere_info_list_) , max_block_size_rows(max_block_size_rows_) , preferred_block_size_bytes(preferred_block_size_bytes_) , preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_) @@ -70,18 +70,18 @@ Chunk MergeTreeBaseSelectProcessor::generate() void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & current_task) { - if (prewhere_info) + if (prewhere_info_list) { if (reader->getColumns().empty()) { - current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, true); + current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info_list, true); } else { MergeTreeRangeReader * pre_reader_ptr = nullptr; if (pre_reader != nullptr) { - current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, false); + current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info_list, false); pre_reader_ptr = ¤t_task.pre_range_reader; } @@ -309,34 +309,37 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns(Chunk & chunk, MergeTree chunk.setColumns(columns, num_rows); } -void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info) +void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const PrewhereInfoListPtr & prewhere_info_list) { - if (prewhere_info) - { - if (prewhere_info->alias_actions) - prewhere_info->alias_actions->execute(block); + if (!prewhere_info_list) + return; - prewhere_info->prewhere_actions->execute(block); - auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); + for (const auto & prewhere_info : *prewhere_info_list) + { + if (prewhere_info.alias_actions) + prewhere_info.alias_actions->execute(block); + + prewhere_info.prewhere_actions->execute(block); + auto & prewhere_column = block.getByName(prewhere_info.prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) throw Exception("Invalid type for filter in PREWHERE: " + prewhere_column.type->getName(), - ErrorCodes::LOGICAL_ERROR); + ErrorCodes::LOGICAL_ERROR); - if (prewhere_info->remove_prewhere_column) - block.erase(prewhere_info->prewhere_column_name); + if (prewhere_info.remove_prewhere_column) + block.erase(prewhere_info.prewhere_column_name); else { - auto & ctn = block.getByName(prewhere_info->prewhere_column_name); + auto & ctn = block.getByName(prewhere_info.prewhere_column_name); ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); } } } Block MergeTreeBaseSelectProcessor::getHeader( - Block block, const PrewhereInfoPtr & prewhere_info, const Names & virtual_columns) + Block block, const PrewhereInfoListPtr & prewhere_info_list, const Names & virtual_columns) { - executePrewhereActions(block, prewhere_info); + executePrewhereActions(block, prewhere_info_list); injectVirtualColumns(block, nullptr, virtual_columns); return block; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 00ef131ae45..a3d7520b89a 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -23,7 +23,7 @@ public: Block header, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, UInt64 preferred_max_column_in_block_size_bytes_, @@ -33,7 +33,7 @@ public: ~MergeTreeBaseSelectProcessor() override; - static void executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info); + static void executePrewhereActions(Block & block, const PrewhereInfoListPtr & prewhere_info_list); protected: Chunk generate() final; @@ -49,7 +49,7 @@ protected: static void injectVirtualColumns(Block & block, MergeTreeReadTask * task, const Names & virtual_columns); static void injectVirtualColumns(Chunk & chunk, MergeTreeReadTask * task, const Names & virtual_columns); - static Block getHeader(Block block, const PrewhereInfoPtr & prewhere_info, const Names & virtual_columns); + static Block getHeader(Block block, const PrewhereInfoListPtr & prewhere_info_list, const Names & virtual_columns); void initializeRangeReaders(MergeTreeReadTask & task); @@ -57,7 +57,7 @@ protected: const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; - PrewhereInfoPtr prewhere_info; + PrewhereInfoListPtr prewhere_info_list; UInt64 max_block_size_rows; UInt64 preferred_block_size_bytes; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f8b5e0a9c0a..f3191a76120 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -118,11 +118,10 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada MergeTreeReadTask::MergeTreeReadTask( const MergeTreeData::DataPartPtr & data_part_, const MarkRanges & mark_ranges_, const size_t part_index_in_query_, const Names & ordered_names_, const NameSet & column_name_set_, const NamesAndTypesList & columns_, - const NamesAndTypesList & pre_columns_, const bool remove_prewhere_column_, const bool should_reorder_, - MergeTreeBlockSizePredictorPtr && size_predictor_) + const NamesAndTypesList & pre_columns_, const bool should_reorder_, MergeTreeBlockSizePredictorPtr && size_predictor_) : data_part{data_part_}, mark_ranges{mark_ranges_}, part_index_in_query{part_index_in_query_}, ordered_names{ordered_names_}, column_name_set{column_name_set_}, columns{columns_}, pre_columns{pre_columns_}, - remove_prewhere_column{remove_prewhere_column_}, should_reorder{should_reorder_}, size_predictor{std::move(size_predictor_)} + should_reorder{should_reorder_}, size_predictor{std::move(size_predictor_)} { } @@ -258,7 +257,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, - const PrewhereInfoPtr & prewhere_info, + const PrewhereInfoListPtr & prewhere_info_list, bool check_columns) { Names column_names = required_columns; @@ -267,12 +266,14 @@ MergeTreeReadTaskColumns getReadTaskColumns( /// inject columns required for defaults evaluation bool should_reorder = !injectRequiredColumns(storage, metadata_snapshot, data_part, column_names).empty(); - if (prewhere_info) + if (prewhere_info_list) { - if (prewhere_info->alias_actions) - pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); - else - pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); + for (const auto & prewhere_info : *prewhere_info_list) + { + const auto required_column_names = (prewhere_info.alias_actions ? + prewhere_info.alias_actions->getRequiredColumns() : prewhere_info.prewhere_actions->getRequiredColumns()); + pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); + } if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index 31d609e4242..f2537c554c3 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -42,8 +42,6 @@ struct MergeTreeReadTask const NamesAndTypesList & columns; /// column names to read during PREWHERE const NamesAndTypesList & pre_columns; - /// should PREWHERE column be returned to requesting side? - const bool remove_prewhere_column; /// resulting block may require reordering in accordance with `ordered_names` const bool should_reorder; /// Used to satistfy preferred_block_size_bytes limitation @@ -57,8 +55,7 @@ struct MergeTreeReadTask MergeTreeReadTask( const MergeTreeData::DataPartPtr & data_part_, const MarkRanges & mark_ranges_, const size_t part_index_in_query_, const Names & ordered_names_, const NameSet & column_name_set_, const NamesAndTypesList & columns_, - const NamesAndTypesList & pre_columns_, const bool remove_prewhere_column_, const bool should_reorder_, - MergeTreeBlockSizePredictorPtr && size_predictor_); + const NamesAndTypesList & pre_columns_, const bool should_reorder_, MergeTreeBlockSizePredictorPtr && size_predictor_); virtual ~MergeTreeReadTask(); }; @@ -78,7 +75,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, - const PrewhereInfoPtr & prewhere_info, + const PrewhereInfoListPtr & prewhere_info_list, bool check_columns); struct MergeTreeBlockSizePredictor diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 4e1f307137a..248efc140fd 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -833,14 +833,20 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( plan->addStep(std::move(adding_column)); } - if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) + if (query_info.prewhere_info_list) { - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); + for (const auto & prewhere_info : *query_info.prewhere_info_list) + { + if (prewhere_info.remove_columns_actions) + { + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + prewhere_info.remove_columns_actions->getActionsDAG().clone()); - expression_step->setStepDescription("Remove unused columns after PREWHERE"); - plan->addStep(std::move(expression_step)); + expression_step->setStepDescription("Remove unused columns after PREWHERE"); + plan->addStep(std::move(expression_step)); + } + } } return plan; @@ -948,7 +954,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( std::move(parts), data, metadata_snapshot, - query_info.prewhere_info, + query_info.prewhere_info_list, true, column_names, MergeTreeReadPool::BackoffSettings(settings), @@ -964,7 +970,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, metadata_snapshot, use_uncompressed_cache, - query_info.prewhere_info, reader_settings, virt_columns); + query_info.prewhere_info_list, reader_settings, virt_columns); if (i == 0) { @@ -987,7 +993,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( auto source = std::make_shared( data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache, - query_info.prewhere_info, true, reader_settings, virt_columns, part.part_index_in_query); + query_info.prewhere_info_list, true, reader_settings, virt_columns, part.part_index_in_query); res.emplace_back(std::move(source)); } @@ -1187,7 +1193,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( column_names, ranges_to_get_from_part, use_uncompressed_cache, - query_info.prewhere_info, + query_info.prewhere_info_list, true, reader_settings, virt_columns, @@ -1205,7 +1211,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( column_names, ranges_to_get_from_part, use_uncompressed_cache, - query_info.prewhere_info, + query_info.prewhere_info_list, true, reader_settings, virt_columns, @@ -1359,7 +1365,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( column_names, part_it->ranges, use_uncompressed_cache, - query_info.prewhere_info, + query_info.prewhere_info_list, true, reader_settings, virt_columns, diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index c13146bd35c..2ca2b30a5eb 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -443,32 +443,79 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con return count; } -void MergeTreeRangeReader::ReadResult::setFilter(const ColumnPtr & new_filter) +void MergeTreeRangeReader::ReadResult::addFilter(const ColumnPtr & new_filter) { - if (!new_filter && filter) - throw Exception("Can't replace existing filter with empty.", ErrorCodes::LOGICAL_ERROR); - if (filter) { - size_t new_size = new_filter->size(); + if (!new_filter) + throw Exception("Can't add an empty filter to the existing one.", ErrorCodes::LOGICAL_ERROR); + const auto new_size = new_filter->size(); if (new_size != total_rows_per_granule) - throw Exception("Can't set filter because it's size is " + toString(new_size) + " but " + throw Exception("Can't add the new filter because it's size is " + toString(new_size) + " but " + toString(total_rows_per_granule) + " rows was read.", ErrorCodes::LOGICAL_ERROR); } ConstantFilterDescription const_description(*new_filter); if (const_description.always_true) - setFilterConstTrue(); + { + if (!filter) + setFilterConstTrue(); + } else if (const_description.always_false) + { clear(); + } else { - FilterDescription filter_description(*new_filter); - filter_holder = filter_description.data_holder ? filter_description.data_holder : new_filter; - filter = typeid_cast(filter_holder.get()); - if (!filter) - throw Exception("setFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); + FilterDescription description(*new_filter); + auto new_holder = (description.data_holder ? description.data_holder : new_filter); + auto * new_holder_cast = typeid_cast(new_holder.get()); + + if (!new_holder_cast) + throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); + + if (filter) + { + MutableColumnPtr new_mutable_holder = IColumn::mutate(std::move(new_holder)); + auto * new_mutable_holder_cast = typeid_cast(new_mutable_holder.get()); + + if (!new_mutable_holder_cast) + throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); + + const auto & data = filter->getData(); + auto it = data.begin(); + + auto & new_data = new_mutable_holder_cast->getData(); + auto n_it = new_data.begin(); + + while (it != data.end() && n_it != new_data.end()) + { + *n_it = (*n_it && *it); + ++it; + ++n_it; + } + + ConstantFilterDescription new_const_description(*new_mutable_holder); + if (new_const_description.always_true) + { + setFilterConstTrue(); + } + else if (new_const_description.always_false) + { + clear(); + } + else + { + filter_holder = std::move(new_mutable_holder); + filter = new_mutable_holder_cast; + } + } + else + { + filter_holder = std::move(new_holder); + filter = new_holder_cast; + } } } @@ -489,11 +536,14 @@ size_t MergeTreeRangeReader::ReadResult::countBytesInResultFilter(const IColumn: MergeTreeRangeReader::MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - const PrewhereInfoPtr & prewhere_, + const PrewhereInfoListPtr & prewhere_info_list_, bool last_reader_in_chain_) : merge_tree_reader(merge_tree_reader_) - , index_granularity(&(merge_tree_reader->data_part->index_granularity)), prev_reader(prev_reader_) - , prewhere(prewhere_), last_reader_in_chain(last_reader_in_chain_), is_initialized(true) + , index_granularity(&(merge_tree_reader->data_part->index_granularity)) + , prev_reader(prev_reader_) + , prewhere_info_list(prewhere_info_list_) + , last_reader_in_chain(last_reader_in_chain_) + , is_initialized(true) { if (prev_reader) sample_block = prev_reader->getSampleBlock(); @@ -501,16 +551,19 @@ MergeTreeRangeReader::MergeTreeRangeReader( for (const auto & name_and_type : merge_tree_reader->getColumns()) sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); - if (prewhere) + if (prewhere_info_list) { - if (prewhere->alias_actions) - prewhere->alias_actions->execute(sample_block, true); + for (const auto & prewhere_info : *prewhere_info_list) + { + if (prewhere_info.alias_actions) + prewhere_info.alias_actions->execute(sample_block, true); - if (prewhere->prewhere_actions) - prewhere->prewhere_actions->execute(sample_block, true); + if (prewhere_info.prewhere_actions) + prewhere_info.prewhere_actions->execute(sample_block, true); - if (prewhere->remove_prewhere_column) - sample_block.erase(prewhere->prewhere_column_name); + if (prewhere_info.remove_prewhere_column) + sample_block.erase(prewhere_info.prewhere_column_name); + } } } @@ -701,7 +754,13 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (read_result.num_rows == 0) return read_result; - executePrewhereActionsAndFilterColumns(read_result); + if (prewhere_info_list) + { + for (const auto & prewhere_info : *prewhere_info_list) + { + executePrewhereActionsAndFilterColumns(read_result, prewhere_info); + } + } return read_result; } @@ -798,11 +857,8 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & return columns; } -void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) +void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result, const PrewhereInfo & prewhere_info) { - if (!prewhere) - return; - const auto & header = merge_tree_reader->getColumns(); size_t num_columns = header.size(); @@ -831,14 +887,14 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type) block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - if (prewhere->alias_actions) - prewhere->alias_actions->execute(block); + if (prewhere_info.alias_actions) + prewhere_info.alias_actions->execute(block); /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; - prewhere->prewhere_actions->execute(block); + prewhere_info.prewhere_actions->execute(block); - prewhere_column_pos = block.getPositionByName(prewhere->prewhere_column_name); + prewhere_column_pos = block.getPositionByName(prewhere_info.prewhere_column_name); result.columns.clear(); result.columns.reserve(block.columns()); @@ -848,15 +904,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r filter.swap(result.columns[prewhere_column_pos]); } - if (result.getFilter()) - { - /// TODO: implement for prewhere chain. - /// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter. - throw Exception("MergeTreeRangeReader chain with several prewhere actions in not implemented.", - ErrorCodes::LOGICAL_ERROR); - } - - result.setFilter(filter); + result.addFilter(filter); /// If there is a WHERE, we filter in there, and only optimize IO and shrink columns here if (!last_reader_in_chain) @@ -866,7 +914,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.totalRowsPerGranule() == 0) result.setFilterConstFalse(); /// If we need to filter in PREWHERE - else if (prewhere->need_filter || result.need_filter) + else if (prewhere_info.need_filter || result.need_filter) { /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) @@ -907,11 +955,11 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Check if the PREWHERE column is needed if (!result.columns.empty()) { - if (prewhere->remove_prewhere_column) + if (prewhere_info.remove_prewhere_column) result.columns.erase(result.columns.begin() + prewhere_column_pos); else result.columns[prewhere_column_pos] = - getSampleBlock().getByName(prewhere->prewhere_column_name).type-> + getSampleBlock().getByName(prewhere_info.prewhere_column_name).type-> createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } @@ -919,7 +967,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r else { result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); - if (getSampleBlock().getByName(prewhere->prewhere_column_name).type->isNullable()) + if (getSampleBlock().getByName(prewhere_info.prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); result.clearFilter(); // Acting as a flag to not filter in PREWHERE } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 381b87ecffd..8f8482d1abf 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -13,7 +13,8 @@ using ColumnUInt8 = ColumnVector; class IMergeTreeReader; class MergeTreeIndexGranularity; struct PrewhereInfo; -using PrewhereInfoPtr = std::shared_ptr; +using PrewhereInfoList = std::vector; +using PrewhereInfoListPtr = std::shared_ptr; /// MergeTreeReader iterator which allows sequential reading for arbitrary number of rows between pairs of marks in the same part. /// Stores reading state, which can be inside granule. Can skip rows in current granule and start reading from next mark. @@ -24,7 +25,7 @@ public: MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - const PrewhereInfoPtr & prewhere_, + const PrewhereInfoListPtr & prewhere_info_list, bool last_reader_in_chain_); MergeTreeRangeReader() = default; @@ -153,8 +154,8 @@ public: void addRows(size_t rows) { num_read_rows += rows; } void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); } - /// Set filter or replace old one. Filter must have more zeroes than previous. - void setFilter(const ColumnPtr & new_filter); + /// Apply a filter on top of the existing one (AND'ed) or set it if there isn't any. + void addFilter(const ColumnPtr & new_filter); /// For each granule calculate the number of filtered rows at the end. Remove them and update filter. void optimize(bool can_read_incomplete_granules); /// Remove all rows from granules. @@ -212,12 +213,12 @@ private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); Columns continueReadingChain(ReadResult & result, size_t & num_rows); - void executePrewhereActionsAndFilterColumns(ReadResult & result); + void executePrewhereActionsAndFilterColumns(ReadResult & result, const PrewhereInfo & prewhere_info); IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly. - PrewhereInfoPtr prewhere; + PrewhereInfoListPtr prewhere_info_list; Stream stream; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index d9a250e3f7a..a3a580fa7f2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -24,7 +24,7 @@ MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, const bool check_columns_, const Names & column_names_, const BackoffSettings & backoff_settings_, @@ -37,7 +37,7 @@ MergeTreeReadPool::MergeTreeReadPool( , column_names{column_names_} , do_not_steal_tasks{do_not_steal_tasks_} , predict_block_size_bytes{preferred_block_size_bytes_ > 0} - , prewhere_info{prewhere_info_} + , prewhere_info_list{prewhere_info_list_} , parts_ranges{std::move(parts_)} { /// parts don't contain duplicate MergeTreeDataPart's. @@ -139,7 +139,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, return std::make_unique( part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names, per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx], - prewhere_info && prewhere_info->remove_prewhere_column, per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); + per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); } MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, const MarkRange & from) const @@ -229,7 +229,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo( per_part_sum_marks.push_back(sum_marks); auto [required_columns, required_pre_columns, should_reorder] = - getReadTaskColumns(data, metadata_snapshot, part.data_part, column_names, prewhere_info, check_columns); + getReadTaskColumns(data, metadata_snapshot, part.data_part, column_names, prewhere_info_list, check_columns); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & required_column_names = required_columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index aa6811661e6..ec9523ccbe3 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -71,10 +71,9 @@ private: public: MergeTreeReadPool( const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_, - RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_, - const bool check_columns_, const Names & column_names_, - const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, - const bool do_not_steal_tasks_ = false); + RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, + const PrewhereInfoListPtr & prewhere_info_list, const bool check_columns_, const Names & column_names_, + const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, const bool do_not_steal_tasks_ = false); MergeTreeReadTaskPtr getTask(const size_t min_marks_to_read, const size_t thread, const Names & ordered_names); @@ -107,7 +106,7 @@ private: std::vector per_part_pre_columns; std::vector per_part_should_reorder; std::vector per_part_size_predictor; - PrewhereInfoPtr prewhere_info; + PrewhereInfoListPtr prewhere_info_list; struct Part { diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index ee0a77ba3cf..35df1106339 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -22,7 +22,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( Names required_columns_, MarkRanges mark_ranges_, bool use_uncompressed_cache_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, bool check_columns, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_, @@ -31,7 +31,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( : MergeTreeBaseSelectProcessor{ metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), - storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, + storage_, metadata_snapshot_, prewhere_info_list_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, @@ -56,7 +56,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( ordered_names = header_without_virtual_columns.getNames(); - task_columns = getReadTaskColumns(storage, metadata_snapshot, data_part, required_columns, prewhere_info, check_columns); + task_columns = getReadTaskColumns(storage, metadata_snapshot, data_part, required_columns, prewhere_info_list, check_columns); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); @@ -71,7 +71,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); - if (prewhere_info) + if (prewhere_info_list) pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); } @@ -100,8 +100,7 @@ try task = std::make_unique( data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set, - task_columns.columns, task_columns.pre_columns, prewhere_info && prewhere_info->remove_prewhere_column, - task_columns.should_reorder, std::move(size_predictor)); + task_columns.columns, task_columns.pre_columns, task_columns.should_reorder, std::move(size_predictor)); return true; } diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index c9fd06c5534..b6da7166457 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -26,7 +26,7 @@ public: Names required_columns_, MarkRanges mark_ranges, bool use_uncompressed_cache, - const PrewhereInfoPtr & prewhere_info, + const PrewhereInfoListPtr & prewhere_info_list, bool check_columns, const MergeTreeReaderSettings & reader_settings, const Names & virt_column_names = {}, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 65f9b1eba3b..cdb97f47a47 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -22,7 +22,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( Names required_columns_, MarkRanges mark_ranges_, bool use_uncompressed_cache_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, bool check_columns_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_, @@ -31,7 +31,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( : MergeTreeBaseSelectProcessor{ metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), - storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, + storage_, metadata_snapshot_, prewhere_info_list_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, @@ -69,7 +69,7 @@ try task_columns = getReadTaskColumns( storage, metadata_snapshot, data_part, - required_columns, prewhere_info, check_columns); + required_columns, prewhere_info_list, check_columns); auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr @@ -81,8 +81,7 @@ try task = std::make_unique( data_part, all_mark_ranges, part_index_in_query, ordered_names, column_name_set, task_columns.columns, - task_columns.pre_columns, prewhere_info && prewhere_info->remove_prewhere_column, - task_columns.should_reorder, std::move(size_predictor)); + task_columns.pre_columns, task_columns.should_reorder, std::move(size_predictor)); if (!reader) { @@ -94,7 +93,7 @@ try reader = data_part->getReader(task_columns.columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); - if (prewhere_info) + if (prewhere_info_list) pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 925c437f1ce..521bbbfdba4 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -26,7 +26,7 @@ public: Names required_columns_, MarkRanges mark_ranges, bool use_uncompressed_cache, - const PrewhereInfoPtr & prewhere_info, + const PrewhereInfoListPtr & prewhere_info_list, bool check_columns, const MergeTreeReaderSettings & reader_settings, const Names & virt_column_names = {}, diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp index f57247e39ab..eb1a80acb49 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp @@ -18,12 +18,12 @@ MergeTreeThreadSelectBlockInputProcessor::MergeTreeThreadSelectBlockInputProcess const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const bool use_uncompressed_cache_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_) : MergeTreeBaseSelectProcessor{ - pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_, + pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_list_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, @@ -78,7 +78,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); - if (prewhere_info) + if (prewhere_info_list) pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); @@ -94,7 +94,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); - if (prewhere_info) + if (prewhere_info_list) pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h index 2b2ed36fc18..dd3ba8c973c 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h @@ -24,7 +24,7 @@ public: const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const bool use_uncompressed_cache_, - const PrewhereInfoPtr & prewhere_info_, + const PrewhereInfoListPtr & prewhere_info_list_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 5a3ada6288b..68f2f8f1361 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -32,6 +32,8 @@ struct PrewhereInfo : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} }; +using PrewhereInfoList = std::vector; + /// Same as PrewhereInfo, but with ActionsDAG struct PrewhereDAGInfo { @@ -75,7 +77,7 @@ struct InputOrderInfo bool operator !=(const InputOrderInfo & other) const { return !(*this == other); } }; -using PrewhereInfoPtr = std::shared_ptr; +using PrewhereInfoListPtr = std::shared_ptr; using PrewhereDAGInfoPtr = std::shared_ptr; using FilterInfoPtr = std::shared_ptr; using InputOrderInfoPtr = std::shared_ptr; @@ -104,7 +106,7 @@ struct SelectQueryInfo TreeRewriterResultPtr syntax_analyzer_result; - PrewhereInfoPtr prewhere_info; + PrewhereInfoListPtr prewhere_info_list; ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index ce74567c62b..53fee054f4b 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -314,21 +314,26 @@ void StorageBuffer::read( } else { - if (query_info.prewhere_info) + if (query_info.prewhere_info_list) { - pipe_from_buffers.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, query_info.prewhere_info->prewhere_actions, - query_info.prewhere_info->prewhere_column_name, query_info.prewhere_info->remove_prewhere_column); - }); - - if (query_info.prewhere_info->alias_actions) + for (const auto & prewhere_info : *query_info.prewhere_info_list) { pipe_from_buffers.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, query_info.prewhere_info->alias_actions); + return std::make_shared( + header, prewhere_info.prewhere_actions, + prewhere_info.prewhere_column_name, + prewhere_info.remove_prewhere_column); }); + + if (prewhere_info.alias_actions) + { + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, prewhere_info.alias_actions); + }); + } } } From ff5ce1a5ae9d5912954ed6027fa4436e96387273 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Mon, 25 Jan 2021 22:01:59 +0400 Subject: [PATCH 005/510] Fix compilation --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2ca2b30a5eb..361ab2d227a 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -470,7 +470,7 @@ void MergeTreeRangeReader::ReadResult::addFilter(const ColumnPtr & new_filter) { FilterDescription description(*new_filter); auto new_holder = (description.data_holder ? description.data_holder : new_filter); - auto * new_holder_cast = typeid_cast(new_holder.get()); + const auto * new_holder_cast = typeid_cast(new_holder.get()); if (!new_holder_cast) throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); From 4870e0af691f62e25efa706c891a118e309796a7 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Mon, 25 Jan 2021 22:09:17 +0400 Subject: [PATCH 006/510] Add filter as a (first) prewhere --- src/Interpreters/ExpressionAnalyzer.cpp | 25 ++++++--- src/Interpreters/InterpreterSelectQuery.cpp | 59 ++++++++++++++++----- src/Interpreters/InterpreterSelectQuery.h | 7 +-- 3 files changed, 64 insertions(+), 27 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 13f23643c3a..2055faca820 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1371,6 +1371,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && filter_info_) { + // TODO: handle filter exactly like prewhere, store the info in PrewhereDAGInfo, collect unnecessary columns, etc.? + filter_info = filter_info_; query_analyzer.appendPreliminaryFilter(chain, filter_info->actions_dag, filter_info->column_name); } @@ -1539,9 +1541,19 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, size_t where_step_num) { + size_t next_step_i = 0; + + if (hasFilter()) + { + const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); + filter_info->do_remove_column = step.can_remove_required_output.at(0); + + // TODO: handle filter exactly like prewhere, collect columns to remove after filter? + } + if (hasPrewhere()) { - const ExpressionActionsChain::Step & step = *chain.steps.at(0); + const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0); NameSet columns_to_remove; @@ -1553,13 +1565,12 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si columns_to_remove_after_prewhere = std::move(columns_to_remove); } - else if (hasFilter()) - { - /// Can't have prewhere and filter set simultaneously - filter_info->do_remove_column = chain.steps.at(0)->can_remove_required_output.at(0); - } + if (hasWhere()) - remove_where_filter = chain.steps.at(where_step_num)->can_remove_required_output.at(0); + { + const ExpressionActionsChain::Step & step = *chain.steps.at(where_step_num); + remove_where_filter = step.can_remove_required_output.at(0); + } } void ExpressionAnalysisResult::removeExtraColumns() const diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9dd63362dbd..57c18f1bb86 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -383,7 +383,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( view = nullptr; } - if (try_move_to_prewhere && storage && !row_policy_filter && query.where() && !query.prewhere() && !query.final()) + if (try_move_to_prewhere && storage && query.where() && !query.prewhere() && !query.final()) { /// PREWHERE optimization: transfer some condition from WHERE to PREWHERE if enabled and viable if (const auto * merge_tree = dynamic_cast(storage.get())) @@ -450,9 +450,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( } } - if (!options.only_analyze && storage && filter_info && query.prewhere()) - throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); - /// Calculate structure of the result. result_header = getSampleBlockImpl(); }; @@ -806,12 +803,30 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu bool intermediate_stage = false; bool to_aggregation_stage = false; bool from_aggregation_stage = false; + const bool filter_in_prewhere = ( + (settings.optimize_move_to_prewhere || expressions.prewhere_info) && + !input && !input_pipe && storage && storage->supportsPrewhere() + ); if (options.only_analyze) { auto read_nothing = std::make_unique(source_header); query_plan.addStep(std::move(read_nothing)); + if (expressions.filter_info && filter_in_prewhere) + { + auto row_level_security_step = std::make_unique( + query_plan.getCurrentDataStream(), + expressions.filter_info->actions_dag, + expressions.filter_info->column_name, + expressions.filter_info->do_remove_column); + + row_level_security_step->setStepDescription("Row-level security filter (PREWHERE)"); + query_plan.addStep(std::move(row_level_security_step)); + + // TODO: handle filter like prewhere, remove unnecessary columns after it, etc.? + } + if (expressions.prewhere_info) { auto prewhere_step = std::make_unique( @@ -862,11 +877,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu if (options.to_stage == QueryProcessingStage::WithMergeableStateAfterAggregation) to_aggregation_stage = true; - if (storage && expressions.filter_info && expressions.prewhere_info) - throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); - - /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ - executeFetchColumns(from_stage, query_plan, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); + /// Read the data from Storage. from_stage - to what stage the request was completed in Storage. + executeFetchColumns(from_stage, query_plan, filter_in_prewhere); LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } @@ -931,7 +943,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu if (expressions.first_stage) { - if (expressions.hasFilter()) + if (expressions.filter_info && !filter_in_prewhere) { auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -941,6 +953,8 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu row_level_security_step->setStepDescription("Row-level security filter"); query_plan.addStep(std::move(row_level_security_step)); + + // TODO: handle filter like prewhere, remove unnecessary columns after it, etc.? } if (expressions.before_array_join) @@ -1228,12 +1242,13 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c query_plan.addStep(std::move(read_from_pipe)); } -void InterpreterSelectQuery::executeFetchColumns( - QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, - const PrewhereDAGInfoPtr & prewhere_info, const NameSet & columns_to_remove_after_prewhere) +void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool filter_in_prewhere) { auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); + auto & expressions = analysis_result; + auto & prewhere_info = expressions.prewhere_info; + auto & columns_to_remove_after_prewhere = expressions.columns_to_remove_after_prewhere; /// Optimization for trivial query like SELECT count() FROM table. bool optimize_trivial_count = @@ -1241,7 +1256,7 @@ void InterpreterSelectQuery::executeFetchColumns( && (settings.max_parallel_replicas <= 1) && storage && storage->getName() != "MaterializeMySQL" - && !filter_info + && !expressions.filter_info && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) @@ -1554,6 +1569,22 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); + if (expressions.filter_info && filter_in_prewhere) + { + if (!query_info.prewhere_info_list) + query_info.prewhere_info_list = std::make_shared(); + + query_info.prewhere_info_list->emplace( + query_info.prewhere_info_list->begin(), + std::make_shared(expressions.filter_info->actions_dag), + expressions.filter_info->column_name); + + auto & new_filter_info = query_info.prewhere_info_list->front(); + + new_filter_info.remove_prewhere_column = expressions.filter_info->do_remove_column; + new_filter_info.need_filter = true; + } + if (prewhere_info) { if (!query_info.prewhere_info_list) diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 1fff316e1d4..6fcbf102b05 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -108,12 +108,7 @@ private: /// Different stages of query execution. - void executeFetchColumns( - QueryProcessingStage::Enum processing_stage, - QueryPlan & query_plan, - const PrewhereDAGInfoPtr & prewhere_info, - const NameSet & columns_to_remove_after_prewhere); - + void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool filter_in_prewhere); void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); void executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final); From 11b53d3b9d9a98748e763a1698aa88639c02ebd0 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Tue, 26 Jan 2021 00:17:48 +0400 Subject: [PATCH 007/510] Fix compilation/linter --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 361ab2d227a..fcac5bc2c59 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -484,10 +484,10 @@ void MergeTreeRangeReader::ReadResult::addFilter(const ColumnPtr & new_filter) throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); const auto & data = filter->getData(); - auto it = data.begin(); + auto * it = data.begin(); auto & new_data = new_mutable_holder_cast->getData(); - auto n_it = new_data.begin(); + auto * n_it = new_data.begin(); while (it != data.end() && n_it != new_data.end()) { From 0f7f8ace7388fd6aa700d21fbc946d48cc8eae43 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 26 Jan 2021 01:39:23 +0300 Subject: [PATCH 008/510] DOCSUP-5266: Add changes from PR --- .../operations/utilities/clickhouse-local.md | 4 ++ .../functions/date-time-functions.md | 12 ++++-- .../operations/utilities/clickhouse-local.md | 7 +++- .../data-types/simpleaggregatefunction.md | 3 ++ .../functions/date-time-functions.md | 38 +++++++++++++++++++ 5 files changed, 59 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-local.md b/docs/en/operations/utilities/clickhouse-local.md index 04f9f3660b5..cfabf42bff1 100644 --- a/docs/en/operations/utilities/clickhouse-local.md +++ b/docs/en/operations/utilities/clickhouse-local.md @@ -91,6 +91,8 @@ $ clickhouse-local --query " Now let’s output memory user for each Unix user: +Query: + ``` bash $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \ | clickhouse-local --structure "user String, mem Float64" \ @@ -98,6 +100,8 @@ $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \ FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty" ``` +Result: + ``` text Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. ┏━━━━━━━━━━┳━━━━━━━━━━┓ diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 9de780fb596..b73d13c59a4 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -661,8 +661,6 @@ Result: └────────────────────────────────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) - ## FROM\_UNIXTIME {#fromunixfime} When there is only single argument of integer type, it act in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md). @@ -670,10 +668,14 @@ type. For example: +Query: + ```sql -SELECT FROM_UNIXTIME(423543535) +SELECT FROM_UNIXTIME(423543535); ``` +Result: + ```text ┌─FROM_UNIXTIME(423543535)─┐ │ 1983-06-04 10:58:55 │ @@ -685,7 +687,7 @@ When there are two arguments, first is integer or DateTime, second is constant f For example: ```sql -SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime +SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; ``` ```text @@ -837,3 +839,5 @@ Result: │ 2020-01-01 │ └────────────────────────────────────┘ ``` + +[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) \ No newline at end of file diff --git a/docs/ru/operations/utilities/clickhouse-local.md b/docs/ru/operations/utilities/clickhouse-local.md index 2b5c9b119e2..e3c421ac75e 100644 --- a/docs/ru/operations/utilities/clickhouse-local.md +++ b/docs/ru/operations/utilities/clickhouse-local.md @@ -21,7 +21,8 @@ toc_title: clickhouse-local Основной формат вызова: ``` bash -$ clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" -q "query" +$ clickhouse-local --structure "table_structure" --input-format "format_of_incoming_data" \ + --query "query" ``` Ключи команды: @@ -78,6 +79,8 @@ $ clickhouse-local --query " А теперь давайте выведем на экран объём оперативной памяти, занимаемой пользователями (Unix): +Запрос: + ``` bash $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \ | clickhouse-local --structure "user String, mem Float64" \ @@ -85,6 +88,8 @@ $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \ FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty" ``` +Ответ: + ``` text Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. ┏━━━━━━━━━━┳━━━━━━━━━━┓ diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 52f0412a177..3ff4e5fd662 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -15,6 +15,9 @@ The following aggregate functions are supported: - [`groupBitXor`](../../sql-reference/aggregate-functions/reference/groupbitxor.md#groupbitxor) - [`groupArrayArray`](../../sql-reference/aggregate-functions/reference/grouparray.md#agg_function-grouparray) - [`groupUniqArrayArray`](../../sql-reference/aggregate-functions/reference/groupuniqarray.md#groupuniqarray) +- [`sumMap`](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) +- [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap) +- [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap) Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 31482cde77f..e923de8ebd2 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -665,4 +665,42 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g') └────────────────────────────────────────────┘ ``` +## FROM\_UNIXTIME {#fromunixfime} + +Когда есть только один аргумент целочисленного типа, он действует так же, как `toDateTime` и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). + +**Пример** + +Запрос: + +```sql +SELECT FROM_UNIXTIME(423543535); +``` + +Ответ: + +```text +┌─FROM_UNIXTIME(423543535)─┐ +│ 1983-06-04 10:58:55 │ +└──────────────────────────┘ +``` + +В случае, когда есть два аргумента, первый типа `Integer` или `DateTime`, а второй — является строкой постоянного формата, функция работает таким же образом, как `formatdatetime` и возвращает значение типа `String`. + +**Пример** + +Запрос: + +```sql +SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; +``` + +Ответ: + +```text +┌─DateTime────────────┐ +│ 2009-02-11 14:42:23 │ +└─────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/functions/date_time_functions/) From 097c9362bdad12d3ffbc7a817fc3bfda81a82156 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 26 Jan 2021 14:00:52 +0300 Subject: [PATCH 009/510] Update date-time-functions.md --- docs/en/sql-reference/functions/date-time-functions.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index b73d13c59a4..856ce830abe 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -602,7 +602,7 @@ This is necessary for searching for pageviews in the corresponding session. ## formatDateTime {#formatdatetime} -Function formats a Time according given Format string. N.B.: Format is a constant expression, e.g. you can not have multiple formats for single result column. +Function formats a Time according to the given Format string. N.B.: Format is a constant expression, e.g. you cannot have multiple formats for a single result column. **Syntax** @@ -663,7 +663,7 @@ Result: ## FROM\_UNIXTIME {#fromunixfime} -When there is only single argument of integer type, it act in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md). +When there is only a single argument of integer type, it acts in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md). type. For example: @@ -682,7 +682,7 @@ Result: └──────────────────────────┘ ``` -When there are two arguments, first is integer or DateTime, second is constant format string, it act in the same way as `formatDateTime` and return `String` type. +When there are two arguments: first is an integer or DateTime, second is a constant format string - it acts in the same way as `formatDateTime` and return `String` type. For example: @@ -840,4 +840,4 @@ Result: └────────────────────────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) \ No newline at end of file +[Original article](https://clickhouse.tech/docs/en/query_language/functions/date_time_functions/) From 1834c5ccae9da4b456544dbfa22d01f16ad0393f Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 26 Jan 2021 14:04:39 +0300 Subject: [PATCH 010/510] Update date-time-functions.md --- docs/ru/sql-reference/functions/date-time-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index e923de8ebd2..4db244d2388 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -665,9 +665,9 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g') └────────────────────────────────────────────┘ ``` -## FROM\_UNIXTIME {#fromunixfime} +## FROM\_UNIXTIME {#fromunixtime} -Когда есть только один аргумент целочисленного типа, он действует так же, как `toDateTime` и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). +Когда указан только один аргумент целочисленного типа, то функция действует так же, как `toDateTime`, и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). **Пример** @@ -685,7 +685,7 @@ SELECT FROM_UNIXTIME(423543535); └──────────────────────────┘ ``` -В случае, когда есть два аргумента, первый типа `Integer` или `DateTime`, а второй — является строкой постоянного формата, функция работает таким же образом, как `formatdatetime` и возвращает значение типа `String`. +В случае, когда есть два аргумента: первый типа `Integer` или `DateTime`, а второй является строкой постоянного формата — функция работает таким же образом, как `formatDateTime`, и возвращает значение типа `String`. **Пример** From 04531f14d9fb55c3eca1ac23070262d200828d60 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Tue, 26 Jan 2021 14:06:08 +0300 Subject: [PATCH 011/510] Fix hyphen --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 856ce830abe..f11bec55697 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -682,7 +682,7 @@ Result: └──────────────────────────┘ ``` -When there are two arguments: first is an integer or DateTime, second is a constant format string - it acts in the same way as `formatDateTime` and return `String` type. +When there are two arguments: first is an integer or DateTime, second is a constant format string — it acts in the same way as `formatDateTime` and return `String` type. For example: From 0d1c9479f8f904ff5c48b2320959f4dd244c4c0a Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Tue, 26 Jan 2021 18:39:12 +0400 Subject: [PATCH 012/510] Fix compilation/linter --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index fcac5bc2c59..0b3765adc6a 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -484,7 +484,7 @@ void MergeTreeRangeReader::ReadResult::addFilter(const ColumnPtr & new_filter) throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); const auto & data = filter->getData(); - auto * it = data.begin(); + const auto * it = data.begin(); auto & new_data = new_mutable_holder_cast->getData(); auto * n_it = new_data.begin(); From b3b832cde741de214ffe225512e47b7c96188adf Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 27 Jan 2021 12:33:11 +0300 Subject: [PATCH 013/510] Work with any number of replicas simultaneously, support max_parallel_replicas --- src/Client/Connection.cpp | 28 ++ src/Client/ConnectionPoolWithFailover.cpp | 6 +- src/Client/ConnectionPoolWithFailover.h | 6 +- src/Client/GetHedgedConnections.cpp | 439 +++++++++++----------- src/Client/GetHedgedConnections.h | 86 +++-- src/Client/HedgedConnections.cpp | 314 +++++++++------- src/Client/HedgedConnections.h | 33 +- src/DataStreams/RemoteQueryExecutor.cpp | 4 +- 8 files changed, 511 insertions(+), 405 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 15f530f4085..75586ea8cae 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -91,6 +91,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) void Connection::disconnect() { +// LOG_DEBUG(log_wrapper.get(), "disconnect"); in = nullptr; last_input_packet_type.reset(); out = nullptr; // can write to socket @@ -102,6 +103,8 @@ void Connection::disconnect() void Connection::prepare(const ConnectionTimeouts & timeouts) { +// LOG_DEBUG(log_wrapper.get(), "Connect"); + LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", default_database.empty() ? "(not specified)" : default_database, user, @@ -154,6 +157,8 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) void Connection::sendHello() { +// LOG_DEBUG(log_wrapper.get(), "sendHello"); + /** Disallow control characters in user controlled parameters * to mitigate the possibility of SSRF. * The user may do server side requests with 'remote' table function. @@ -210,6 +215,8 @@ void Connection::sendHello() void Connection::receiveHello() { +// LOG_DEBUG(log_wrapper.get(), "receiveHello"); + /// Receive hello packet. UInt64 packet_type = 0; @@ -313,6 +320,8 @@ const String & Connection::getServerDisplayName(const ConnectionTimeouts & timeo void Connection::forceConnected(const ConnectionTimeouts & timeouts) { +// LOG_DEBUG(log_wrapper.get(), "forceConnected"); + if (!connected) { connect(timeouts); @@ -339,6 +348,8 @@ void Connection::sendClusterNameAndSalt() bool Connection::ping() { +// LOG_DEBUG(log_wrapper.get(), "ping"); + TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); try { @@ -390,6 +401,8 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) { +// LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest"); + writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); out->next(); @@ -397,6 +410,8 @@ void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) TablesStatusResponse Connection::receiveTablesStatusResponse() { +// LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse"); + UInt64 response_type = 0; readVarUInt(response_type, *in); @@ -422,6 +437,8 @@ void Connection::sendQuery( if (!connected) connect(timeouts); +// LOG_DEBUG(log_wrapper.get(), "sendQuery"); + TimeoutSetter timeout_setter(*socket, timeouts.send_timeout, timeouts.receive_timeout, true); if (settings) @@ -520,6 +537,8 @@ void Connection::sendCancel() if (!out) return; +// LOG_DEBUG(log_wrapper.get(), "sendCancel"); + writeVarUInt(Protocol::Client::Cancel, *out); out->next(); } @@ -527,6 +546,8 @@ void Connection::sendCancel() void Connection::sendData(const Block & block, const String & name, bool scalar) { +// LOG_DEBUG(log_wrapper.get(), "sendData"); + if (!block_out) { if (compression == Protocol::Compression::Enable) @@ -557,6 +578,7 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name) { /// NOTE 'Throttler' is not used in this method (could use, but it's not important right now). +// LOG_DEBUG(log_wrapper.get(), "sendPreparedData"); writeVarUInt(Protocol::Client::Data, *out); writeStringBinary(name, *out); @@ -574,6 +596,8 @@ void Connection::sendScalarsData(Scalars & data) if (data.empty()) return; +// LOG_DEBUG(log_wrapper.get(), "sendScalarsData"); + Stopwatch watch; size_t out_bytes = out ? out->count() : 0; size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0; @@ -659,6 +683,8 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) return; } +// LOG_DEBUG(log_wrapper.get(), "sendExternalTablesData"); + Stopwatch watch; size_t out_bytes = out ? out->count() : 0; size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0; @@ -757,6 +783,8 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) Packet Connection::receivePacket(AsyncCallback async_callback) { +// LOG_DEBUG(log_wrapper.get(), "receivePacket"); + in->setAsyncCallback(std::move(async_callback)); SCOPE_EXIT(in->setAsyncCallback({})); diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 00ec1e30f10..af4f8bb2d25 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -353,7 +353,7 @@ void TryGetConnection::reset() { resetResult(); stage = Stage::CONNECT; - epoll = nullptr; + action_before_disconnect = nullptr; socket_fd = -1; fail_message.clear(); } @@ -369,8 +369,8 @@ void TryGetConnection::resetResult() void TryGetConnection::processFail(bool add_description) { - if (epoll) - epoll->remove(socket_fd); + if (action_before_disconnect) + action_before_disconnect(socket_fd); fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); if (add_description) diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index c57a7bb984a..86f63191608 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -62,9 +62,7 @@ public: /// Reset class to initial stage. void reset(); - /// If connection is failed and epoll is set, before disconnecting - /// socket will be removed from epoll. - void setEpoll(Epoll * epoll_) { epoll = epoll_; } + void setActionBeforeDisconnect(std::function action) { action_before_disconnect = action; } /// Process fail connection. void processFail(bool add_description = false); @@ -78,7 +76,7 @@ public: TryResult result; Stage stage; int socket_fd; - Epoll * epoll = nullptr; + std::function action_before_disconnect; }; class ConnectionPoolWithFailover : public IConnectionPool, private PoolWithFailoverBase diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 839d6bf37c2..4c729dc0722 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -7,6 +7,7 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; extern const int ALL_CONNECTION_TRIES_FAILED; } @@ -26,6 +27,9 @@ GetHedgedConnections::GetHedgedConnections( = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries : false; + entries_count = 0; + usable_count = 0; + failed_pools_count = 0; } GetHedgedConnections::~GetHedgedConnections() @@ -33,173 +37,175 @@ GetHedgedConnections::~GetHedgedConnections() pool->updateSharedError(shuffled_pools); } -GetHedgedConnections::Replicas GetHedgedConnections::getConnections() +std::vector GetHedgedConnections::getManyConnections(PoolMode pool_mode) { - entries_count = 0; - usable_count = 0; - failed_pools_count = 0; + size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; - ReplicaStatePtr replica = &first_replica; - int index = 0; + size_t max_entries; + if (pool_mode == PoolMode::GET_ALL) + { + min_entries = shuffled_pools.size(); + max_entries = shuffled_pools.size(); + } + else if (pool_mode == PoolMode::GET_ONE) + max_entries = 1; + else if (pool_mode == PoolMode::GET_MANY) + max_entries = settings ? size_t(settings->max_parallel_replicas) : 1; + else + throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR); + + std::vector replicas; + replicas.reserve(max_entries); + for (size_t i = 0; i != max_entries; ++i) + { + auto replica = getNextConnection(false); + if (replica->isCannotChoose()) + { + if (replicas.size() >= min_entries) + break; + + /// Determine the reason of not enough replicas. + if (!fallback_to_stale_replicas && usable_count >= min_entries) + throw DB::Exception( + "Could not find enough connections to up-to-date replicas. Got: " + std::to_string(replicas.size()) + + ", needed: " + std::to_string(min_entries), + DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); + + throw DB::NetException( + "Could not connect to " + std::to_string(min_entries) + " replicas. Log: \n\n" + fail_messages + "\n", + DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + } + replicas.push_back(replica); + } + + return replicas; +} + +GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bool non_blocking) +{ +// LOG_DEBUG(log, "getNextConnection"); + ReplicaStatePtr replica = createNewReplica(); + + int index; + + /// Check if it's the first time. + if (epoll.size() == 0 && ready_indexes.size() == 0) + { + index = 0; + last_used_index = 0; + } + else + index = getNextIndex(); + + bool is_first = true; while (index != -1 || epoll.size() != 0) { + if (index == -1 && !is_first && non_blocking) + { + replica->state = State::NOT_READY; + return replica; + } + + if (is_first) + is_first = false; + if (index != -1) { Action action = startTryGetConnection(index, replica); - if (action == Action::TRY_NEXT_REPLICA) - { - index = getNextIndex(index); - continue; - } if (action == Action::FINISH) - { - swapReplicasIfNeeded(); - return {&first_replica, &second_replica}; - } - } - - /// Process epoll events - replica = processEpollEvents(); - if (replica->isReady()) - { - swapReplicasIfNeeded(); - return {&first_replica, &second_replica}; - } - - index = getNextIndex(index); - } - - /// We reach this point only if there was no up to date replica - - if (usable_count == 0) - { - if (settings && settings->skip_unavailable_shards) - { - first_replica.state = State::CANNOT_CHOOSE; - second_replica.state = State::CANNOT_CHOOSE; - return {&first_replica, &second_replica}; - } - - throw NetException("All connection tries failed. Log: \n\n" + fail_messages + "\n", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); - } - if (!fallback_to_stale_replicas) - throw DB::Exception("Could not find connection to up-to-date replica.", DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); - - setBestUsableReplica(first_replica); - return {&first_replica, &second_replica}; -} - -void GetHedgedConnections::chooseSecondReplica() -{ - LOG_DEBUG(log, "choose second replica"); - - if (second_replica.isCannotChoose() || second_replica.isReady()) - return; - - int index; - if (second_replica.isNotReady()) - index = second_replica.index; - else - index = first_replica.index; - - while (true) - { - if (second_replica.isEmpty()) - { - - index = getNextIndex(index); - if (index == -1) - break; - - Action action = startTryGetConnection(index, &second_replica); + return replica; if (action == Action::TRY_NEXT_REPLICA) + { + index = getNextIndex(); continue; + } - /// Second replica is ready or we are waiting for response from it - return; + if (action == Action::PROCESS_EPOLL_EVENTS && non_blocking) + return replica; } - if (!second_replica.isNotReady()) - throw Exception("Second replica state must be 'NOT_READY' before process epoll events", ErrorCodes::LOGICAL_ERROR); + replica = processEpollEvents(non_blocking); + if (replica->isReady() || (replica->isNotReady() && non_blocking)) + return replica; - ReplicaStatePtr replica = processEpollEvents( true); + if (replica->isNotReady()) + throw Exception("Not ready replica after processing epoll events.", ErrorCodes::LOGICAL_ERROR); - if (replica != &second_replica) - throw Exception("Epoll could return only second replica here", ErrorCodes::LOGICAL_ERROR); - - /// If replica is not empty than it is ready or we are waiting for a response from it - if (!second_replica.isEmpty()) - return; + index = getNextIndex(); } - /// There is no up to date replica + /// We reach this point only if there was no free up to date replica. - LOG_DEBUG(log, "there is no up to date replica for second replica"); + /// Check if there is no even a free usable replica + if (!canGetNewConnection()) + { + replica->state = State::CANNOT_CHOOSE; + return replica; + } - if (!fallback_to_stale_replicas || usable_count <= 1) - second_replica.state = State::CANNOT_CHOOSE; - else - setBestUsableReplica(second_replica, first_replica.index); + if (!fallback_to_stale_replicas) + { + replica->state = State::CANNOT_CHOOSE; + return replica; + } + + setBestUsableReplica(replica); + return replica; } -void GetHedgedConnections::stopChoosingSecondReplica() +void GetHedgedConnections::stopChoosingReplicas() { - LOG_DEBUG(log, "stop choosing second replica"); +// LOG_DEBUG(log, "stopChoosingReplicas"); + for (auto & [fd, replica] : fd_to_replica) + { + removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); + epoll.remove(fd); + try_get_connections[replica->index].reset(); + replica->reset(); + } - if (!second_replica.isNotReady()) - throw Exception("Can't stop choosing second replica, because it's not in process of choosing", ErrorCodes::LOGICAL_ERROR); - - removeTimeoutsFromReplica(&second_replica, epoll); - epoll.remove(second_replica.fd); - - try_get_connections[second_replica.index].reset(); - second_replica.reset(); + fd_to_replica.clear(); } -int GetHedgedConnections::getNextIndex(int cur_index) +int GetHedgedConnections::getNextIndex() { /// Check if there is no more available replicas - if (cur_index == -1 || entries_count + failed_pools_count >= shuffled_pools.size()) + if (entries_count + failed_pools_count >= shuffled_pools.size()) return -1; - /// We can work with two replicas simultaneously and they must have different indexes - int skip_index = -1; - if (!first_replica.isEmpty()) - skip_index = first_replica.index; - else if (!second_replica.isEmpty()) - skip_index = second_replica.index; - bool finish = false; - int next_index = cur_index; + int next_index = last_used_index; while (!finish) { next_index = (next_index + 1) % shuffled_pools.size(); /// Check if we can try this replica - if (next_index != skip_index && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) + if (indexes_in_process.find(next_index) == indexes_in_process.end() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) && try_get_connections[next_index].stage != TryGetConnection::Stage::FINISHED) finish = true; /// If we made a complete round, there is no replica to connect - else if (next_index == cur_index) + else if (next_index == last_used_index) return -1; } - LOG_DEBUG(log, "get next index: {}", next_index); +// LOG_DEBUG(log, "get next index: {}", next_index); + last_used_index = next_index; return next_index; } -GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr replica) +GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr & replica) { - LOG_DEBUG(log, "start try get connection with {} replica", index); +// LOG_DEBUG(log, "start try get connection with {} replica", index); TryGetConnection & try_get_connection = try_get_connections[index]; replica->state = State::NOT_READY; replica->index = index; + indexes_in_process.insert(index); try_get_connection.reset(); try_get_connection.run(); @@ -215,7 +221,13 @@ GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int ind if (action == Action::PROCESS_EPOLL_EVENTS) { epoll.add(try_get_connection.socket_fd); - try_get_connection.setEpoll(&epoll); + fd_to_replica[try_get_connection.socket_fd] = replica; + try_get_connection.setActionBeforeDisconnect( + [&](int fd) + { + epoll.remove(fd); + fd_to_replica.erase(fd); + }); addTimeouts(replica); } @@ -223,51 +235,58 @@ GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int ind } GetHedgedConnections::Action -GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr replica, bool remove_from_epoll) +GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll) { - LOG_DEBUG(log, "process get connection stage for {} replica", replica->index); +// LOG_DEBUG(log, "process get connection stage for {} replica", replica->index); TryGetConnection & try_get_connection = try_get_connections[replica->index]; if (try_get_connection.stage == TryGetConnection::Stage::FINISHED) { - LOG_DEBUG(log, "stage: FINISHED"); + indexes_in_process.erase(replica->index); + +// LOG_DEBUG(log, "stage: FINISHED"); ++entries_count; if (remove_from_epoll) + { epoll.remove(try_get_connection.socket_fd); + fd_to_replica.erase(try_get_connection.socket_fd); + } if (try_get_connection.result.is_usable) { - LOG_DEBUG(log, "replica is usable"); +// LOG_DEBUG(log, "replica is usable"); ++usable_count; if (try_get_connection.result.is_up_to_date) { - LOG_DEBUG(log, "replica is up to date, finish get hedged connections"); +// LOG_DEBUG(log, "replica is up to date, finish get hedged connections"); replica->state = State::READY; + ready_indexes.insert(replica->index); return Action::FINISH; } - - /// This replica is not up to date, we will try to find up to date - replica->reset(); - return Action::TRY_NEXT_REPLICA; } + + /// This replica is not up to date, we will try to find up to date + fd_to_replica.erase(replica->fd); + replica->reset(); + return Action::TRY_NEXT_REPLICA; } else if (try_get_connection.stage == TryGetConnection::Stage::FAILED) { - LOG_DEBUG(log, "stage: FAILED"); +// LOG_DEBUG(log, "stage: FAILED"); processFailedConnection(replica); return Action::TRY_NEXT_REPLICA; } - LOG_DEBUG(log, "middle stage, process epoll events"); +// LOG_DEBUG(log, "middle stage, process epoll events"); /// Get connection process is not finished return Action::PROCESS_EPOLL_EVENTS; } -void GetHedgedConnections::processFailedConnection(ReplicaStatePtr replica) +void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica) { - LOG_DEBUG(log, "failed connection with {} replica", replica->index); +// LOG_DEBUG(log, "failed connection with {} replica", replica->index); ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; LOG_WARNING( @@ -286,105 +305,65 @@ void GetHedgedConnections::processFailedConnection(ReplicaStatePtr replica) if (!fail_message.empty()) fail_messages += fail_message + "\n"; + indexes_in_process.erase(replica->index); replica->reset(); } -void GetHedgedConnections::addTimeouts(ReplicaState * replica) +void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica) { - LOG_DEBUG(log, "add timeouts for {} replica", replica->index); +// LOG_DEBUG(log, "add timeouts for {} replica", replica->index); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeouts); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); - /// If we haven't connected to second replica yet, set special timeout for it - if (second_replica.isEmpty()) - { - auto stage = try_get_connections[replica->index].stage; - if (stage == TryGetConnection::Stage::RECEIVE_HELLO) - addTimeoutToReplica(TimerTypes::RECEIVE_HELLO_TIMEOUT, replica, epoll, timeouts); - else if (stage == TryGetConnection::Stage::RECEIVE_TABLES_STATUS) - addTimeoutToReplica(TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT, replica, epoll, timeouts); - } -} - -void GetHedgedConnections::swapReplicasIfNeeded() -{ - if ((!first_replica.isReady() && second_replica.isReady())) - { - LOG_DEBUG(log, "swap replicas"); - swapReplicas(); - } + auto stage = try_get_connections[replica->index].stage; + if (stage == TryGetConnection::Stage::RECEIVE_HELLO) + addTimeoutToReplica(TimerTypes::RECEIVE_HELLO_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); + else if (stage == TryGetConnection::Stage::RECEIVE_TABLES_STATUS) + addTimeoutToReplica(TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); } GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(bool non_blocking) { - LOG_DEBUG(log, "process epoll events"); +// LOG_DEBUG(log, "process epoll events"); int event_fd; ReplicaStatePtr replica = nullptr; bool finish = false; while (!finish) { - event_fd = getReadyFileDescriptor(epoll); + event_fd = getReadyFileDescriptor(); - if ((replica = isEventReplica(event_fd))) - finish = processReplicaEvent(replica, non_blocking); - - else if (auto * timeout_descriptor = isEventTimeout(event_fd, replica)) + if (fd_to_replica.find(event_fd) != fd_to_replica.end()) { - processTimeoutEvent(replica, timeout_descriptor); - finish = true; + replica = fd_to_replica[event_fd]; + finish = processReplicaEvent(replica, non_blocking); + } + else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) + { + replica = timeout_fd_to_replica[event_fd]; + finish = processTimeoutEvent(replica, replica->active_timeouts[event_fd].get(), non_blocking); } else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } - LOG_DEBUG(log, "cancel process epoll events"); +// LOG_DEBUG(log, "cancel process epoll events"); return replica; } -GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::isEventReplica(int event_fd) +int GetHedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { - if (event_fd == first_replica.fd) - return &first_replica; + for (auto & [fd, replica] : fd_to_replica) + if (replica->connection->hasReadPendingData()) + return replica->fd; - if (event_fd == second_replica.fd) - return &second_replica; - - return nullptr; + return epoll.getReady(std::move(async_callback)).data.fd; } -TimerDescriptorPtr GetHedgedConnections::isEventTimeout(int event_fd, ReplicaStatePtr & replica_out) +bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking) { - if (first_replica.active_timeouts.find(event_fd) != first_replica.active_timeouts.end()) - { - replica_out = &first_replica; - return first_replica.active_timeouts[event_fd].get(); - } - - if (second_replica.active_timeouts.find(event_fd) != second_replica.active_timeouts.end()) - { - replica_out = &second_replica; - return second_replica.active_timeouts[event_fd].get(); - } - - return nullptr; -} - -int GetHedgedConnections::getReadyFileDescriptor(Epoll & epoll_, AsyncCallback async_callback) -{ - if (first_replica.connection && first_replica.connection->hasReadPendingData()) - return first_replica.fd; - - if (second_replica.connection && second_replica.connection->hasReadPendingData()) - return second_replica.fd; - - return epoll_.getReady(std::move(async_callback)).data.fd; -} - -bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr replica, bool non_blocking) -{ - LOG_DEBUG(log, "epoll event is {} replica", replica->index); - removeTimeoutsFromReplica(replica, epoll); +// LOG_DEBUG(log, "epoll event is {} replica", replica->index); + removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); try_get_connections[replica->index].run(); Action action = processTryGetConnectionStage(replica, true); if (action == Action::PROCESS_EPOLL_EVENTS) @@ -396,70 +375,84 @@ bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr replica, bool non return true; } -void GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor) +bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking) { - LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index); +// LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index); epoll.remove(timeout_descriptor->getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); + timeout_fd_to_replica[timeout_descriptor->getDescriptor()]; if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) { - LOG_DEBUG(log, "process receive timeout for {} replica", replica->index); - removeTimeoutsFromReplica(replica, epoll); +// LOG_DEBUG(log, "process receive timeout for {} replica", replica->index); + removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); epoll.remove(replica->fd); + fd_to_replica.erase(replica->fd); TryGetConnection & try_get_connection = try_get_connections[replica->index]; try_get_connection.fail_message = "Receive timeout expired (" + try_get_connection.result.entry->getDescription() + ")"; try_get_connection.resetResult(); try_get_connection.stage = TryGetConnection::Stage::FAILED; processFailedConnection(replica); + + return true; } - else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT + else if ((timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT || timeout_descriptor->getType() == TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT) + && entries_count + ready_indexes.size() + failed_pools_count < shuffled_pools.size()) { - if (replica->index == second_replica.index || !second_replica.isEmpty()) - throw Exception( - "Received timeout to connect with second replica, but current replica is second or second replica is not empty", - ErrorCodes::LOGICAL_ERROR); - replica = &second_replica; + replica = createNewReplica(); + return true; } + + return non_blocking; } -void GetHedgedConnections::setBestUsableReplica(ReplicaState & replica, int skip_index) +void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) { - LOG_DEBUG(log, "set best usable replica"); +// LOG_DEBUG(log, "set best usable replica"); std::vector indexes(try_get_connections.size()); for (size_t i = 0; i != indexes.size(); ++i) indexes[i] = i; - /// Remove unusable and failed replicas, skip the replica with skip_index index + /// Remove unusable and failed replicas, skip ready replicas indexes.erase( std::remove_if( indexes.begin(), indexes.end(), [&](int i) { - return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable || i == skip_index; + return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable || + indexes_in_process.find(i) != indexes_in_process.end() || ready_indexes.find(i) != ready_indexes.end(); }), indexes.end()); if (indexes.empty()) - throw Exception("There is no usable replica to choose", ErrorCodes::LOGICAL_ERROR); + { + replica->state = State::CANNOT_CHOOSE; + return; + } /// Sort replicas by staleness std::stable_sort(indexes.begin(), indexes.end(), [&](size_t lhs, size_t rhs) { return try_get_connections[lhs].result.staleness < try_get_connections[rhs].result.staleness; }); - replica.index = indexes[0]; - replica.connection = &*try_get_connections[indexes[0]].result.entry; - replica.state = State::READY; - replica.fd = replica.connection->getSocket()->impl()->sockfd(); + replica->index = indexes[0]; + replica->connection = &*try_get_connections[indexes[0]].result.entry; + replica->state = State::READY; + replica->fd = replica->connection->getSocket()->impl()->sockfd(); + ready_indexes.insert(replica->index); } -void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll, const ConnectionTimeouts & timeouts) +void addTimeoutToReplica( + int type, + GetHedgedConnections::ReplicaStatePtr & replica, + Epoll & epoll, + std::unordered_map & timeout_fd_to_replica, + const ConnectionTimeouts & timeouts) { Poco::Timespan timeout; switch (type) @@ -484,17 +477,28 @@ void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica timeout_descriptor->setType(type); timeout_descriptor->setRelative(timeout); epoll.add(timeout_descriptor->getDescriptor()); + timeout_fd_to_replica[timeout_descriptor->getDescriptor()] = replica; replica->active_timeouts[timeout_descriptor->getDescriptor()] = std::move(timeout_descriptor); } -void removeTimeoutsFromReplica(GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll) +void removeTimeoutsFromReplica( + GetHedgedConnections::ReplicaStatePtr & replica, + Epoll & epoll, + std::unordered_map & timeout_fd_to_replica) { for (auto & [fd, _] : replica->active_timeouts) + { epoll.remove(fd); + timeout_fd_to_replica.erase(fd); + } replica->active_timeouts.clear(); } -void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll) +void removeTimeoutFromReplica( + int type, + GetHedgedConnections::ReplicaStatePtr & replica, + Epoll & epoll, + std::unordered_map & timeout_fd_to_replica) { auto it = std::find_if( replica->active_timeouts.begin(), @@ -505,6 +509,7 @@ void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr re if (it != replica->active_timeouts.end()) { epoll.remove(it->first); + timeout_fd_to_replica.erase(it->first); replica->active_timeouts.erase(it); } } diff --git a/src/Client/GetHedgedConnections.h b/src/Client/GetHedgedConnections.h index c42dc24ddc7..df060e9ecd5 100644 --- a/src/Client/GetHedgedConnections.h +++ b/src/Client/GetHedgedConnections.h @@ -31,6 +31,7 @@ public: State state = State::EMPTY; int index = -1; int fd = -1; + size_t parallel_replica_offset = 0; std::unordered_map> active_timeouts; void reset() @@ -39,6 +40,7 @@ public: state = State::EMPTY; index = -1; fd = -1; + parallel_replica_offset = 0; active_timeouts.clear(); } @@ -48,7 +50,8 @@ public: bool isCannotChoose() const { return state == State::CANNOT_CHOOSE; }; }; - using ReplicaStatePtr = ReplicaState *; + using ReplicaStatePtr = std::shared_ptr; + struct Replicas { @@ -61,32 +64,15 @@ public: const ConnectionTimeouts & timeouts_, std::shared_ptr table_to_check_ = nullptr); - /// Establish connection with replicas. Return replicas as soon as connection with one of them is finished. - /// The first replica is always has state FINISHED and ready for sending query, the second replica - /// may have any state. To continue working with second replica call chooseSecondReplica(). - Replicas getConnections(); + std::vector getManyConnections(PoolMode pool_mode); - /// Continue choosing second replica, this function is not blocking. Second replica will be ready - /// for sending query when it has state FINISHED. - void chooseSecondReplica(); + ReplicaStatePtr getNextConnection(bool non_blocking); - void stopChoosingSecondReplica(); + bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); } - void swapReplicas() { std::swap(first_replica, second_replica); } + void stopChoosingReplicas(); - /// Move ready replica to the first place. - void swapReplicasIfNeeded(); - - /// Check if the file descriptor is belong to one of replicas. If yes, return this replica, if no, return nullptr. - ReplicaStatePtr isEventReplica(int event_fd); - - /// Check if the file descriptor is belong to timeout to any replica. - /// If yes, return corresponding TimerDescriptor and set timeout owner to replica, - /// if no, return nullptr. - TimerDescriptorPtr isEventTimeout(int event_fd, ReplicaStatePtr & replica); - - /// Get file rescriptor that ready for reading. - int getReadyFileDescriptor(Epoll & epoll_, AsyncCallback async_callback = {}); + bool hasEventsInProcess() const { return epoll.size() > 0; } int getFileDescriptor() const { return epoll.getFileDescriptor(); } @@ -103,25 +89,29 @@ private: TRY_NEXT_REPLICA = 2, }; - Action startTryGetConnection(int index, ReplicaStatePtr replica); + Action startTryGetConnection(int index, ReplicaStatePtr & replica); - Action processTryGetConnectionStage(ReplicaStatePtr replica, bool remove_from_epoll = false); + Action processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); - int getNextIndex(int cur_index = -1); + int getNextIndex(); - void addTimeouts(ReplicaStatePtr replica); + int getReadyFileDescriptor(AsyncCallback async_callback = {}); - void processFailedConnection(ReplicaStatePtr replica); + void addTimeouts(ReplicaStatePtr & replica); - void processReceiveTimeout(ReplicaStatePtr replica); + void processFailedConnection(ReplicaStatePtr & replica); - bool processReplicaEvent(ReplicaStatePtr replica, bool non_blocking); + void processReceiveTimeout(ReplicaStatePtr & replica); - void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor); + bool processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking); + + bool processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking); ReplicaStatePtr processEpollEvents(bool non_blocking = false); - void setBestUsableReplica(ReplicaState & replica, int skip_index = -1); + void setBestUsableReplica(ReplicaStatePtr & replica); + + ReplicaStatePtr createNewReplica() { return std::make_shared(); } const ConnectionPoolWithFailoverPtr pool; const Settings * settings; @@ -129,8 +119,14 @@ private: std::shared_ptr table_to_check; std::vector try_get_connections; std::vector shuffled_pools; - ReplicaState first_replica; - ReplicaState second_replica; + + std::unordered_map fd_to_replica; + std::unordered_map timeout_fd_to_replica; + +// std::vector> replicas; +// std::unordered_map> replicas_store; +// ReplicaState first_replica; +// ReplicaState second_replica; bool fallback_to_stale_replicas; Epoll epoll; Poco::Logger * log; @@ -139,16 +135,30 @@ private: size_t usable_count; size_t failed_pools_count; size_t max_tries; + int last_used_index; + std::unordered_set indexes_in_process; + std::unordered_set ready_indexes; }; /// Add timeout with particular type to replica and add it to epoll. -void addTimeoutToReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll, const ConnectionTimeouts & timeouts); - +void addTimeoutToReplica( + int type, + GetHedgedConnections::ReplicaStatePtr & replica, + Epoll & epoll, + std::unordered_map & timeout_fd_to_replica, + const ConnectionTimeouts & timeouts); /// Remove timeout with particular type from replica and epoll. -void removeTimeoutFromReplica(int type, GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll); +void removeTimeoutFromReplica( + int type, + GetHedgedConnections::ReplicaStatePtr & replica, + Epoll & epoll, + std::unordered_map & timeout_fd_to_replica); /// Remove all timeouts from replica and epoll. -void removeTimeoutsFromReplica(GetHedgedConnections::ReplicaStatePtr replica, Epoll & epoll); +void removeTimeoutsFromReplica( + GetHedgedConnections::ReplicaStatePtr & replica, + Epoll & epoll, + std::unordered_map & timeout_fd_to_replica); } diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 4963c74c327..a4231b2c172 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -15,91 +15,83 @@ HedgedConnections::HedgedConnections( const Settings & settings_, const ConnectionTimeouts & timeouts_, const ThrottlerPtr & throttler_, + PoolMode pool_mode, std::shared_ptr table_to_check_) : get_hedged_connections(pool_, &settings_, timeouts_, table_to_check_), settings(settings_), throttler(throttler_), log(&Poco::Logger::get("HedgedConnections")) { - replicas = get_hedged_connections.getConnections(); + std::vector replicas_states = get_hedged_connections.getManyConnections(pool_mode); - /// First replica may have state CANNOT_CHOOSE if setting skip_unavailable_shards is enabled - if (replicas.first_replica->isReady()) - replicas.first_replica->connection->setThrottler(throttler); - - if (!replicas.second_replica->isCannotChoose()) + for (size_t i = 0; i != replicas_states.size(); ++i) { - if (replicas.second_replica->isNotReady()) - epoll.add(get_hedged_connections.getFileDescriptor()); - - auto set_throttler = [throttler_](ReplicaStatePtr replica) - { - replica->connection->setThrottler(throttler_); - }; - second_replica_pipeline.add(std::function(set_throttler)); + replicas_states[i]->parallel_replica_offset = i; + replicas_states[i]->connection->setThrottler(throttler_); + epoll.add(replicas_states[i]->fd); + fd_to_replica[replicas_states[i]->fd] = replicas_states[i]; + replicas.push_back({std::move(replicas_states[i])}); + active_connections_count_by_offset[i] = 1; } + + pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr & replica_){ replica_->connection->setThrottler(throttler_); }); } -void HedgedConnections::Pipeline::add(std::function send_function) +void HedgedConnections::Pipeline::add(std::function send_function) { pipeline.push_back(send_function); } -void HedgedConnections::Pipeline::run(ReplicaStatePtr replica) +void HedgedConnections::Pipeline::run(ReplicaStatePtr & replica) { for (auto & send_func : pipeline) send_func(replica); - - pipeline.clear(); } size_t HedgedConnections::size() const { - if (replicas.first_replica->isReady() || replicas.second_replica->isReady()) - return 1; + if (replicas.empty()) + return 0; - return 0; -} - -bool HedgedConnections::hasActiveConnections() const -{ - return replicas.first_replica->isReady() || replicas.second_replica->isReady(); + return 1; } void HedgedConnections::sendScalarsData(Scalars & data) { std::lock_guard lock(cancel_mutex); +// LOG_DEBUG(log, "sendScalarsData"); + if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); - auto send_scalars_data = [&data](ReplicaStatePtr replica) { replica->connection->sendScalarsData(data); }; + auto send_scalars_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendScalarsData(data); }; - if (replicas.first_replica->isReady()) - send_scalars_data(replicas.first_replica); + for (auto & replicas_with_same_offset : replicas) + for (auto & replica : replicas_with_same_offset) + if (replica->isReady()) + send_scalars_data(replica); - if (replicas.second_replica->isReady()) - send_scalars_data(replicas.second_replica); - else if (!replicas.second_replica->isCannotChoose()) - second_replica_pipeline.add(std::function(send_scalars_data)); + pipeline_for_new_replicas.add(send_scalars_data); } void HedgedConnections::sendExternalTablesData(std::vector & data) { std::lock_guard lock(cancel_mutex); +// LOG_DEBUG(log, "sendExternalTablesData"); + if (!sent_query) throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); if (data.size() != size()) throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES); - auto send_external_tables_data = [&data](ReplicaStatePtr replica) { replica->connection->sendExternalTablesData(data[0]); }; + auto send_external_tables_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendExternalTablesData(data[0]); }; - if (replicas.first_replica->isReady()) - send_external_tables_data(replicas.first_replica); + for (auto & replicas_with_same_offset : replicas) + for (auto & replica : replicas_with_same_offset) + if (replica->isReady()) + send_external_tables_data(replica); - if (replicas.second_replica->isReady()) - send_external_tables_data(replicas.second_replica); - else if (!replicas.second_replica->isCannotChoose()) - second_replica_pipeline.add(send_external_tables_data); + pipeline_for_new_replicas.add(send_external_tables_data); } void HedgedConnections::sendQuery( @@ -112,35 +104,52 @@ void HedgedConnections::sendQuery( { std::lock_guard lock(cancel_mutex); +// LOG_DEBUG(log, "sendQuery"); + if (sent_query) throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR); - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr replica) + for (auto & replicas_with_same_offset : replicas) { - Settings modified_settings = settings; - if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + for (auto & replica : replicas_with_same_offset) { + if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + { + has_two_level_aggregation_incompatibility = true; + break; + } + } + if (has_two_level_aggregation_incompatibility) + break; + } + + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica) + { + Settings modified_settings = this->settings; + + if (this->has_two_level_aggregation_incompatibility) + { + /// Disable two-level aggregation due to version incompatibility. modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold_bytes = 0; } + if (this->replicas.size() > 1) + { + modified_settings.parallel_replicas_count = this->replicas.size(); + modified_settings.parallel_replica_offset = replica->parallel_replica_offset; + } + replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); - this->epoll.add(replica->fd); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, timeouts); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); + addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); }; - if (replicas.first_replica->isReady()) - { - send_query(replicas.first_replica); - if (replicas.second_replica->isEmpty()) - addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replicas.first_replica, epoll, timeouts); - } - - if (replicas.second_replica->isReady()) - send_query(replicas.second_replica); - else if (!replicas.second_replica->isCannotChoose()) - second_replica_pipeline.add(send_query); + for (auto & replicas_with_same_offset : replicas) + for (auto & replica : replicas_with_same_offset) + send_query(replica); + pipeline_for_new_replicas.add(send_query); sent_query = true; } @@ -148,32 +157,41 @@ void HedgedConnections::disconnect() { std::lock_guard lock(cancel_mutex); - if (replicas.first_replica->isReady()) - { - replicas.first_replica->connection->disconnect(); - replicas.first_replica->reset(); - } +// LOG_DEBUG(log, "disconnect"); - if (replicas.second_replica->isReady()) + for (auto & replicas_with_same_offset : replicas) + for (auto & replica : replicas_with_same_offset) + if (replica->isReady()) + finishProcessReplica(replica, true); + + if (get_hedged_connections.hasEventsInProcess()) { - replicas.second_replica->connection->disconnect(); - replicas.second_replica->reset(); + get_hedged_connections.stopChoosingReplicas(); + if (next_replica_in_process) + epoll.remove(get_hedged_connections.getFileDescriptor()); } - else if (replicas.second_replica->isNotReady()) - get_hedged_connections.stopChoosingSecondReplica(); } std::string HedgedConnections::dumpAddresses() const { std::lock_guard lock(cancel_mutex); +// LOG_DEBUG(log, "dumpAddresses"); + std::string addresses = ""; + bool is_first = true; - if (replicas.first_replica->isReady()) - addresses += replicas.first_replica->connection->getDescription(); - - if (replicas.second_replica->isReady()) - addresses += "; " + replicas.second_replica->connection->getDescription(); + for (auto & replicas_with_same_offset : replicas) + { + for (auto & replica : replicas_with_same_offset) + { + if (replica->isReady()) + { + addresses += (is_first ? "" : "; ") + replica->connection->getDescription(); + is_first = false; + } + } + } return addresses; } @@ -182,14 +200,15 @@ void HedgedConnections::sendCancel() { std::lock_guard lock(cancel_mutex); +// LOG_DEBUG(log, "sendCancel"); + if (!sent_query || cancelled) throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR); - if (replicas.first_replica->isReady()) - replicas.first_replica->connection->sendCancel(); - - if (replicas.second_replica->isReady()) - replicas.second_replica->connection->sendCancel(); + for (auto & replicas_with_same_offset : replicas) + for (auto & replica : replicas_with_same_offset) + if (replica->isReady()) + replica->connection->sendCancel(); cancelled = true; } @@ -202,6 +221,8 @@ Packet HedgedConnections::drain() if (!cancelled) throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR); +// LOG_DEBUG(log, "drain"); + Packet res; res.type = Protocol::Server::EndOfStream; @@ -250,23 +271,31 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { +// LOG_DEBUG(log, "sreceivePacketImpl"); + int event_fd; - ReplicaStatePtr replica; + ReplicaStatePtr replica = nullptr; Packet packet; bool finish = false; while (!finish) { - event_fd = get_hedged_connections.getReadyFileDescriptor(epoll, async_callback); + event_fd = getReadyFileDescriptor(async_callback); - if (auto timeout_descriptor = get_hedged_connections.isEventTimeout(event_fd, replica)) - processTimeoutEvent(replica, timeout_descriptor); - else if ((replica = get_hedged_connections.isEventReplica(event_fd))) + if (fd_to_replica.find(event_fd) != fd_to_replica.end()) { +// LOG_DEBUG(log, "event is replica"); + replica = fd_to_replica[event_fd]; packet = receivePacketFromReplica(replica, async_callback); finish = true; } + else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) + { +// LOG_DEBUG(log, "event is timeout"); + replica = timeout_fd_to_replica[event_fd]; + processTimeoutEvent(replica, replica->active_timeouts[event_fd].get()); + } else if (event_fd == get_hedged_connections.getFileDescriptor()) - processGetHedgedConnectionsEvent(); + tryGetNewReplica(); else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } @@ -274,23 +303,33 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) return packet; }; -Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback) +int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { + for (auto & [fd, replica] : fd_to_replica) + if (replica->connection->hasReadPendingData()) + return replica->fd; + + return epoll.getReady(std::move(async_callback)).data.fd; +} + +Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) +{ +// LOG_DEBUG(log, "sreceivePacketFromReplica"); Packet packet = replica->connection->receivePacket(std::move(async_callback)); switch (packet.type) { case Protocol::Server::Data: - removeTimeoutsFromReplica(replica, epoll); + removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); processReceiveData(replica); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, get_hedged_connections.getConnectionTimeouts()); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, get_hedged_connections.getConnectionTimeouts()); break; case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: - removeTimeoutFromReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, get_hedged_connections.getConnectionTimeouts()); + removeTimeoutFromReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica); + addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, get_hedged_connections.getConnectionTimeouts()); break; case Protocol::Server::EndOfStream: @@ -306,26 +345,29 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, Asyn return packet; } -void HedgedConnections::processReceiveData(ReplicaStatePtr replica) +void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) { /// When we receive first packet of data from any replica, we continue working with this replica - /// and stop working with another replica (if there is another replica). If current replica is - /// second, move it to the first place. - if (replica == replicas.second_replica) - get_hedged_connections.swapReplicas(); + /// and stop working with other replicas (if there are other replicas). - if (replicas.second_replica->isCannotChoose() || replicas.second_replica->isEmpty()) - return; +// LOG_DEBUG(log, "processReceiveData"); - if (replicas.second_replica->isNotReady()) + offsets_with_received_data.insert(replica->parallel_replica_offset); + + for (auto & other_replica : replicas[replica->parallel_replica_offset]) { - get_hedged_connections.stopChoosingSecondReplica(); - epoll.remove(get_hedged_connections.getFileDescriptor()); + if (other_replica->isReady() && other_replica != replica) + { + other_replica->connection->sendCancel(); + finishProcessReplica(other_replica, true); + } } - else if (replicas.second_replica->isReady()) + + if (get_hedged_connections.hasEventsInProcess() && offsets_with_received_data.size() == replicas.size()) { - replicas.second_replica->connection->sendCancel(); - finishProcessReplica(replicas.second_replica, true); + get_hedged_connections.stopChoosingReplicas(); + if (next_replica_in_process) + epoll.remove(get_hedged_connections.getFileDescriptor()); } } @@ -333,57 +375,73 @@ void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDesc { epoll.remove(timeout_descriptor->getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); + timeout_fd_to_replica.erase(timeout_descriptor->getDescriptor()); if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) { + size_t offset = replica->parallel_replica_offset; finishProcessReplica(replica, true); - if (!replicas.first_replica->isReady() && !replicas.second_replica->isNotReady()) + /// Check if there is no active connection with same offset. + if (active_connections_count_by_offset[offset] == 0) throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); } else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_DATA_TIMEOUT) { - if (!replicas.second_replica->isEmpty()) - throw Exception("Cannot start choosing second replica, it's not empty", ErrorCodes::LOGICAL_ERROR); - - get_hedged_connections.chooseSecondReplica(); - - if (replicas.second_replica->isReady()) - processChosenSecondReplica(); - else if (replicas.second_replica->isNotReady()) - epoll.add(get_hedged_connections.getFileDescriptor()); + offsets_queue.push(replica->parallel_replica_offset); + tryGetNewReplica(); } } -void HedgedConnections::processGetHedgedConnectionsEvent() +void HedgedConnections::tryGetNewReplica() { - get_hedged_connections.chooseSecondReplica(); - if (replicas.second_replica->isReady()) - processChosenSecondReplica(); +// LOG_DEBUG(log, "tryGetNewReplica"); - if (!replicas.second_replica->isNotReady()) + ReplicaStatePtr new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); + + /// Skip replicas with old server version if we didn't disable two-level aggregation in sendQuery. + while (new_replica->isReady() && !has_two_level_aggregation_incompatibility + && new_replica->connection->getServerRevision(get_hedged_connections.getConnectionTimeouts()) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); + + if (new_replica->isReady()) + { +// LOG_DEBUG(log, "processNewReadyReplica"); + new_replica->parallel_replica_offset = offsets_queue.front(); + offsets_queue.pop(); + replicas[new_replica->parallel_replica_offset].push_back(new_replica); + epoll.add(new_replica->fd); + fd_to_replica[new_replica->fd] = new_replica; + ++active_connections_count_by_offset[new_replica->parallel_replica_offset]; + pipeline_for_new_replicas.run(new_replica); + } + else if (new_replica->isNotReady() && !next_replica_in_process) + { + epoll.add(get_hedged_connections.getFileDescriptor()); + next_replica_in_process = true; + } + + if (next_replica_in_process && (new_replica->isCannotChoose() || offsets_queue.empty())) + { epoll.remove(get_hedged_connections.getFileDescriptor()); + next_replica_in_process = false; + } } -void HedgedConnections::processChosenSecondReplica() +void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect) { - second_replica_pipeline.run(replicas.second_replica); +// LOG_DEBUG(log, "finishProcessReplica"); - /// In case when the first replica get receive timeout before the second is chosen, - /// we need to move the second replica to the first place - get_hedged_connections.swapReplicasIfNeeded(); -} - -void HedgedConnections::finishProcessReplica(ReplicaStatePtr replica, bool disconnect) -{ - removeTimeoutsFromReplica(replica, epoll); + removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); epoll.remove(replica->fd); + fd_to_replica.erase(replica->fd); + --active_connections_count_by_offset[replica->parallel_replica_offset]; + if (active_connections_count_by_offset[replica->parallel_replica_offset] == 0) + active_connections_count_by_offset.erase(replica->parallel_replica_offset); + if (disconnect) replica->connection->disconnect(); replica->reset(); - - /// Move active connection to the first replica if it exists - get_hedged_connections.swapReplicasIfNeeded(); } } diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index b84ad89658f..b6e64ac45ad 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -17,6 +18,7 @@ public: const Settings & settings_, const ConnectionTimeouts & timeouts_, const ThrottlerPtr & throttler, + PoolMode pool_mode, std::shared_ptr table_to_check_ = nullptr); void sendScalarsData(Scalars & data) override; @@ -45,47 +47,52 @@ public: size_t size() const override; - bool hasActiveConnections() const override; + bool hasActiveConnections() const override { return !active_connections_count_by_offset.empty(); } private: class Pipeline { public: - void add(std::function send_function); + void add(std::function send_function); - void run(ReplicaStatePtr replica); + void run(ReplicaStatePtr & replica); bool empty() const { return pipeline.empty(); } private: - std::vector> pipeline; + std::vector> pipeline; }; - void processChosenSecondReplica(); - - Packet receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback = {}); Packet receivePacketImpl(AsyncCallback async_callback = {}); - void processReceiveData(ReplicaStatePtr replica); + void processReceiveData(ReplicaStatePtr & replica); void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor); - void processGetHedgedConnectionsEvent(); + void tryGetNewReplica(); - void removeReceiveTimeout(ReplicaStatePtr replica); + void finishProcessReplica(ReplicaStatePtr & replica, bool disconnect); - void finishProcessReplica(ReplicaStatePtr replica, bool disconnect); + int getReadyFileDescriptor(AsyncCallback async_callback = {}); GetHedgedConnections get_hedged_connections; - Replicas replicas; + std::vector> replicas; + std::unordered_map fd_to_replica; + std::unordered_map timeout_fd_to_replica; + std::queue offsets_queue; Epoll epoll; const Settings & settings; ThrottlerPtr throttler; Poco::Logger * log; - Pipeline second_replica_pipeline; + Pipeline pipeline_for_new_replicas; bool sent_query = false; bool cancelled = false; + std::unordered_map active_connections_count_by_offset; + bool next_replica_in_process = false; + bool has_two_level_aggregation_incompatibility = false; + std::unordered_set offsets_with_received_data; mutable std::mutex cancel_mutex; }; diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 49654b51199..01c31eac640 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -63,13 +63,13 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Settings & current_settings = context.getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - if (current_settings.use_hedged_requests && current_settings.max_parallel_replicas <= 1) + if (current_settings.use_hedged_requests) { std::shared_ptr table_to_check = nullptr; if (main_table) table_to_check = std::make_shared(main_table.getQualifiedName()); - return std::make_unique(pool, current_settings, timeouts, throttler, table_to_check); + return std::make_unique(pool, current_settings, timeouts, throttler, pool_mode, table_to_check); } else { From 1f22ba4bbb384c72f6fc57538c7ebb13dacd73ca Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 27 Jan 2021 12:35:08 +0300 Subject: [PATCH 014/510] DOCSUP-5266: fix PR and ticket comments --- .../data-types/simpleaggregatefunction.md | 6 +++++- .../data-types/simpleaggregatefunction.md | 21 ++++++++++++------- 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index 2d2746f85d3..015972d7dbe 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -21,7 +21,11 @@ The following aggregate functions are supported: - [`argMin`](../../sql-reference/aggregate-functions/reference/argmin.md) - [`argMax`](../../sql-reference/aggregate-functions/reference/argmax.md) -Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. + +!!! note "Note" + Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. + + `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. **Parameters** diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 3ff4e5fd662..84e20877866 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -1,8 +1,9 @@ # SimpleAggregateFunction {#data-type-simpleaggregatefunction} -`SimpleAggregateFunction(name, types_of_arguments…)` data type stores current value of the aggregate function, and does not store its full state as [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md) does. This optimization can be applied to functions for which the following property holds: the result of applying a function `f` to a row set `S1 UNION ALL S2` can be obtained by applying `f` to parts of the row set separately, and then again applying `f` to the results: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. This property guarantees that partial aggregation results are enough to compute the combined one, so we don’t have to store and process any extra data. +Тип данных `SimpleAggregateFunction(name, types_of_arguments…)` хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`] (../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, для которых выполняется следующее свойство: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, +а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому нам не нужно хранить и обрабатывать какие-либо дополнительные данные. -The following aggregate functions are supported: +Поддерживаются следующие агрегатные функции: - [`any`](../../sql-reference/aggregate-functions/reference/any.md#agg_function-any) - [`anyLast`](../../sql-reference/aggregate-functions/reference/anylast.md#anylastx) @@ -19,14 +20,18 @@ The following aggregate functions are supported: - [`minMap`](../../sql-reference/aggregate-functions/reference/minmap.md#agg_functions-minmap) - [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap) -Values of the `SimpleAggregateFunction(func, Type)` look and stored the same way as `Type`, so you do not need to apply functions with `-Merge`/`-State` suffixes. `SimpleAggregateFunction` has better performance than `AggregateFunction` with same aggregation function. +!!! note "Примечание" + Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому вам не требуется применять функции с суффиксами `-Merge`/`-State`. + + `SimpleAggregateFunction` имеет лучшую производительность, чем `AggregateFunction` с той же агрегатной функцией. -**Parameters** -- Name of the aggregate function. -- Types of the aggregate function arguments. +**Параметры** -**Example** +- имя агрегатной функции. +- типы аргументов агрегатной функции. + +**Пример** ``` sql CREATE TABLE t @@ -36,4 +41,4 @@ CREATE TABLE t ) ENGINE = ... ``` -[Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) +[Оригинальная статья](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From 68119d78680b0e6dc181caf81eb8e7724ce8c535 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Wed, 27 Jan 2021 12:50:49 +0300 Subject: [PATCH 015/510] DOCSUP-5266: fix PR and ticket comments --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 84e20877866..c1b3ac240f0 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -1,6 +1,6 @@ # SimpleAggregateFunction {#data-type-simpleaggregatefunction} -Тип данных `SimpleAggregateFunction(name, types_of_arguments…)` хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`] (../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, для которых выполняется следующее свойство: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, +Тип данных `SimpleAggregateFunction(name, types_of_arguments…)` хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, для которых выполняется следующее свойство: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому нам не нужно хранить и обрабатывать какие-либо дополнительные данные. Поддерживаются следующие агрегатные функции: From 6a922959826441028885069d707d10a6946ac482 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Wed, 27 Jan 2021 19:16:31 +0400 Subject: [PATCH 016/510] Remove TODOs --- src/Interpreters/ExpressionAnalyzer.cpp | 4 ---- src/Interpreters/InterpreterSelectQuery.cpp | 4 ---- 2 files changed, 8 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2055faca820..8599f5c15f8 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1371,8 +1371,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && filter_info_) { - // TODO: handle filter exactly like prewhere, store the info in PrewhereDAGInfo, collect unnecessary columns, etc.? - filter_info = filter_info_; query_analyzer.appendPreliminaryFilter(chain, filter_info->actions_dag, filter_info->column_name); } @@ -1547,8 +1545,6 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si { const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); filter_info->do_remove_column = step.can_remove_required_output.at(0); - - // TODO: handle filter exactly like prewhere, collect columns to remove after filter? } if (hasPrewhere()) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 57c18f1bb86..946f22198fb 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -823,8 +823,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu row_level_security_step->setStepDescription("Row-level security filter (PREWHERE)"); query_plan.addStep(std::move(row_level_security_step)); - - // TODO: handle filter like prewhere, remove unnecessary columns after it, etc.? } if (expressions.prewhere_info) @@ -953,8 +951,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu row_level_security_step->setStepDescription("Row-level security filter"); query_plan.addStep(std::move(row_level_security_step)); - - // TODO: handle filter like prewhere, remove unnecessary columns after it, etc.? } if (expressions.before_array_join) From 9c7881f4c9dba5ce9fe241603368228fc87e9420 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Jan 2021 09:22:01 +0000 Subject: [PATCH 017/510] Fix --- .../AggregateFunctionFactory.cpp | 7 +++++- src/DataTypes/DataTypeFactory.cpp | 23 +++++++++++-------- src/Functions/FunctionFactory.cpp | 3 ++- ...56_test_query_log_factories_info.reference | 10 ++++---- .../01656_test_query_log_factories_info.sql | 4 +++- 5 files changed, 29 insertions(+), 18 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index 5fc690d59f2..53fc895849b 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -98,6 +98,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( bool has_null_arguments) const { String name = getAliasToOrName(name_param); + bool is_case_insensitive = false; Value found; /// Find by exact match. @@ -107,7 +108,10 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( } if (auto jt = case_insensitive_aggregate_functions.find(Poco::toLower(name)); jt != case_insensitive_aggregate_functions.end()) + { found = jt->second; + is_case_insensitive = true; + } const Context * query_context = nullptr; if (CurrentThread::isInitialized()) @@ -118,7 +122,8 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( out_properties = found.properties; if (query_context && query_context->getSettingsRef().log_queries) - query_context->addQueryFactoriesInfo(Context::QueryLogFactories::AggregateFunction, name); + query_context->addQueryFactoriesInfo( + Context::QueryLogFactories::AggregateFunction, is_case_insensitive ? Poco::toLower(name) : name); /// The case when aggregate function should return NULL on NULL arguments. This case is handled in "get" method. if (!out_properties.returns_default_when_only_null && has_null_arguments) diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index 2f100202ee9..1bc2a307915 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -78,16 +78,7 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr return get("LowCardinality", low_cardinality_params); } - DataTypePtr res = findCreatorByName(family_name)(parameters); - - if (CurrentThread::isInitialized()) - { - const auto * query_context = CurrentThread::get().getQueryContext(); - if (query_context && query_context->getSettingsRef().log_queries) - query_context->addQueryFactoriesInfo(Context::QueryLogFactories::DataType, family_name); - } - - return res; + return findCreatorByName(family_name)(parameters); } DataTypePtr DataTypeFactory::getCustom(DataTypeCustomDescPtr customization) const @@ -159,10 +150,18 @@ void DataTypeFactory::registerSimpleDataTypeCustom(const String &name, SimpleCre const DataTypeFactory::Value & DataTypeFactory::findCreatorByName(const String & family_name) const { + const Context * query_context = nullptr; + if (CurrentThread::isInitialized()) + query_context = CurrentThread::get().getQueryContext(); + { DataTypesDictionary::const_iterator it = data_types.find(family_name); if (data_types.end() != it) + { + if (query_context && query_context->getSettingsRef().log_queries) + query_context->addQueryFactoriesInfo(Context::QueryLogFactories::DataType, family_name); return it->second; + } } String family_name_lowercase = Poco::toLower(family_name); @@ -170,7 +169,11 @@ const DataTypeFactory::Value & DataTypeFactory::findCreatorByName(const String & { DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase); if (case_insensitive_data_types.end() != it) + { + if (query_context && query_context->getSettingsRef().log_queries) + query_context->addQueryFactoriesInfo(Context::QueryLogFactories::DataType, family_name_lowercase); return it->second; + } } auto hints = this->getHints(family_name); diff --git a/src/Functions/FunctionFactory.cpp b/src/Functions/FunctionFactory.cpp index 768f1cfe487..e98cb543df6 100644 --- a/src/Functions/FunctionFactory.cpp +++ b/src/Functions/FunctionFactory.cpp @@ -92,7 +92,8 @@ FunctionOverloadResolverImplPtr FunctionFactory::tryGetImpl( res = it->second(context); else { - it = case_insensitive_functions.find(Poco::toLower(name)); + name = Poco::toLower(name); + it = case_insensitive_functions.find(name); if (case_insensitive_functions.end() != it) res = it->second(context); } diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference index 3c93cd9ec26..77486e99ea5 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference @@ -1,8 +1,8 @@ -2 worl [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 -2 worl [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 +2 worl 1 0.7615946626193841 0 4950 99 [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 +2 worl 1 0.7615946626193841 0 4950 99 [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 arraySort(used_aggregate_functions) -['avg','count','groupBitAnd','sum','uniq'] +['avg','count','groupBitAnd','max','sum','uniq'] arraySort(used_aggregate_function_combinators) ['Array','If','OrDefault','OrNull'] @@ -11,7 +11,7 @@ arraySort(used_table_functions) ['numbers'] arraySort(used_functions) -['addDays','array','arrayFlatten','cast','modulo','plus','substring','toDate','toDayOfYear','toTypeName','toWeek'] +['addDays','array','arrayFlatten','cast','crc32','modulo','plus','pow','substring','tanh','toDate','toDayOfYear','toTypeName','toWeek'] arraySort(used_data_type_families) ['Array','Int32','Nullable','String'] @@ -20,5 +20,5 @@ used_database_engines ['Atomic'] arraySort(used_data_type_families) used_storages -['DateTime','Int64'] ['Memory'] +['Int64','datetime'] ['Memory'] diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql index aa9bdd42a71..0856681e9c5 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql @@ -1,5 +1,7 @@ SELECT uniqArray([1, 1, 2]), SUBSTRING('Hello, world', 7, 5), + POW(1, 2), TANh(1), CrC32(''), + SUM(number), MAX(number), flatten([[[BIT_AND(123)]], [[mod(3, 2)], [CAST('1' AS INTEGER)]]]), week(toDate('2000-12-05')), CAST(arrayJoin([NULL, NULL]) AS Nullable(TEXT)), @@ -47,7 +49,7 @@ WHERE current_database = currentDatabase() AND type == 'QueryFinish' AND (query ORDER BY query_start_time DESC LIMIT 1 FORMAT TabSeparatedWithNames; SELECT ''; -CREATE OR REPLACE TABLE test_query_log_factories_info1.memory_table (id BIGINT, date DateTime) ENGINE=Memory(); +CREATE OR REPLACE TABLE test_query_log_factories_info1.memory_table (id BIGINT, date DATETIME) ENGINE=Memory(); SYSTEM FLUSH LOGS; SELECT arraySort(used_data_type_families), used_storages From 65c061de4978f83c048cfd4c0292a81510ae7bfb Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 28 Jan 2021 13:28:11 +0000 Subject: [PATCH 018/510] FFix --- .../01656_test_query_log_factories_info.reference | 6 +++--- .../0_stateless/01656_test_query_log_factories_info.sql | 2 +- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference index 77486e99ea5..e12ee221a7b 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference @@ -1,5 +1,5 @@ -2 worl 1 0.7615946626193841 0 4950 99 [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 -2 worl 1 0.7615946626193841 0 4950 99 [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 +2 worl 1 1 0 4950 99 [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 +2 worl 1 1 0 4950 99 [123,1,1] 49 \N 50 4950 Nullable(UInt64) 50 arraySort(used_aggregate_functions) ['avg','count','groupBitAnd','max','sum','uniq'] @@ -11,7 +11,7 @@ arraySort(used_table_functions) ['numbers'] arraySort(used_functions) -['addDays','array','arrayFlatten','cast','crc32','modulo','plus','pow','substring','tanh','toDate','toDayOfYear','toTypeName','toWeek'] +['addDays','array','arrayFlatten','cast','crc32','modulo','plus','pow','round','substring','tanh','toDate','toDayOfYear','toTypeName','toWeek'] arraySort(used_data_type_families) ['Array','Int32','Nullable','String'] diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql index 0856681e9c5..b584f2c38c8 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.sql +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.sql @@ -1,6 +1,6 @@ SELECT uniqArray([1, 1, 2]), SUBSTRING('Hello, world', 7, 5), - POW(1, 2), TANh(1), CrC32(''), + POW(1, 2), ROUND(TANh(1)), CrC32(''), SUM(number), MAX(number), flatten([[[BIT_AND(123)]], [[mod(3, 2)], [CAST('1' AS INTEGER)]]]), week(toDate('2000-12-05')), From 2d0f742fdab2504402432580fda1b1f182aee4c7 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Thu, 28 Jan 2021 23:16:29 +0300 Subject: [PATCH 019/510] edited EN docs --- .../example-datasets/brown-benchmark.md | 6 +- .../functions/array-functions.md | 105 +++++++++++++++++- .../en/sql-reference/table-functions/mysql.md | 2 +- 3 files changed, 104 insertions(+), 9 deletions(-) diff --git a/docs/en/getting-started/example-datasets/brown-benchmark.md b/docs/en/getting-started/example-datasets/brown-benchmark.md index b5ca23eddb9..effae6d5adb 100644 --- a/docs/en/getting-started/example-datasets/brown-benchmark.md +++ b/docs/en/getting-started/example-datasets/brown-benchmark.md @@ -5,7 +5,7 @@ toc_title: Brown University Benchmark # Brown University Benchmark -MgBench - A new analytical benchmark for machine-generated log data, [Andrew Crotty](http://cs.brown.edu/people/acrotty/). +`MgBench` is a new analytical benchmark for machine-generated log data, [Andrew Crotty](http://cs.brown.edu/people/acrotty/). Download the data: ``` @@ -153,7 +153,7 @@ ORDER BY dt, hr; --- Q1.4: Over a 1-month period, how often was each server blocked on disk I/O? +-- Q1.4: Over 1 month, how often was each server blocked on disk I/O? SELECT machine_name, COUNT(*) AS spikes @@ -301,7 +301,7 @@ WHERE event_type = 'temperature' AND log_time >= '2019-11-29 17:00:00.000'; --- Q3.4: Over the past 6 months, how frequently was each door opened? +-- Q3.4: Over the past 6 months, how frequently were each door opened? SELECT device_name, device_floor, diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index dc7727bdfd8..48c5176f0e1 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1294,15 +1294,47 @@ Returns the min of the `func` values. If the function is omitted, it just return Note that the `arrayMin` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. -Examples: -```sql +**Syntax** + +``` sql +arrayMin(arr) +``` + +**Returned value** + +- A number. + +Type: [Int](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Parameters** + +- `arr` — [Array](../../sql-reference/data-types/array.md). + +**Examples** + +Query: + +``` sql SELECT arrayMin([1, 2, 4]) AS res +``` + +Result: + +``` text ┌─res─┐ │ 1 │ └─────┘ +``` +Query: +``` sql SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res +``` + +Result: + +``` text ┌─res─┐ │ -4 │ └─────┘ @@ -1314,15 +1346,47 @@ Returns the max of the `func` values. If the function is omitted, it just return Note that the `arrayMax` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. -Examples: +**Syntax** + +``` sql +arrayMax(arr) +``` + +**Returned value** + +- A number. + +Type: [Int](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Parameters** + +- `arr` — [Array](../../sql-reference/data-types/array.md). + +**Examples** + +Query: + ```sql SELECT arrayMax([1, 2, 4]) AS res +``` + +Result: + +``` text ┌─res─┐ │ 4 │ └─────┘ +``` +Query: +``` sql SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res +``` + +Result: + +``` text ┌─res─┐ │ -1 │ └─────┘ @@ -1334,21 +1398,52 @@ Returns the sum of the `func` values. If the function is omitted, it just return Note that the `arraySum` is a [higher-order function](../../sql-reference/functions/index.md#higher-order-functions). You can pass a lambda function to it as the first argument. -Examples: +**Syntax** + +``` sql +arraySum(arr) +``` + +**Returned value** + +- A number. + +Type: [Int](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-reference/data-types/float.md). + +**Parameters** + +- `arr` — [Array](../../sql-reference/data-types/array.md). + +**Examples** + +Query: + ```sql SELECT arraySum([2,3]) AS res +``` + +Result: + +``` text ┌─res─┐ │ 5 │ └─────┘ +``` +Query: +``` sql SELECT arraySum(x -> x*x, [2, 3]) AS res +``` + +Result: + +``` text ┌─res─┐ │ 13 │ └─────┘ ``` - ## arrayAvg(\[func,\] arr1, …) {#array-avg} Returns the average of the `func` values. If the function is omitted, it just returns the average of the array elements. diff --git a/docs/en/sql-reference/table-functions/mysql.md b/docs/en/sql-reference/table-functions/mysql.md index eec4a1d0c46..3126f635817 100644 --- a/docs/en/sql-reference/table-functions/mysql.md +++ b/docs/en/sql-reference/table-functions/mysql.md @@ -44,7 +44,7 @@ The rest of the conditions and the `LIMIT` sampling constraint are executed in C A table object with the same columns as the original MySQL table. !!! info "Note" - In the `INSERT` query to distinguish table function `mysql(...)` from table name with column names list you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. + In the `INSERT` query to distinguish table function `mysql(...)` from table name with column names list, you must use keywords `FUNCTION` or `TABLE FUNCTION`. See examples below. **Examples** From 45cb78a67b1ba39fe874817e523a7964751fb7cc Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 29 Jan 2021 08:14:34 +0000 Subject: [PATCH 020/510] continue of #19487 fix --- src/Interpreters/TreeRewriter.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index ce4103e97ec..a1d1605afd5 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -693,18 +693,17 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select if (storage) { - String hint_name{}; + std::vector hint_name{}; for (const auto & name : columns_context.requiredColumns()) { auto hints = storage->getHints(name); - if (!hints.empty()) - hint_name = hint_name + " '" + toString(hints) + "'"; + hint_name.insert(hint_name.end(), hints.begin(), hints.end()); } if (!hint_name.empty()) { ss << ", maybe you meant: "; - ss << hint_name; + ss << toString(hint_name); } } else From 4929fe2063f368e48bb53fde011487303426d460 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 Jan 2021 18:13:09 +0300 Subject: [PATCH 021/510] Update MergeTreeRangeReader --- .../MergeTree/MergeTreeRangeReader.cpp | 57 +++++++++++-------- src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- 2 files changed, 35 insertions(+), 24 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 0b3765adc6a..afbc28e6883 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -284,6 +284,13 @@ void MergeTreeRangeReader::ReadResult::shrink(Columns & old_columns) { if (!column) continue; + + if (const auto * column_const = typeid_cast(column.get())) + { + column = column_const->cloneResized(total_rows_per_granule); + continue; + } + auto new_column = column->cloneEmpty(); new_column->reserve(total_rows_per_granule); for (size_t j = 0, pos = 0; j < rows_per_granule_original.size(); pos += rows_per_granule_original[j++]) @@ -754,13 +761,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (read_result.num_rows == 0) return read_result; - if (prewhere_info_list) - { - for (const auto & prewhere_info : *prewhere_info_list) - { - executePrewhereActionsAndFilterColumns(read_result, prewhere_info); - } - } + executePrewhereActionsAndFilterColumns(read_result); return read_result; } @@ -857,8 +858,11 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & return columns; } -void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result, const PrewhereInfo & prewhere_info) +void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { + if (prewhere_info_list->empty()) + return; + const auto & header = merge_tree_reader->getColumns(); size_t num_columns = header.size(); @@ -867,7 +871,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r "Expected " + toString(num_columns) + ", " "got " + toString(result.columns.size()), ErrorCodes::LOGICAL_ERROR); - ColumnPtr filter; size_t prewhere_column_pos; { @@ -887,25 +890,33 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type) block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - if (prewhere_info.alias_actions) - prewhere_info.alias_actions->execute(block); + for (size_t i = 0; i < prewhere_info_list->size(); ++i) + { + const auto & prewhere_info = (*prewhere_info_list)[i]; - /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. - result.block_before_prewhere = block; - prewhere_info.prewhere_actions->execute(block); + if (prewhere_info.alias_actions) + prewhere_info.alias_actions->execute(block); - prewhere_column_pos = block.getPositionByName(prewhere_info.prewhere_column_name); + /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. + result.block_before_prewhere = block; + prewhere_info.prewhere_actions->execute(block); + prewhere_column_pos = block.getPositionByName(prewhere_info.prewhere_column_name); + result.addFilter(block.getByPosition(prewhere_column_pos).column); + + if (i + 1 != prewhere_info_list->size() && prewhere_info.remove_prewhere_column) + block.erase(prewhere_column_pos); + else + block.getByPosition(prewhere_column_pos).column = block.getByPosition(prewhere_column_pos).type->createColumnConst(result.num_rows, 1); + } + + block.getByPosition(prewhere_column_pos).column = nullptr; result.columns.clear(); result.columns.reserve(block.columns()); for (auto & col : block) result.columns.emplace_back(std::move(col.column)); - - filter.swap(result.columns[prewhere_column_pos]); } - result.addFilter(filter); - /// If there is a WHERE, we filter in there, and only optimize IO and shrink columns here if (!last_reader_in_chain) result.optimize(merge_tree_reader->canReadIncompleteGranules()); @@ -914,7 +925,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.totalRowsPerGranule() == 0) result.setFilterConstFalse(); /// If we need to filter in PREWHERE - else if (prewhere_info.need_filter || result.need_filter) + else if (prewhere_info_list->back().need_filter || result.need_filter) { /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) @@ -955,11 +966,11 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Check if the PREWHERE column is needed if (!result.columns.empty()) { - if (prewhere_info.remove_prewhere_column) + if (prewhere_info_list->back().remove_prewhere_column) result.columns.erase(result.columns.begin() + prewhere_column_pos); else result.columns[prewhere_column_pos] = - getSampleBlock().getByName(prewhere_info.prewhere_column_name).type-> + getSampleBlock().getByName(prewhere_info_list->back().prewhere_column_name).type-> createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } @@ -967,7 +978,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r else { result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); - if (getSampleBlock().getByName(prewhere_info.prewhere_column_name).type->isNullable()) + if (getSampleBlock().getByName(prewhere_info_list->back().prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); result.clearFilter(); // Acting as a flag to not filter in PREWHERE } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 8f8482d1abf..6ee7c9f3e29 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -213,7 +213,7 @@ private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); Columns continueReadingChain(ReadResult & result, size_t & num_rows); - void executePrewhereActionsAndFilterColumns(ReadResult & result, const PrewhereInfo & prewhere_info); + void executePrewhereActionsAndFilterColumns(ReadResult & result); IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; From 01a0cb649a3afc28726b36dbe1b10e4243ad34b2 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 29 Jan 2021 18:46:28 +0300 Subject: [PATCH 022/510] Fix build, style, tests --- src/Client/GetHedgedConnections.cpp | 9 ++++++--- src/Client/GetHedgedConnections.h | 11 +++++++---- src/Client/HedgedConnections.cpp | 17 ++++++++++------- src/Client/HedgedConnections.h | 4 +++- src/Client/MultiplexedConnections.cpp | 4 ++-- src/Client/MultiplexedConnections.h | 2 +- src/Common/Epoll.cpp | 3 +++ src/Common/Epoll.h | 4 ++++ src/DataStreams/RemoteQueryExecutor.cpp | 8 +++++++- 9 files changed, 43 insertions(+), 19 deletions(-) diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 4c729dc0722..7c1e7e1ced8 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -1,3 +1,5 @@ +#if defined(OS_LINUX) + #include #include @@ -72,7 +74,7 @@ std::vector GetHedgedConnections::getMany DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); throw DB::NetException( - "Could not connect to " + std::to_string(min_entries) + " replicas. Log: \n\n" + fail_messages + "\n", + "All connection tries failed. Log: \n\n" + fail_messages + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); } replicas.push_back(replica); @@ -89,7 +91,7 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bo int index; /// Check if it's the first time. - if (epoll.size() == 0 && ready_indexes.size() == 0) + if (epoll.empty() && ready_indexes.empty()) { index = 0; last_used_index = 0; @@ -99,7 +101,7 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bo bool is_first = true; - while (index != -1 || epoll.size() != 0) + while (index != -1 || !epoll.empty()) { if (index == -1 && !is_first && non_blocking) { @@ -515,3 +517,4 @@ void removeTimeoutFromReplica( } } +#endif diff --git a/src/Client/GetHedgedConnections.h b/src/Client/GetHedgedConnections.h index df060e9ecd5..88daad779fe 100644 --- a/src/Client/GetHedgedConnections.h +++ b/src/Client/GetHedgedConnections.h @@ -1,5 +1,7 @@ #pragma once +#if defined(OS_LINUX) + #include #include #include @@ -44,10 +46,10 @@ public: active_timeouts.clear(); } - bool isReady() const { return state == State::READY; }; - bool isNotReady() const { return state == State::NOT_READY; }; - bool isEmpty() const { return state == State::EMPTY; }; - bool isCannotChoose() const { return state == State::CANNOT_CHOOSE; }; + bool isReady() const { return state == State::READY; } + bool isNotReady() const { return state == State::NOT_READY; } + bool isEmpty() const { return state == State::EMPTY; } + bool isCannotChoose() const { return state == State::CANNOT_CHOOSE; } }; using ReplicaStatePtr = std::shared_ptr; @@ -162,3 +164,4 @@ void removeTimeoutsFromReplica( std::unordered_map & timeout_fd_to_replica); } +#endif diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index a4231b2c172..6dc746ec7f4 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -1,3 +1,5 @@ +#if defined(OS_LINUX) + #include #include @@ -142,7 +144,7 @@ void HedgedConnections::sendQuery( replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); - addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); + addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); }; for (auto & replicas_with_same_offset : replicas) @@ -178,12 +180,12 @@ std::string HedgedConnections::dumpAddresses() const // LOG_DEBUG(log, "dumpAddresses"); - std::string addresses = ""; + std::string addresses; bool is_first = true; - for (auto & replicas_with_same_offset : replicas) + for (const auto & replicas_with_same_offset : replicas) { - for (auto & replica : replicas_with_same_offset) + for (const auto & replica : replicas_with_same_offset) { if (replica->isReady()) { @@ -226,7 +228,7 @@ Packet HedgedConnections::drain() Packet res; res.type = Protocol::Server::EndOfStream; - while (epoll.size() != 0) + while (!epoll.empty()) { Packet packet = receivePacketImpl(); switch (packet.type) @@ -253,7 +255,7 @@ Packet HedgedConnections::drain() Packet HedgedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); - return receivePacketUnlocked(); + return receivePacketUnlocked({}); } Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) @@ -263,7 +265,7 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) if (!hasActiveConnections()) throw Exception("No more packets are available.", ErrorCodes::LOGICAL_ERROR); - if (epoll.size() == 0) + if (epoll.empty()) throw Exception("No pending events in epoll.", ErrorCodes::LOGICAL_ERROR); return receivePacketImpl(std::move(async_callback)); @@ -445,3 +447,4 @@ void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool dis } } +#endif diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index b6e64ac45ad..1400ff89de4 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -1,4 +1,5 @@ #pragma once +#if defined(OS_LINUX) #include #include @@ -35,7 +36,7 @@ public: Packet receivePacket() override; - Packet receivePacketUnlocked(AsyncCallback async_callback = {}) override; + Packet receivePacketUnlocked(AsyncCallback async_callback) override; void disconnect() override; @@ -98,3 +99,4 @@ private: }; } +#endif diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index fbf8c9aa172..3e7850e5f85 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -143,7 +143,7 @@ void MultiplexedConnections::sendQuery( Packet MultiplexedConnections::receivePacket() { std::lock_guard lock(cancel_mutex); - Packet packet = receivePacketUnlocked(); + Packet packet = receivePacketUnlocked({}); return packet; } @@ -191,7 +191,7 @@ Packet MultiplexedConnections::drain() while (hasActiveConnections()) { - Packet packet = receivePacketUnlocked(); + Packet packet = receivePacketUnlocked({}); switch (packet.type) { diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index 720add1ba81..a7c20200fcf 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -56,7 +56,7 @@ public: bool hasActiveConnections() const override { return active_connection_count > 0; } private: - Packet receivePacketUnlocked(AsyncCallback async_callback = {}) override; + Packet receivePacketUnlocked(AsyncCallback async_callback) override; /// Internal version of `dumpAddresses` function without locking. std::string dumpAddressesUnlocked() const; diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 8ce100c7834..cb34f81cf36 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -1,3 +1,5 @@ +#if defined(OS_LINUX) + #include "Epoll.h" #include #include @@ -80,3 +82,4 @@ Epoll::~Epoll() } } +#endif diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index 0e04d666af0..1dc65d15d08 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -1,4 +1,5 @@ #pragma once +#if defined(OS_LINUX) #include #include @@ -34,6 +35,8 @@ public: int size() const { return events_count; } + bool empty() const { return events_count == 0; } + ~Epoll(); private: @@ -42,3 +45,4 @@ private: }; } +#endif diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 01c31eac640..52a7a3e0a78 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -63,7 +63,13 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Settings & current_settings = context.getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - if (current_settings.use_hedged_requests) + bool use_hedged_requests = current_settings.use_hedged_requests; + +#if !defined(OS_LINUX) + use_hedged_requests = false; +#endif + + if (use_hedged_requests) { std::shared_ptr table_to_check = nullptr; if (main_table) From afdc9635cb1183c6c37e3ccd804b5a8ca6498311 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 Jan 2021 19:12:53 +0300 Subject: [PATCH 023/510] Update MergeTreeRangeReader --- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++++++ src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index db0edefb5e8..6a0e2515801 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1579,6 +1579,12 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc std::make_shared(expressions.filter_info->actions_dag), expressions.filter_info->column_name); + if (alias_actions) + { + query_info.prewhere_info_list->back().alias_actions = std::make_shared(alias_actions); + alias_actions = nullptr; + } + auto & new_filter_info = query_info.prewhere_info_list->front(); new_filter_info.remove_prewhere_column = expressions.filter_info->do_remove_column; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index afbc28e6883..14ab70a992c 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -860,7 +860,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - if (prewhere_info_list->empty()) + if (!prewhere_info_list || prewhere_info_list->empty()) return; const auto & header = merge_tree_reader->getColumns(); From 47a7273fe363f4577e9dbb8e7ca4a95da600ec02 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 29 Jan 2021 19:21:52 +0300 Subject: [PATCH 024/510] Fix build --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 14ab70a992c..7c9b1b36b33 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -871,7 +871,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r "Expected " + toString(num_columns) + ", " "got " + toString(result.columns.size()), ErrorCodes::LOGICAL_ERROR); - size_t prewhere_column_pos; + size_t prewhere_column_pos = 0; { /// Restore block from columns list. From 60295978779db573fe80f54c38c47664459532bc Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 29 Jan 2021 19:56:47 +0300 Subject: [PATCH 025/510] Fix style error --- src/Client/GetHedgedConnections.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 7c1e7e1ced8..269edd252a2 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -11,6 +11,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int ALL_CONNECTION_TRIES_FAILED; + extern const int ALL_REPLICAS_ARE_STALE; } GetHedgedConnections::GetHedgedConnections( From d27f5114c5697c212f211cb389bffa44d0c36b54 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 29 Jan 2021 23:21:11 +0300 Subject: [PATCH 026/510] Add LOG_DEBUG for tests debug --- src/Client/Connection.cpp | 30 ++++++++++++------------ src/Client/GetHedgedConnections.cpp | 36 ++++++++++++++--------------- src/Client/HedgedConnections.cpp | 30 ++++++++++++------------ 3 files changed, 48 insertions(+), 48 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 0bf0456c79c..42d9c86739e 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -92,7 +92,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) void Connection::disconnect() { -// LOG_DEBUG(log_wrapper.get(), "disconnect"); + LOG_DEBUG(log_wrapper.get(), "disconnect"); maybe_compressed_out = nullptr; in = nullptr; @@ -106,7 +106,7 @@ void Connection::disconnect() void Connection::prepare(const ConnectionTimeouts & timeouts) { -// LOG_DEBUG(log_wrapper.get(), "Connect"); + LOG_DEBUG(log_wrapper.get(), "Connect"); LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", default_database.empty() ? "(not specified)" : default_database, @@ -160,7 +160,7 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) void Connection::sendHello() { -// LOG_DEBUG(log_wrapper.get(), "sendHello"); + LOG_DEBUG(log_wrapper.get(), "sendHello"); /** Disallow control characters in user controlled parameters * to mitigate the possibility of SSRF. @@ -218,7 +218,7 @@ void Connection::sendHello() void Connection::receiveHello() { -// LOG_DEBUG(log_wrapper.get(), "receiveHello"); + LOG_DEBUG(log_wrapper.get(), "receiveHello"); /// Receive hello packet. UInt64 packet_type = 0; @@ -323,7 +323,7 @@ const String & Connection::getServerDisplayName(const ConnectionTimeouts & timeo void Connection::forceConnected(const ConnectionTimeouts & timeouts) { -// LOG_DEBUG(log_wrapper.get(), "forceConnected"); + LOG_DEBUG(log_wrapper.get(), "forceConnected"); if (!connected) { @@ -351,7 +351,7 @@ void Connection::sendClusterNameAndSalt() bool Connection::ping() { -// LOG_DEBUG(log_wrapper.get(), "ping"); + LOG_DEBUG(log_wrapper.get(), "ping"); TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); try @@ -404,7 +404,7 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) { -// LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest"); + LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest"); writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); @@ -413,7 +413,7 @@ void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) TablesStatusResponse Connection::receiveTablesStatusResponse() { -// LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse"); + LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse"); UInt64 response_type = 0; readVarUInt(response_type, *in); @@ -440,7 +440,7 @@ void Connection::sendQuery( if (!connected) connect(timeouts); -// LOG_DEBUG(log_wrapper.get(), "sendQuery"); + LOG_DEBUG(log_wrapper.get(), "sendQuery"); TimeoutSetter timeout_setter(*socket, timeouts.send_timeout, timeouts.receive_timeout, true); @@ -540,7 +540,7 @@ void Connection::sendCancel() if (!out) return; -// LOG_DEBUG(log_wrapper.get(), "sendCancel"); + LOG_DEBUG(log_wrapper.get(), "sendCancel"); writeVarUInt(Protocol::Client::Cancel, *out); out->next(); @@ -549,7 +549,7 @@ void Connection::sendCancel() void Connection::sendData(const Block & block, const String & name, bool scalar) { -// LOG_DEBUG(log_wrapper.get(), "sendData"); + LOG_DEBUG(log_wrapper.get(), "sendData"); if (!block_out) { @@ -581,7 +581,7 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name) { /// NOTE 'Throttler' is not used in this method (could use, but it's not important right now). -// LOG_DEBUG(log_wrapper.get(), "sendPreparedData"); + LOG_DEBUG(log_wrapper.get(), "sendPreparedData"); if (input.eof()) throw Exception("Buffer is empty (some kind of corruption)", ErrorCodes::EMPTY_DATA_PASSED); @@ -602,7 +602,7 @@ void Connection::sendScalarsData(Scalars & data) if (data.empty()) return; -// LOG_DEBUG(log_wrapper.get(), "sendScalarsData"); + LOG_DEBUG(log_wrapper.get(), "sendScalarsData"); Stopwatch watch; size_t out_bytes = out ? out->count() : 0; @@ -689,7 +689,7 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) return; } -// LOG_DEBUG(log_wrapper.get(), "sendExternalTablesData"); + LOG_DEBUG(log_wrapper.get(), "sendExternalTablesData"); Stopwatch watch; size_t out_bytes = out ? out->count() : 0; @@ -789,7 +789,7 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) Packet Connection::receivePacket(AsyncCallback async_callback) { -// LOG_DEBUG(log_wrapper.get(), "receivePacket"); + LOG_DEBUG(log_wrapper.get(), "receivePacket"); in->setAsyncCallback(std::move(async_callback)); SCOPE_EXIT(in->setAsyncCallback({})); diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 269edd252a2..546068ca4ee 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -86,7 +86,7 @@ std::vector GetHedgedConnections::getMany GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bool non_blocking) { -// LOG_DEBUG(log, "getNextConnection"); + LOG_DEBUG(log, "getNextConnection"); ReplicaStatePtr replica = createNewReplica(); int index; @@ -161,7 +161,7 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bo void GetHedgedConnections::stopChoosingReplicas() { -// LOG_DEBUG(log, "stopChoosingReplicas"); + LOG_DEBUG(log, "stopChoosingReplicas"); for (auto & [fd, replica] : fd_to_replica) { removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); @@ -195,7 +195,7 @@ int GetHedgedConnections::getNextIndex() return -1; } -// LOG_DEBUG(log, "get next index: {}", next_index); + LOG_DEBUG(log, "get next index: {}", next_index); last_used_index = next_index; return next_index; @@ -203,7 +203,7 @@ int GetHedgedConnections::getNextIndex() GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr & replica) { -// LOG_DEBUG(log, "start try get connection with {} replica", index); + LOG_DEBUG(log, "start try get connection with {} replica", index); TryGetConnection & try_get_connection = try_get_connections[index]; replica->state = State::NOT_READY; @@ -240,14 +240,14 @@ GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int ind GetHedgedConnections::Action GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll) { -// LOG_DEBUG(log, "process get connection stage for {} replica", replica->index); + LOG_DEBUG(log, "process get connection stage for {} replica", replica->index); TryGetConnection & try_get_connection = try_get_connections[replica->index]; if (try_get_connection.stage == TryGetConnection::Stage::FINISHED) { indexes_in_process.erase(replica->index); -// LOG_DEBUG(log, "stage: FINISHED"); + LOG_DEBUG(log, "stage: FINISHED"); ++entries_count; if (remove_from_epoll) @@ -258,11 +258,11 @@ GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bo if (try_get_connection.result.is_usable) { -// LOG_DEBUG(log, "replica is usable"); + LOG_DEBUG(log, "replica is usable"); ++usable_count; if (try_get_connection.result.is_up_to_date) { -// LOG_DEBUG(log, "replica is up to date, finish get hedged connections"); + LOG_DEBUG(log, "replica is up to date, finish get hedged connections"); replica->state = State::READY; ready_indexes.insert(replica->index); return Action::FINISH; @@ -276,12 +276,12 @@ GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bo } else if (try_get_connection.stage == TryGetConnection::Stage::FAILED) { -// LOG_DEBUG(log, "stage: FAILED"); + LOG_DEBUG(log, "stage: FAILED"); processFailedConnection(replica); return Action::TRY_NEXT_REPLICA; } -// LOG_DEBUG(log, "middle stage, process epoll events"); + LOG_DEBUG(log, "middle stage, process epoll events"); /// Get connection process is not finished return Action::PROCESS_EPOLL_EVENTS; @@ -289,7 +289,7 @@ GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bo void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica) { -// LOG_DEBUG(log, "failed connection with {} replica", replica->index); + LOG_DEBUG(log, "failed connection with {} replica", replica->index); ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; LOG_WARNING( @@ -314,7 +314,7 @@ void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica) void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica) { -// LOG_DEBUG(log, "add timeouts for {} replica", replica->index); + LOG_DEBUG(log, "add timeouts for {} replica", replica->index); addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); @@ -327,7 +327,7 @@ void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica) GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(bool non_blocking) { -// LOG_DEBUG(log, "process epoll events"); + LOG_DEBUG(log, "process epoll events"); int event_fd; ReplicaStatePtr replica = nullptr; bool finish = false; @@ -349,7 +349,7 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(b throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } -// LOG_DEBUG(log, "cancel process epoll events"); + LOG_DEBUG(log, "cancel process epoll events"); return replica; } @@ -365,7 +365,7 @@ int GetHedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking) { -// LOG_DEBUG(log, "epoll event is {} replica", replica->index); + LOG_DEBUG(log, "epoll event is {} replica", replica->index); removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); try_get_connections[replica->index].run(); Action action = processTryGetConnectionStage(replica, true); @@ -380,7 +380,7 @@ bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool n bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking) { -// LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index); + LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index); epoll.remove(timeout_descriptor->getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); @@ -388,7 +388,7 @@ bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerD if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) { -// LOG_DEBUG(log, "process receive timeout for {} replica", replica->index); + LOG_DEBUG(log, "process receive timeout for {} replica", replica->index); removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); epoll.remove(replica->fd); fd_to_replica.erase(replica->fd); @@ -415,7 +415,7 @@ bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerD void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) { -// LOG_DEBUG(log, "set best usable replica"); + LOG_DEBUG(log, "set best usable replica"); std::vector indexes(try_get_connections.size()); for (size_t i = 0; i != indexes.size(); ++i) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 6dc746ec7f4..4282e6b8e21 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -59,7 +59,7 @@ void HedgedConnections::sendScalarsData(Scalars & data) { std::lock_guard lock(cancel_mutex); -// LOG_DEBUG(log, "sendScalarsData"); + LOG_DEBUG(log, "sendScalarsData"); if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); @@ -78,7 +78,7 @@ void HedgedConnections::sendExternalTablesData(std::vector & { std::lock_guard lock(cancel_mutex); -// LOG_DEBUG(log, "sendExternalTablesData"); + LOG_DEBUG(log, "sendExternalTablesData"); if (!sent_query) throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); @@ -106,7 +106,7 @@ void HedgedConnections::sendQuery( { std::lock_guard lock(cancel_mutex); -// LOG_DEBUG(log, "sendQuery"); + LOG_DEBUG(log, "sendQuery"); if (sent_query) throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR); @@ -159,7 +159,7 @@ void HedgedConnections::disconnect() { std::lock_guard lock(cancel_mutex); -// LOG_DEBUG(log, "disconnect"); + LOG_DEBUG(log, "disconnect"); for (auto & replicas_with_same_offset : replicas) for (auto & replica : replicas_with_same_offset) @@ -178,7 +178,7 @@ std::string HedgedConnections::dumpAddresses() const { std::lock_guard lock(cancel_mutex); -// LOG_DEBUG(log, "dumpAddresses"); + LOG_DEBUG(log, "dumpAddresses"); std::string addresses; bool is_first = true; @@ -202,7 +202,7 @@ void HedgedConnections::sendCancel() { std::lock_guard lock(cancel_mutex); -// LOG_DEBUG(log, "sendCancel"); + LOG_DEBUG(log, "sendCancel"); if (!sent_query || cancelled) throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR); @@ -223,7 +223,7 @@ Packet HedgedConnections::drain() if (!cancelled) throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR); -// LOG_DEBUG(log, "drain"); + LOG_DEBUG(log, "drain"); Packet res; res.type = Protocol::Server::EndOfStream; @@ -273,7 +273,7 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { -// LOG_DEBUG(log, "sreceivePacketImpl"); + LOG_DEBUG(log, "sreceivePacketImpl"); int event_fd; ReplicaStatePtr replica = nullptr; @@ -285,14 +285,14 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) if (fd_to_replica.find(event_fd) != fd_to_replica.end()) { -// LOG_DEBUG(log, "event is replica"); + LOG_DEBUG(log, "event is replica"); replica = fd_to_replica[event_fd]; packet = receivePacketFromReplica(replica, async_callback); finish = true; } else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) { -// LOG_DEBUG(log, "event is timeout"); + LOG_DEBUG(log, "event is timeout"); replica = timeout_fd_to_replica[event_fd]; processTimeoutEvent(replica, replica->active_timeouts[event_fd].get()); } @@ -316,7 +316,7 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) { -// LOG_DEBUG(log, "sreceivePacketFromReplica"); + LOG_DEBUG(log, "sreceivePacketFromReplica"); Packet packet = replica->connection->receivePacket(std::move(async_callback)); switch (packet.type) { @@ -352,7 +352,7 @@ void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) /// When we receive first packet of data from any replica, we continue working with this replica /// and stop working with other replicas (if there are other replicas). -// LOG_DEBUG(log, "processReceiveData"); + LOG_DEBUG(log, "processReceiveData"); offsets_with_received_data.insert(replica->parallel_replica_offset); @@ -397,7 +397,7 @@ void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDesc void HedgedConnections::tryGetNewReplica() { -// LOG_DEBUG(log, "tryGetNewReplica"); + LOG_DEBUG(log, "tryGetNewReplica"); ReplicaStatePtr new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); @@ -408,7 +408,7 @@ void HedgedConnections::tryGetNewReplica() if (new_replica->isReady()) { -// LOG_DEBUG(log, "processNewReadyReplica"); + LOG_DEBUG(log, "processNewReadyReplica"); new_replica->parallel_replica_offset = offsets_queue.front(); offsets_queue.pop(); replicas[new_replica->parallel_replica_offset].push_back(new_replica); @@ -432,7 +432,7 @@ void HedgedConnections::tryGetNewReplica() void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect) { -// LOG_DEBUG(log, "finishProcessReplica"); + LOG_DEBUG(log, "finishProcessReplica"); removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); epoll.remove(replica->fd); From ddd828e7847da270d457b0c7e747b96c7a8ad81d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:46:26 +0300 Subject: [PATCH 027/510] Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index f11bec55697..624e04ca21c 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -602,7 +602,7 @@ This is necessary for searching for pageviews in the corresponding session. ## formatDateTime {#formatdatetime} -Function formats a Time according to the given Format string. N.B.: Format is a constant expression, e.g. you cannot have multiple formats for a single result column. +Formats a Time according to the given Format string. Format is a constant expression, so you cannot have multiple formats for a single result column. **Syntax** From 7a9863194a9310270c8b6f8ebd1d75195f7bae59 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:46:36 +0300 Subject: [PATCH 028/510] Update docs/ru/operations/utilities/clickhouse-local.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/operations/utilities/clickhouse-local.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/utilities/clickhouse-local.md b/docs/ru/operations/utilities/clickhouse-local.md index e3c421ac75e..f439049401c 100644 --- a/docs/ru/operations/utilities/clickhouse-local.md +++ b/docs/ru/operations/utilities/clickhouse-local.md @@ -77,7 +77,7 @@ $ clickhouse-local --query " 1 2 ``` -А теперь давайте выведем на экран объём оперативной памяти, занимаемой пользователями (Unix): +Объём оперативной памяти, занимаемой пользователями (Unix): Запрос: From 9e0d5c4c9819914d682806f1a7e550bff4125d61 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:46:53 +0300 Subject: [PATCH 029/510] Update docs/ru/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index c1b3ac240f0..2ca949843b7 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -1,6 +1,6 @@ # SimpleAggregateFunction {#data-type-simpleaggregatefunction} -Тип данных `SimpleAggregateFunction(name, types_of_arguments…)` хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, для которых выполняется следующее свойство: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, +Хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, которые обладают следующим свойством: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому нам не нужно хранить и обрабатывать какие-либо дополнительные данные. Поддерживаются следующие агрегатные функции: From 320e78dea614311bd8fcd7451906be1c90f71538 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:47:15 +0300 Subject: [PATCH 030/510] Update docs/ru/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 2ca949843b7..cb6c4b8208d 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -21,7 +21,7 @@ - [`maxMap`](../../sql-reference/aggregate-functions/reference/maxmap.md#agg_functions-maxmap) !!! note "Примечание" - Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому вам не требуется применять функции с суффиксами `-Merge`/`-State`. + Значения `SimpleAggregateFunction(func, Type)` отображаются и хранятся так же, как и `Type`, поэтому комбинаторы [-Merge](../../sql-reference/aggregate-functions/combinators.md#aggregate_functions_combinators-merge) и [-State]((../../sql-reference/aggregate-functions/combinators.md#agg-functions-combinator-state) не требуются. `SimpleAggregateFunction` имеет лучшую производительность, чем `AggregateFunction` с той же агрегатной функцией. From 7126ca376995fa58eb3f07a2c55ba4a5cd88a11f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:47:25 +0300 Subject: [PATCH 031/510] Update docs/ru/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index cb6c4b8208d..b906a56516f 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -29,7 +29,7 @@ **Параметры** - имя агрегатной функции. -- типы аргументов агрегатной функции. +- `type` — типы аргументов агрегатной функции. **Пример** From ac0ec2753c9a3021b3efaee1b7dbc4898242942f Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:47:33 +0300 Subject: [PATCH 032/510] Update docs/ru/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index b906a56516f..bf866f7bc58 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -1,4 +1,4 @@ -# SimpleAggregateFunction {#data-type-simpleaggregatefunction} +# SimpleAggregateFunction(func, type) {#data-type-simpleaggregatefunction} Хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, которые обладают следующим свойством: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому нам не нужно хранить и обрабатывать какие-либо дополнительные данные. From 004b9dd09823c729a800310c8449f56ad28bb51a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:47:41 +0300 Subject: [PATCH 033/510] Update docs/ru/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index bf866f7bc58..39f3ef99b1c 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -28,7 +28,7 @@ **Параметры** -- имя агрегатной функции. +- `func` — имя агрегатной функции. - `type` — типы аргументов агрегатной функции. **Пример** From ebcee0525d24464222534c002632589b9d1ad318 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Sat, 30 Jan 2021 18:47:50 +0300 Subject: [PATCH 034/510] Update docs/ru/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 39f3ef99b1c..10daad93cc6 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -1,7 +1,7 @@ # SimpleAggregateFunction(func, type) {#data-type-simpleaggregatefunction} Хранит только текущее значение агрегатной функции и не сохраняет ее полное состояние, как это делает [`AggregateFunction`](../../sql-reference/data-types/aggregatefunction.md). Такая оптимизация может быть применена к функциям, которые обладают следующим свойством: результат выполнения функции `f` к набору строк `S1 UNION ALL S2` может быть получен путем выполнения `f` к отдельным частям набора строк, -а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому нам не нужно хранить и обрабатывать какие-либо дополнительные данные. +а затем повторного выполнения `f` к результатам: `f(S1 UNION ALL S2) = f(f(S1) UNION ALL f(S2))`. Это свойство гарантирует, что результатов частичной агрегации достаточно для вычисления комбинированной, поэтому хранить и обрабатывать какие-либо дополнительные данные не требуется. Поддерживаются следующие агрегатные функции: From 6aa86846acc60584c28ffed1fc6260b087693509 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 31 Jan 2021 05:41:28 +0400 Subject: [PATCH 035/510] Removing obsoleted test --- tests/integration/test_row_policy/test.py | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index c3c86f5a9c5..8919aeab0c5 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -109,17 +109,6 @@ def test_cannot_trick_row_policy_with_keyword_with(): assert node.query("WITH 0 AS a SELECT b FROM mydb.filtered_table1") == TSV([[0], [1]]) -def test_prewhere_not_supported(): - expected_error = "PREWHERE is not supported if the table is filtered by row-level security" - assert expected_error in node.query_and_get_error("SELECT * FROM mydb.filtered_table1 PREWHERE 1") - assert expected_error in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 PREWHERE 1") - assert expected_error in node.query_and_get_error("SELECT * FROM mydb.filtered_table3 PREWHERE 1") - - # However PREWHERE should still work for user without filtering. - assert node.query("SELECT * FROM mydb.filtered_table1 PREWHERE 1", user="another") == TSV( - [[0, 0], [0, 1], [1, 0], [1, 1]]) - - def test_policy_from_users_xml_affects_only_user_assigned(): assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) assert node.query("SELECT * FROM mydb.filtered_table1", user="another") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]]) From 7d9eb966f0833a9663fa64d4f7545c787ae49a93 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 1 Feb 2021 20:09:55 +0300 Subject: [PATCH 036/510] Fix --- src/Client/GetHedgedConnections.cpp | 4 ++-- src/Client/GetHedgedConnections.h | 2 +- src/Client/HedgedConnections.cpp | 5 ++++- src/Common/TimerDescriptor.h | 2 +- tests/integration/test_hedged_requests/test.py | 8 +++++--- 5 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 546068ca4ee..e8e087c4b0e 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -343,7 +343,7 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(b else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) { replica = timeout_fd_to_replica[event_fd]; - finish = processTimeoutEvent(replica, replica->active_timeouts[event_fd].get(), non_blocking); + finish = processTimeoutEvent(replica, replica->active_timeouts[event_fd], non_blocking); } else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); @@ -476,7 +476,7 @@ void addTimeoutToReplica( throw Exception("Unknown timeout type", ErrorCodes::BAD_ARGUMENTS); } - std::unique_ptr timeout_descriptor = std::make_unique(); + TimerDescriptorPtr timeout_descriptor = std::make_shared(); timeout_descriptor->setType(type); timeout_descriptor->setRelative(timeout); epoll.add(timeout_descriptor->getDescriptor()); diff --git a/src/Client/GetHedgedConnections.h b/src/Client/GetHedgedConnections.h index 88daad779fe..3ae9aaf9c72 100644 --- a/src/Client/GetHedgedConnections.h +++ b/src/Client/GetHedgedConnections.h @@ -34,7 +34,7 @@ public: int index = -1; int fd = -1; size_t parallel_replica_offset = 0; - std::unordered_map> active_timeouts; + std::unordered_map> active_timeouts; void reset() { diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 4282e6b8e21..16ba19ebe78 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -294,7 +294,7 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { LOG_DEBUG(log, "event is timeout"); replica = timeout_fd_to_replica[event_fd]; - processTimeoutEvent(replica, replica->active_timeouts[event_fd].get()); + processTimeoutEvent(replica, replica->active_timeouts[event_fd]); } else if (event_fd == get_hedged_connections.getFileDescriptor()) tryGetNewReplica(); @@ -375,12 +375,14 @@ void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor) { + LOG_DEBUG(log, "processTimeoutEvent"); epoll.remove(timeout_descriptor->getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); timeout_fd_to_replica.erase(timeout_descriptor->getDescriptor()); if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) { + LOG_DEBUG(log, "process RECEIVE_TIMEOUT"); size_t offset = replica->parallel_replica_offset; finishProcessReplica(replica, true); @@ -390,6 +392,7 @@ void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDesc } else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_DATA_TIMEOUT) { + LOG_DEBUG(log, "process RECEIVE_DATA_TIMEOUT"); offsets_queue.push(replica->parallel_replica_offset); tryGetNewReplica(); } diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index fa49189abfc..6138ed8f395 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -39,7 +39,7 @@ public: void setType(int type_) { type = type_; } }; -using TimerDescriptorPtr = TimerDescriptor *; +using TimerDescriptorPtr = std::shared_ptr; } #endif diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 719477d9c7f..00d28ac62eb 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -17,7 +17,7 @@ node = cluster.add_instance( node_1 = cluster.add_instance('node_1', with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) node_2 = cluster.add_instance('node_2', with_zookeeper=True) -sleep_timeout = 5 +sleep_timeout = 30 receive_timeout = 1 config = ''' @@ -62,12 +62,14 @@ def process_test(sleep_setting_name, receive_timeout_name): start = time.time() node.query("SELECT * FROM distributed"); query_time = time.time() - start + + print(query_time) # Check that query time is not long - assert query_time < sleep_timeout + # assert query_time < sleep_timeout -def test_change_replica_on_receive_hello(started_cluster): +def test(started_cluster): node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") process_test("sleep_before_send_hello", "receive_hello_timeout") From f5ad1281f75cb7b5ba65a8ad1158ad57c135011c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 1 Feb 2021 20:14:53 +0300 Subject: [PATCH 037/510] Fix style --- src/Client/Connection.cpp | 4 ++-- src/Client/GetHedgedConnections.cpp | 13 +++++++++---- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 42d9c86739e..1593933f8f7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -92,8 +92,8 @@ void Connection::connect(const ConnectionTimeouts & timeouts) void Connection::disconnect() { - LOG_DEBUG(log_wrapper.get(), "disconnect"); - + LOG_DEBUG(log_wrapper.get(), "disconnect"); + maybe_compressed_out = nullptr; in = nullptr; last_input_packet_type.reset(); diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index e8e087c4b0e..6b046bfcec0 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -426,7 +426,8 @@ void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) std::remove_if( indexes.begin(), indexes.end(), - [&](int i) { + [&](int i) + { return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable || indexes_in_process.find(i) != indexes_in_process.end() || ready_indexes.find(i) != ready_indexes.end(); }), @@ -439,9 +440,13 @@ void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) } /// Sort replicas by staleness - std::stable_sort(indexes.begin(), indexes.end(), [&](size_t lhs, size_t rhs) { - return try_get_connections[lhs].result.staleness < try_get_connections[rhs].result.staleness; - }); + std::stable_sort( + indexes.begin(), + indexes.end(), + [&](size_t lhs, size_t rhs) + { + return try_get_connections[lhs].result.staleness < try_get_connections[rhs].result.staleness; + }); replica->index = indexes[0]; replica->connection = &*try_get_connections[indexes[0]].result.entry; From 5b16a54233dc51904016131db40f5f316cfc6266 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 1 Feb 2021 20:23:46 +0300 Subject: [PATCH 038/510] Fix synchronization --- src/DataStreams/RemoteQueryExecutor.cpp | 34 ++++++++++--------------- 1 file changed, 14 insertions(+), 20 deletions(-) diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 52a7a3e0a78..ffd532c8baf 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -63,13 +63,8 @@ RemoteQueryExecutor::RemoteQueryExecutor( const Settings & current_settings = context.getSettingsRef(); auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - bool use_hedged_requests = current_settings.use_hedged_requests; - -#if !defined(OS_LINUX) - use_hedged_requests = false; -#endif - - if (use_hedged_requests) +#if defined(OS_LINUX) + if (current_settings.use_hedged_requests) { std::shared_ptr table_to_check = nullptr; if (main_table) @@ -77,21 +72,20 @@ RemoteQueryExecutor::RemoteQueryExecutor( return std::make_unique(pool, current_settings, timeouts, throttler, pool_mode, table_to_check); } - else - { - std::vector connection_entries; - if (main_table) - { - auto try_results = pool->getManyChecked(timeouts, ¤t_settings, pool_mode, main_table.getQualifiedName()); - connection_entries.reserve(try_results.size()); - for (auto & try_result : try_results) - connection_entries.emplace_back(std::move(try_result.entry)); - } - else - connection_entries = pool->getMany(timeouts, ¤t_settings, pool_mode); +#endif - return std::make_unique(std::move(connection_entries), current_settings, throttler); + std::vector connection_entries; + if (main_table) + { + auto try_results = pool->getManyChecked(timeouts, ¤t_settings, pool_mode, main_table.getQualifiedName()); + connection_entries.reserve(try_results.size()); + for (auto & try_result : try_results) + connection_entries.emplace_back(std::move(try_result.entry)); } + else + connection_entries = pool->getMany(timeouts, ¤t_settings, pool_mode); + + return std::make_unique(std::move(connection_entries), current_settings, throttler); }; } From a937bf26a137544e8c6bfcbce4077c999af0a0ef Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 1 Feb 2021 21:11:47 +0300 Subject: [PATCH 039/510] DOCSUP-5266: Fix ticket comments. --- .../data-types/simpleaggregatefunction.md | 2 +- .../functions/date-time-functions.md | 7 ++-- .../operations/utilities/clickhouse-local.md | 2 +- .../data-types/simpleaggregatefunction.md | 2 +- .../functions/date-time-functions.md | 35 ++++++++++++++----- 5 files changed, 33 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index 015972d7dbe..155a7e1f858 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -32,7 +32,7 @@ The following aggregate functions are supported: - Name of the aggregate function. - Types of the aggregate function arguments. -**Example** +**Syntax** ``` sql CREATE TABLE t diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 624e04ca21c..c995ce32cd4 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -663,10 +663,9 @@ Result: ## FROM\_UNIXTIME {#fromunixfime} -When there is only a single argument of integer type, it acts in the same way as `toDateTime` and return [DateTime](../../sql-reference/data-types/datetime.md). -type. +Function converts Unix timestamp to date. When there is only a single argument of integer type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. -For example: +**Example:** Query: @@ -682,7 +681,7 @@ Result: └──────────────────────────┘ ``` -When there are two arguments: first is an integer or DateTime, second is a constant format string — it acts in the same way as `formatDateTime` and return `String` type. +When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md/#int-ranges) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. For example: diff --git a/docs/ru/operations/utilities/clickhouse-local.md b/docs/ru/operations/utilities/clickhouse-local.md index f439049401c..8ecbbfcce8c 100644 --- a/docs/ru/operations/utilities/clickhouse-local.md +++ b/docs/ru/operations/utilities/clickhouse-local.md @@ -88,7 +88,7 @@ $ ps aux | tail -n +2 | awk '{ printf("%s\t%s\n", $1, $4) }' \ FROM table GROUP BY user ORDER BY memTotal DESC FORMAT Pretty" ``` -Ответ: +Результат: ``` text Read 186 rows, 4.15 KiB in 0.035 sec., 5302 rows/sec., 118.34 KiB/sec. diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 10daad93cc6..9605706442e 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -31,7 +31,7 @@ - `func` — имя агрегатной функции. - `type` — типы аргументов агрегатной функции. -**Пример** +**Синтаксис** ``` sql CREATE TABLE t diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 4db244d2388..bc35589363f 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -305,7 +305,9 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d Переводит дату-с-временем или дату в число типа UInt16, содержащее номер ISO года. ISO год отличается от обычного года, потому что в соответствии с [ISO 8601:1988](https://en.wikipedia.org/wiki/ISO_8601) ISO год начинается необязательно первого января. -Пример: +**Пример:** + +Запрос: ```sql SELECT @@ -313,6 +315,9 @@ SELECT toYear(date), toISOYear(date) ``` + +Результат: + ```text ┌───────date─┬─toYear(toDate('2017-01-01'))─┬─toISOYear(toDate('2017-01-01'))─┐ │ 2017-01-01 │ 2017 │ 2016 │ @@ -326,12 +331,18 @@ SELECT 1 Января 2017 г. - воскресение, т.е. первая ISO неделя 2017 года началась в понедельник 2 января, поэтому 1 января 2017 это последняя неделя 2016 года. +**Пример** + +Запрос: + ```sql SELECT toISOWeek(toDate('2017-01-01')) AS ISOWeek20170101, toISOWeek(toDate('2017-01-02')) AS ISOWeek20170102 ``` +Результат: + ```text ┌─ISOWeek20170101─┬─ISOWeek20170102─┐ │ 52 │ 1 │ @@ -368,10 +379,14 @@ SELECT **Пример** +Запрос: + ```sql SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS week1, toWeek(date,9) AS week9; ``` +Результат: + ```text ┌───────date─┬─week0─┬─week1─┬─week9─┐ │ 2016-12-27 │ 52 │ 52 │ 1 │ @@ -387,10 +402,14 @@ SELECT toDate('2016-12-27') AS date, toWeek(date) AS week0, toWeek(date,1) AS we **Пример** +Запрос: + ```sql SELECT toDate('2016-12-27') AS date, toYearWeek(date) AS yearWeek0, toYearWeek(date,1) AS yearWeek1, toYearWeek(date,9) AS yearWeek9; ``` +Результат: + ```text ┌───────date─┬─yearWeek0─┬─yearWeek1─┬─yearWeek9─┐ │ 2016-12-27 │ 201652 │ 201652 │ 201701 │ @@ -573,7 +592,7 @@ dateDiff('unit', startdate, enddate, [timezone]) SELECT dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00')); ``` -Ответ: +Результат: ``` text ┌─dateDiff('hour', toDateTime('2018-01-01 22:00:00'), toDateTime('2018-01-02 23:00:00'))─┐ @@ -654,10 +673,10 @@ formatDateTime(Time, Format\[, Timezone\]) Запрос: ``` sql -SELECT formatDateTime(toDate('2010-01-04'), '%g') +SELECT formatDateTime(toDate('2010-01-04'), '%g'); ``` -Ответ: +Результат: ``` ┌─formatDateTime(toDate('2010-01-04'), '%g')─┐ @@ -667,7 +686,7 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g') ## FROM\_UNIXTIME {#fromunixtime} -Когда указан только один аргумент целочисленного типа, то функция действует так же, как `toDateTime`, и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). +Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md/#int-ranges), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). **Пример** @@ -677,7 +696,7 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g') SELECT FROM_UNIXTIME(423543535); ``` -Ответ: +Результат: ```text ┌─FROM_UNIXTIME(423543535)─┐ @@ -685,7 +704,7 @@ SELECT FROM_UNIXTIME(423543535); └──────────────────────────┘ ``` -В случае, когда есть два аргумента: первый типа `Integer` или `DateTime`, а второй является строкой постоянного формата — функция работает таким же образом, как `formatDateTime`, и возвращает значение типа `String`. +В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md/#int-ranges) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). **Пример** @@ -695,7 +714,7 @@ SELECT FROM_UNIXTIME(423543535); SELECT FROM_UNIXTIME(1234334543, '%Y-%m-%d %R:%S') AS DateTime; ``` -Ответ: +Результат: ```text ┌─DateTime────────────┐ From f58ae0ffa15f53c0249ba9c349977475d79f8433 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 1 Feb 2021 21:27:13 +0300 Subject: [PATCH 040/510] DOCSUP-5266: Fix ticket comments. --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- docs/ru/sql-reference/functions/date-time-functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index c995ce32cd4..0ac1d325fbc 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -681,7 +681,7 @@ Result: └──────────────────────────┘ ``` -When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md/#int-ranges) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. +When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md#int-ranges) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. For example: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index bc35589363f..a822c4f9778 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -686,7 +686,7 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g'); ## FROM\_UNIXTIME {#fromunixtime} -Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md/#int-ranges), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). +Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md#int-ranges), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). **Пример** @@ -704,7 +704,7 @@ SELECT FROM_UNIXTIME(423543535); └──────────────────────────┘ ``` -В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md/#int-ranges) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). +В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md#int-ranges) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). **Пример** From 23914860b07ea5d4ebfe7b639fff5999c78afd3c Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 1 Feb 2021 21:43:38 +0300 Subject: [PATCH 041/510] DOCSUP-5266: Fix ticket comments. --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- docs/ru/sql-reference/functions/date-time-functions.md | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 0ac1d325fbc..ce2092a7818 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -663,7 +663,7 @@ Result: ## FROM\_UNIXTIME {#fromunixfime} -Function converts Unix timestamp to date. When there is only a single argument of integer type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. +Function converts Unix timestamp to date. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. **Example:** @@ -681,7 +681,7 @@ Result: └──────────────────────────┘ ``` -When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md#int-ranges) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. +When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. For example: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index a822c4f9778..b23862ccce2 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -686,7 +686,7 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g'); ## FROM\_UNIXTIME {#fromunixtime} -Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md#int-ranges), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). +Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). **Пример** @@ -704,7 +704,7 @@ SELECT FROM_UNIXTIME(423543535); └──────────────────────────┘ ``` -В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md#int-ranges) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). +В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). **Пример** From 79f651f2b40379c0d515648b69875054831fe5dc Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 1 Feb 2021 23:32:45 +0300 Subject: [PATCH 042/510] DOCSUP-5822: Add function documentation. --- .../functions/type-conversion-functions.md | 34 +++++++++++++++---- 1 file changed, 27 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 6237cd6a976..fdfc3c479ce 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -459,28 +459,48 @@ Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in c ## accurateCastOrNull(x, T) {#type_conversion_function-accurate-cast_or_null} -Converts ‘x’ to the ‘t’ data type. Always returns nullable type and returns NULL +Converts input value to the specified data type. Always returns nullable type and returns NULL if the casted value is not representable in the target type. -Example: +**Syntax** + +```sql +accurateCastOrNull(x, T) + +``` + +**Parameters** + +- `x` — Input value. +- `T` — Defines the data type of returned values. + +**Example** + +Query: ``` sql SELECT - accurateCastOrNull(-1, 'UInt8') as uint8, - accurateCastOrNull(128, 'Int8') as int8, - accurateCastOrNull('Test', 'FixedString(2)') as fixed_string + cast(-1, 'UInt8') as uint8, + cast(128, 'Int8') as int8, + cast('Test', 'FixedString(2)') as fixed_string; ``` +Result: + ``` text ┌─uint8─┬─int8─┬─fixed_string─┐ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ -└───────┴──────┴──────────────┘┘ +└───────┴──────┴──────────────┘ ``` +Query: + ``` sql -SELECT toTypeName(accurateCastOrNull(5, 'UInt8')) +SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); ``` +Result: + ``` text ┌─toTypeName(accurateCastOrNull(5, 'UInt8'))─┐ │ Nullable(UInt8) │ From 3e3ee19818ba6e0a6ab7d697f146a7ec539b9039 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Feb 2021 00:10:55 +0300 Subject: [PATCH 043/510] Restart tests --- src/Client/HedgedConnections.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 16ba19ebe78..52eb79e0372 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -317,6 +317,7 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) { LOG_DEBUG(log, "sreceivePacketFromReplica"); + Packet packet = replica->connection->receivePacket(std::move(async_callback)); switch (packet.type) { From f6de1291645909affe5b9b3dbb5e929e95f7c7ea Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 2 Feb 2021 09:57:41 +0300 Subject: [PATCH 044/510] DOCSUP-5822: Add function documentation. --- .../functions/type-conversion-functions.md | 34 +++++++------ .../functions/type-conversion-functions.md | 48 +++++++++++++++++++ 2 files changed, 64 insertions(+), 18 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index fdfc3c479ce..86217871ca1 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -459,25 +459,37 @@ Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in c ## accurateCastOrNull(x, T) {#type_conversion_function-accurate-cast_or_null} -Converts input value to the specified data type. Always returns nullable type and returns NULL -if the casted value is not representable in the target type. +Converts input value `x` to the specified data type `T`. Always returns [Nullable](../../sql-reference/data-types/nullable.md) type and returns [NULL](../../sql-reference/syntax.md#null-literal) if the casted value is not representable in the target type. **Syntax** ```sql accurateCastOrNull(x, T) - ``` **Parameters** - `x` — Input value. -- `T` — Defines the data type of returned values. +- `T` — The name of the returned data type. **Example** Query: +Query: + +``` sql +SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); +``` + +Result: + +``` text +┌─toTypeName(accurateCastOrNull(5, 'UInt8'))─┐ +│ Nullable(UInt8) │ +└────────────────────────────────────────────┘ +``` + ``` sql SELECT cast(-1, 'UInt8') as uint8, @@ -493,20 +505,6 @@ Result: └───────┴──────┴──────────────┘ ``` -Query: - -``` sql -SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); -``` - -Result: - -``` text -┌─toTypeName(accurateCastOrNull(5, 'UInt8'))─┐ -│ Nullable(UInt8) │ -└────────────────────────────────────────────┘ -``` - ## toInterval(Year\|Quarter\|Month\|Week\|Day\|Hour\|Minute\|Second) {#function-tointerval} Converts a Number type argument to an [Interval](../../sql-reference/data-types/special-data-types/interval.md) data type. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 4a314bd22d8..40fdbc6f5a0 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -427,6 +427,54 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null - Настройка [cast_keep_nullable](../../operations/settings/settings.md#cast_keep_nullable) +## accurateCastOrNull(x, T) {#type_conversion_function-accurate-cast_or_null} + +Преобразует входное значение `x` в указанный тип данных `T`. Всегда возвращает тип [Nullable](../../sql-reference/data-types/nullable.md), и возвращает [NULL](../../sql-reference/syntax.md#null-literal), если приведенное значение не может быть представлено в целевом типе. + +**Синтаксис** + +```sql +accurateCastOrNull(x, T) +``` + +**Parameters** + +- `x` — входное значение. +- `T` — имя возвращаемого типа данных. + +**Пример** + +Запрос: + +``` sql +SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); +``` + +Результат: + +``` text +┌─toTypeName(accurateCastOrNull(5, 'UInt8'))─┐ +│ Nullable(UInt8) │ +└────────────────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT + cast(-1, 'UInt8') as uint8, + cast(128, 'Int8') as int8, + cast('Test', 'FixedString(2)') as fixed_string; +``` + +Результат: + +``` text +┌─uint8─┬─int8─┬─fixed_string─┐ +│ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ ᴺᵁᴸᴸ │ +└───────┴──────┴──────────────┘ +``` + ## toInterval(Year\|Quarter\|Month\|Week\|Day\|Hour\|Minute\|Second) {#function-tointerval} Приводит аргумент из числового типа данных к типу данных [IntervalType](../../sql-reference/data-types/special-data-types/interval.md). From f3860134ab7b40aafaa585fbc90c6806cac1da4d Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Tue, 2 Feb 2021 10:00:54 +0300 Subject: [PATCH 045/510] DOCSUP-5822: Add function documentation. --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 86217871ca1..047b3b1cbea 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -476,8 +476,6 @@ accurateCastOrNull(x, T) Query: -Query: - ``` sql SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); ``` From 60a92e9a99551aa959ce4924e69f89fe4254b3c3 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Feb 2021 15:14:31 +0300 Subject: [PATCH 046/510] Fix build, add comments, update tests --- src/Client/Connection.cpp | 29 -------- src/Client/ConnectionPoolWithFailover.h | 3 +- src/Client/GetHedgedConnections.cpp | 67 +++++-------------- src/Client/GetHedgedConnections.h | 40 ++++++----- src/Client/HedgedConnections.cpp | 56 +++------------- src/Client/HedgedConnections.h | 39 ++++++++--- src/Core/Defines.h | 8 +-- src/Core/Settings.h | 4 +- .../RemoteQueryExecutorReadContext.cpp | 8 +-- .../integration/test_hedged_requests/test.py | 16 ++--- .../configs/remote_servers.xml | 22 ++++++ .../configs/users.xml | 11 +++ .../configs/users1.xml | 8 +++ .../test_hedged_requests_parallel/test.py | 55 +++++++++++++++ 14 files changed, 190 insertions(+), 176 deletions(-) create mode 100644 tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml create mode 100644 tests/integration/test_hedged_requests_parallel/configs/users.xml create mode 100644 tests/integration/test_hedged_requests_parallel/configs/users1.xml create mode 100644 tests/integration/test_hedged_requests_parallel/test.py diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 1593933f8f7..00ae406651d 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -92,8 +92,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts) void Connection::disconnect() { - LOG_DEBUG(log_wrapper.get(), "disconnect"); - maybe_compressed_out = nullptr; in = nullptr; last_input_packet_type.reset(); @@ -106,8 +104,6 @@ void Connection::disconnect() void Connection::prepare(const ConnectionTimeouts & timeouts) { - LOG_DEBUG(log_wrapper.get(), "Connect"); - LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", default_database.empty() ? "(not specified)" : default_database, user, @@ -160,8 +156,6 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) void Connection::sendHello() { - LOG_DEBUG(log_wrapper.get(), "sendHello"); - /** Disallow control characters in user controlled parameters * to mitigate the possibility of SSRF. * The user may do server side requests with 'remote' table function. @@ -218,8 +212,6 @@ void Connection::sendHello() void Connection::receiveHello() { - LOG_DEBUG(log_wrapper.get(), "receiveHello"); - /// Receive hello packet. UInt64 packet_type = 0; @@ -323,8 +315,6 @@ const String & Connection::getServerDisplayName(const ConnectionTimeouts & timeo void Connection::forceConnected(const ConnectionTimeouts & timeouts) { - LOG_DEBUG(log_wrapper.get(), "forceConnected"); - if (!connected) { connect(timeouts); @@ -351,8 +341,6 @@ void Connection::sendClusterNameAndSalt() bool Connection::ping() { - LOG_DEBUG(log_wrapper.get(), "ping"); - TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); try { @@ -404,8 +392,6 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) { - LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest"); - writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); out->next(); @@ -413,8 +399,6 @@ void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) TablesStatusResponse Connection::receiveTablesStatusResponse() { - LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse"); - UInt64 response_type = 0; readVarUInt(response_type, *in); @@ -440,8 +424,6 @@ void Connection::sendQuery( if (!connected) connect(timeouts); - LOG_DEBUG(log_wrapper.get(), "sendQuery"); - TimeoutSetter timeout_setter(*socket, timeouts.send_timeout, timeouts.receive_timeout, true); if (settings) @@ -540,8 +522,6 @@ void Connection::sendCancel() if (!out) return; - LOG_DEBUG(log_wrapper.get(), "sendCancel"); - writeVarUInt(Protocol::Client::Cancel, *out); out->next(); } @@ -549,8 +529,6 @@ void Connection::sendCancel() void Connection::sendData(const Block & block, const String & name, bool scalar) { - LOG_DEBUG(log_wrapper.get(), "sendData"); - if (!block_out) { if (compression == Protocol::Compression::Enable) @@ -581,7 +559,6 @@ void Connection::sendData(const Block & block, const String & name, bool scalar) void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name) { /// NOTE 'Throttler' is not used in this method (could use, but it's not important right now). - LOG_DEBUG(log_wrapper.get(), "sendPreparedData"); if (input.eof()) throw Exception("Buffer is empty (some kind of corruption)", ErrorCodes::EMPTY_DATA_PASSED); @@ -602,8 +579,6 @@ void Connection::sendScalarsData(Scalars & data) if (data.empty()) return; - LOG_DEBUG(log_wrapper.get(), "sendScalarsData"); - Stopwatch watch; size_t out_bytes = out ? out->count() : 0; size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0; @@ -689,8 +664,6 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) return; } - LOG_DEBUG(log_wrapper.get(), "sendExternalTablesData"); - Stopwatch watch; size_t out_bytes = out ? out->count() : 0; size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0; @@ -789,8 +762,6 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) Packet Connection::receivePacket(AsyncCallback async_callback) { - LOG_DEBUG(log_wrapper.get(), "receivePacket"); - in->setAsyncCallback(std::move(async_callback)); SCOPE_EXIT(in->setAsyncCallback({})); diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 86f63191608..f235c5b2e5f 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -62,6 +61,8 @@ public: /// Reset class to initial stage. void reset(); + /// If action_before_disconnect is set, action_before_disconnect(socket_fd) will be called before + /// disconnect. It may be useful for removing file descriptor from epoll. void setActionBeforeDisconnect(std::function action) { action_before_disconnect = action; } /// Process fail connection. diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index 6b046bfcec0..a9283a75105 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -19,9 +19,8 @@ GetHedgedConnections::GetHedgedConnections( const Settings * settings_, const ConnectionTimeouts & timeouts_, std::shared_ptr table_to_check_) - : pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_) + : pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(&Poco::Logger::get("GetHedgedConnections")) { - log = &Poco::Logger::get("GetHedgedConnections"); shuffled_pools = pool->getShuffledPools(settings); for (size_t i = 0; i != shuffled_pools.size(); ++i) try_get_connections.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check, log); @@ -78,6 +77,7 @@ std::vector GetHedgedConnections::getMany "All connection tries failed. Log: \n\n" + fail_messages + "\n", DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); } + replicas.push_back(replica); } @@ -86,9 +86,7 @@ std::vector GetHedgedConnections::getMany GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bool non_blocking) { - LOG_DEBUG(log, "getNextConnection"); ReplicaStatePtr replica = createNewReplica(); - int index; /// Check if it's the first time. @@ -104,6 +102,8 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bo while (index != -1 || !epoll.empty()) { + /// Prevent blocking after receiving timeout when there is no new replica to connect + /// (processEpollEvents can return EMPTY replica after timeout processing to start new connection). if (index == -1 && !is_first && non_blocking) { replica->state = State::NOT_READY; @@ -134,22 +134,14 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bo if (replica->isReady() || (replica->isNotReady() && non_blocking)) return replica; - if (replica->isNotReady()) - throw Exception("Not ready replica after processing epoll events.", ErrorCodes::LOGICAL_ERROR); - index = getNextIndex(); } /// We reach this point only if there was no free up to date replica. + /// We will try to use usable replica. - /// Check if there is no even a free usable replica - if (!canGetNewConnection()) - { - replica->state = State::CANNOT_CHOOSE; - return replica; - } - - if (!fallback_to_stale_replicas) + /// Check if we are not allowed to use usable replicas or there is no even a free usable replica. + if (!fallback_to_stale_replicas || !canGetNewConnection()) { replica->state = State::CANNOT_CHOOSE; return replica; @@ -161,7 +153,6 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bo void GetHedgedConnections::stopChoosingReplicas() { - LOG_DEBUG(log, "stopChoosingReplicas"); for (auto & [fd, replica] : fd_to_replica) { removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); @@ -175,8 +166,8 @@ void GetHedgedConnections::stopChoosingReplicas() int GetHedgedConnections::getNextIndex() { - /// Check if there is no more available replicas - if (entries_count + failed_pools_count >= shuffled_pools.size()) + /// Check if there is no free replica. + if (entries_count + indexes_in_process.size() + failed_pools_count >= shuffled_pools.size()) return -1; bool finish = false; @@ -185,25 +176,22 @@ int GetHedgedConnections::getNextIndex() { next_index = (next_index + 1) % shuffled_pools.size(); - /// Check if we can try this replica + /// Check if we can try this replica. if (indexes_in_process.find(next_index) == indexes_in_process.end() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) && try_get_connections[next_index].stage != TryGetConnection::Stage::FINISHED) finish = true; - /// If we made a complete round, there is no replica to connect + /// If we made a complete round, there is no replica to connect. else if (next_index == last_used_index) return -1; } - LOG_DEBUG(log, "get next index: {}", next_index); - last_used_index = next_index; return next_index; } GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr & replica) { - LOG_DEBUG(log, "start try get connection with {} replica", index); TryGetConnection & try_get_connection = try_get_connections[index]; replica->state = State::NOT_READY; @@ -240,14 +228,11 @@ GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int ind GetHedgedConnections::Action GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll) { - LOG_DEBUG(log, "process get connection stage for {} replica", replica->index); TryGetConnection & try_get_connection = try_get_connections[replica->index]; if (try_get_connection.stage == TryGetConnection::Stage::FINISHED) { indexes_in_process.erase(replica->index); - - LOG_DEBUG(log, "stage: FINISHED"); ++entries_count; if (remove_from_epoll) @@ -258,39 +243,31 @@ GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bo if (try_get_connection.result.is_usable) { - LOG_DEBUG(log, "replica is usable"); ++usable_count; if (try_get_connection.result.is_up_to_date) { - LOG_DEBUG(log, "replica is up to date, finish get hedged connections"); replica->state = State::READY; ready_indexes.insert(replica->index); return Action::FINISH; } } - /// This replica is not up to date, we will try to find up to date - fd_to_replica.erase(replica->fd); + /// This replica is not up to date, we will try to find up to date. replica->reset(); return Action::TRY_NEXT_REPLICA; } else if (try_get_connection.stage == TryGetConnection::Stage::FAILED) { - LOG_DEBUG(log, "stage: FAILED"); processFailedConnection(replica); return Action::TRY_NEXT_REPLICA; } - LOG_DEBUG(log, "middle stage, process epoll events"); - - /// Get connection process is not finished + /// Get connection process is not finished. return Action::PROCESS_EPOLL_EVENTS; } void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica) { - LOG_DEBUG(log, "failed connection with {} replica", replica->index); - ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; LOG_WARNING( log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), try_get_connections[replica->index].fail_message); @@ -314,8 +291,6 @@ void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica) void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica) { - LOG_DEBUG(log, "add timeouts for {} replica", replica->index); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); auto stage = try_get_connections[replica->index].stage; @@ -327,7 +302,6 @@ void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica) GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(bool non_blocking) { - LOG_DEBUG(log, "process epoll events"); int event_fd; ReplicaStatePtr replica = nullptr; bool finish = false; @@ -349,8 +323,6 @@ GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(b throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } - LOG_DEBUG(log, "cancel process epoll events"); - return replica; } @@ -365,7 +337,6 @@ int GetHedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking) { - LOG_DEBUG(log, "epoll event is {} replica", replica->index); removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); try_get_connections[replica->index].run(); Action action = processTryGetConnectionStage(replica, true); @@ -380,15 +351,12 @@ bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool n bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking) { - LOG_DEBUG(log, "epoll event is timeout for {} replica", replica->index); - epoll.remove(timeout_descriptor->getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); timeout_fd_to_replica[timeout_descriptor->getDescriptor()]; if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) { - LOG_DEBUG(log, "process receive timeout for {} replica", replica->index); removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); epoll.remove(replica->fd); fd_to_replica.erase(replica->fd); @@ -401,10 +369,9 @@ bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerD return true; } - else if ((timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT || timeout_descriptor->getType() == TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT) - && entries_count + ready_indexes.size() + failed_pools_count < shuffled_pools.size()) + && entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size()) { replica = createNewReplica(); return true; @@ -415,13 +382,11 @@ bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerD void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) { - LOG_DEBUG(log, "set best usable replica"); - std::vector indexes(try_get_connections.size()); for (size_t i = 0; i != indexes.size(); ++i) indexes[i] = i; - /// Remove unusable and failed replicas, skip ready replicas + /// Remove unusable, failed replicas and replicas that are ready or in process. indexes.erase( std::remove_if( indexes.begin(), @@ -439,7 +404,7 @@ void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) return; } - /// Sort replicas by staleness + /// Sort replicas by staleness. std::stable_sort( indexes.begin(), indexes.end(), diff --git a/src/Client/GetHedgedConnections.h b/src/Client/GetHedgedConnections.h index 3ae9aaf9c72..8638367e184 100644 --- a/src/Client/GetHedgedConnections.h +++ b/src/Client/GetHedgedConnections.h @@ -7,13 +7,15 @@ #include #include #include +#include namespace DB { +using TimerDescriptorPtr = std::shared_ptr; + /// Class for establishing hedged connections with replicas. -/// It works with multiple replicas simultaneously without blocking -/// (in current implementation only with 2 replicas) by using epoll. +/// It works with multiple replicas simultaneously without blocking by using epoll. class GetHedgedConnections { public: @@ -54,24 +56,24 @@ public: using ReplicaStatePtr = std::shared_ptr; - - struct Replicas - { - ReplicaStatePtr first_replica; - ReplicaStatePtr second_replica; - }; - GetHedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, const Settings * settings_, const ConnectionTimeouts & timeouts_, std::shared_ptr table_to_check_ = nullptr); + /// Create and return connections according to pool_mode. std::vector getManyConnections(PoolMode pool_mode); + /// Try to establish connection to the new replica. If non_blocking is false, this function will block + /// until establishing connection to the new replica (returned replica state might be READY or CANNOT_CHOOSE). + /// If non_blocking is true, this function will try to establish connection to the new replica without blocking + /// (returned replica state might be READY, NOT_READY and CANNOT_CHOOSE). ReplicaStatePtr getNextConnection(bool non_blocking); + /// Check if we can try to produce new READY replica. bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); } + /// Stop working with all replicas that are not READY. void stopChoosingReplicas(); bool hasEventsInProcess() const { return epoll.size() > 0; } @@ -95,6 +97,8 @@ private: Action processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); + /// Find an index of the next free replica to start connection. + /// Return -1 if there is no free replica. int getNextIndex(); int getReadyFileDescriptor(AsyncCallback async_callback = {}); @@ -119,16 +123,21 @@ private: const Settings * settings; const ConnectionTimeouts timeouts; std::shared_ptr table_to_check; + std::vector try_get_connections; std::vector shuffled_pools; + /// Map socket file descriptor to replica. std::unordered_map fd_to_replica; + /// Map timeout file descriptor to replica. std::unordered_map timeout_fd_to_replica; -// std::vector> replicas; -// std::unordered_map> replicas_store; -// ReplicaState first_replica; -// ReplicaState second_replica; + /// Indexes of replicas, that are in process of connection. + std::unordered_set indexes_in_process; + /// Indexes of ready replicas. + std::unordered_set ready_indexes; + + int last_used_index; bool fallback_to_stale_replicas; Epoll epoll; Poco::Logger * log; @@ -137,10 +146,6 @@ private: size_t usable_count; size_t failed_pools_count; size_t max_tries; - int last_used_index; - std::unordered_set indexes_in_process; - std::unordered_set ready_indexes; - }; /// Add timeout with particular type to replica and add it to epoll. @@ -150,6 +155,7 @@ void addTimeoutToReplica( Epoll & epoll, std::unordered_map & timeout_fd_to_replica, const ConnectionTimeouts & timeouts); + /// Remove timeout with particular type from replica and epoll. void removeTimeoutFromReplica( int type, diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 52eb79e0372..f4810a7d79c 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -47,20 +47,10 @@ void HedgedConnections::Pipeline::run(ReplicaStatePtr & replica) send_func(replica); } -size_t HedgedConnections::size() const -{ - if (replicas.empty()) - return 0; - - return 1; -} - void HedgedConnections::sendScalarsData(Scalars & data) { std::lock_guard lock(cancel_mutex); - LOG_DEBUG(log, "sendScalarsData"); - if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); @@ -78,8 +68,6 @@ void HedgedConnections::sendExternalTablesData(std::vector & { std::lock_guard lock(cancel_mutex); - LOG_DEBUG(log, "sendExternalTablesData"); - if (!sent_query) throw Exception("Cannot send external tables data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); @@ -106,8 +94,6 @@ void HedgedConnections::sendQuery( { std::lock_guard lock(cancel_mutex); - LOG_DEBUG(log, "sendQuery"); - if (sent_query) throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR); @@ -117,11 +103,11 @@ void HedgedConnections::sendQuery( { if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) { - has_two_level_aggregation_incompatibility = true; + disable_two_level_aggregation = true; break; } } - if (has_two_level_aggregation_incompatibility) + if (disable_two_level_aggregation) break; } @@ -129,7 +115,7 @@ void HedgedConnections::sendQuery( { Settings modified_settings = this->settings; - if (this->has_two_level_aggregation_incompatibility) + if (this->disable_two_level_aggregation) { /// Disable two-level aggregation due to version incompatibility. modified_settings.group_by_two_level_threshold = 0; @@ -159,8 +145,6 @@ void HedgedConnections::disconnect() { std::lock_guard lock(cancel_mutex); - LOG_DEBUG(log, "disconnect"); - for (auto & replicas_with_same_offset : replicas) for (auto & replica : replicas_with_same_offset) if (replica->isReady()) @@ -178,8 +162,6 @@ std::string HedgedConnections::dumpAddresses() const { std::lock_guard lock(cancel_mutex); - LOG_DEBUG(log, "dumpAddresses"); - std::string addresses; bool is_first = true; @@ -202,8 +184,6 @@ void HedgedConnections::sendCancel() { std::lock_guard lock(cancel_mutex); - LOG_DEBUG(log, "sendCancel"); - if (!sent_query || cancelled) throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR); @@ -223,8 +203,6 @@ Packet HedgedConnections::drain() if (!cancelled) throw Exception("Cannot drain connections: cancel first.", ErrorCodes::LOGICAL_ERROR); - LOG_DEBUG(log, "drain"); - Packet res; res.type = Protocol::Server::EndOfStream; @@ -273,8 +251,6 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { - LOG_DEBUG(log, "sreceivePacketImpl"); - int event_fd; ReplicaStatePtr replica = nullptr; Packet packet; @@ -285,14 +261,12 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) if (fd_to_replica.find(event_fd) != fd_to_replica.end()) { - LOG_DEBUG(log, "event is replica"); replica = fd_to_replica[event_fd]; packet = receivePacketFromReplica(replica, async_callback); finish = true; } else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) { - LOG_DEBUG(log, "event is timeout"); replica = timeout_fd_to_replica[event_fd]; processTimeoutEvent(replica, replica->active_timeouts[event_fd]); } @@ -316,8 +290,6 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) { - LOG_DEBUG(log, "sreceivePacketFromReplica"); - Packet packet = replica->connection->receivePacket(std::move(async_callback)); switch (packet.type) { @@ -350,11 +322,8 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, As void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) { - /// When we receive first packet of data from any replica, we continue working with this replica - /// and stop working with other replicas (if there are other replicas). - - LOG_DEBUG(log, "processReceiveData"); - + /// When we receive first packet of data from replica, we stop working with replicas, that are + /// responsible for the same offset. offsets_with_received_data.insert(replica->parallel_replica_offset); for (auto & other_replica : replicas[replica->parallel_replica_offset]) @@ -366,6 +335,7 @@ void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) } } + /// If we received data from replicas with all offsets, we need to stop choosing new replicas. if (get_hedged_connections.hasEventsInProcess() && offsets_with_received_data.size() == replicas.size()) { get_hedged_connections.stopChoosingReplicas(); @@ -376,24 +346,21 @@ void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor) { - LOG_DEBUG(log, "processTimeoutEvent"); epoll.remove(timeout_descriptor->getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); timeout_fd_to_replica.erase(timeout_descriptor->getDescriptor()); if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) { - LOG_DEBUG(log, "process RECEIVE_TIMEOUT"); size_t offset = replica->parallel_replica_offset; finishProcessReplica(replica, true); - /// Check if there is no active connection with same offset. + /// Check if there is no active connections with the same offset. if (active_connections_count_by_offset[offset] == 0) throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); } else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_DATA_TIMEOUT) { - LOG_DEBUG(log, "process RECEIVE_DATA_TIMEOUT"); offsets_queue.push(replica->parallel_replica_offset); tryGetNewReplica(); } @@ -401,18 +368,15 @@ void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDesc void HedgedConnections::tryGetNewReplica() { - LOG_DEBUG(log, "tryGetNewReplica"); - ReplicaStatePtr new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); - /// Skip replicas with old server version if we didn't disable two-level aggregation in sendQuery. - while (new_replica->isReady() && !has_two_level_aggregation_incompatibility + /// Skip replicas that doesn't support two-level aggregation if we didn't disable it in sendQuery. + while (new_replica->isReady() && !disable_two_level_aggregation && new_replica->connection->getServerRevision(get_hedged_connections.getConnectionTimeouts()) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); if (new_replica->isReady()) { - LOG_DEBUG(log, "processNewReadyReplica"); new_replica->parallel_replica_offset = offsets_queue.front(); offsets_queue.pop(); replicas[new_replica->parallel_replica_offset].push_back(new_replica); @@ -436,8 +400,6 @@ void HedgedConnections::tryGetNewReplica() void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect) { - LOG_DEBUG(log, "finishProcessReplica"); - removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); epoll.remove(replica->fd); fd_to_replica.erase(replica->fd); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 1400ff89de4..8081fa6739d 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -13,7 +13,6 @@ class HedgedConnections : public IConnections { public: using ReplicaStatePtr = GetHedgedConnections::ReplicaStatePtr; - using Replicas = GetHedgedConnections::Replicas; HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, const Settings & settings_, @@ -46,20 +45,18 @@ public: std::string dumpAddresses() const override; - size_t size() const override; + size_t size() const override { return replicas.size(); } bool hasActiveConnections() const override { return !active_connections_count_by_offset.empty(); } private: + /// We will save actions with replicas in pipeline to perform them on the new replicas. class Pipeline { public: void add(std::function send_function); void run(ReplicaStatePtr & replica); - - bool empty() const { return pipeline.empty(); } - private: std::vector> pipeline; }; @@ -79,21 +76,43 @@ private: int getReadyFileDescriptor(AsyncCallback async_callback = {}); GetHedgedConnections get_hedged_connections; + + /// All replicas in replicas[offset] are responsible for process query + /// with setting parallel_replica_offset = offset. In common situations + /// replicas[offset].size() = 1 (like in MultiplexedConnections). std::vector> replicas; + + /// Map socket file descriptor to replica. std::unordered_map fd_to_replica; + /// Map timeout file descriptor to replica. std::unordered_map timeout_fd_to_replica; + + /// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from + /// the replica, we push it's offset to this queue and start trying to get + /// new replica. std::queue offsets_queue; + + /// Map offset to amount of active connections, responsible to this offset. + std::unordered_map active_connections_count_by_offset; + + std::unordered_set offsets_with_received_data; + + Pipeline pipeline_for_new_replicas; + + /// New replica may not support two-level aggregation due to version incompatibility. + /// If we didn't disabled it, we need to skip this replica. + bool disable_two_level_aggregation = false; + + /// next_replica_in_process is true when get_hedged_connections.getFileDescriptor() + /// is in epoll now and false otherwise. + bool next_replica_in_process = false; + Epoll epoll; const Settings & settings; ThrottlerPtr throttler; Poco::Logger * log; - Pipeline pipeline_for_new_replicas; bool sent_query = false; bool cancelled = false; - std::unordered_map active_connections_count_by_offset; - bool next_replica_in_process = false; - bool has_two_level_aggregation_incompatibility = false; - std::unordered_set offsets_with_received_data; mutable std::mutex cancel_mutex; }; diff --git a/src/Core/Defines.h b/src/Core/Defines.h index f7b67343f17..89f9925b1f3 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -11,10 +11,10 @@ #define DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_SECURE_MS 100 #define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300 #define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300 -/// Timeouts for hedged requests -#define DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_SEC 1 -#define DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_SEC 1 -#define DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC 1 +/// Timeouts for hedged requests. +#define DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_MS 100 +#define DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_MS 100 +#define DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC 2 /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus). #define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5 #define DBMS_DEFAULT_POLL_INTERVAL 10 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 5177f10386e..56c5f7e54ee 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,8 +55,8 @@ class IColumn; M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ - M(Seconds, receive_hello_timeout, DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_SEC, "Connection timeout for receiving hello from replica", 0) \ - M(Seconds, receive_tables_status_timeout, DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_SEC, "Connection timeout for receiving tables status from replica", 0) \ + M(Milliseconds, receive_hello_timeout, DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_MS, "Connection timeout for receiving hello from replica", 0) \ + M(Milliseconds, receive_tables_status_timeout, DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_MS, "Connection timeout for receiving tables status from replica", 0) \ M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data from replica", 0) \ M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \ M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp index c854794cd27..c77b2d48f05 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -127,13 +127,13 @@ bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const bool is_pipe_alarmed = false; bool has_timer_alarm = false; - for (size_t i = 0; i < events.size(); ++i) + for (const auto & event : events) { - if (events[i].data.fd == connection_fd) + if (event.data.fd == connection_fd) is_socket_ready = true; - if (events[i].data.fd == timer.getDescriptor()) + if (event.data.fd == timer.getDescriptor()) has_timer_alarm = true; - if (events[i].data.fd == pipe_fd[0]) + if (event.data.fd == pipe_fd[0]) is_pipe_alarmed = true; } diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 00d28ac62eb..5e63d92b6c5 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -17,9 +17,6 @@ node = cluster.add_instance( node_1 = cluster.add_instance('node_1', with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) node_2 = cluster.add_instance('node_2', with_zookeeper=True) -sleep_timeout = 30 -receive_timeout = 1 - config = ''' @@ -48,14 +45,14 @@ def started_cluster(): finally: cluster.shutdown() -def process_test(sleep_setting_name, receive_timeout_name): +def process_test(sleep_setting_name, receive_timeout_name, receive_timeout, sleep_timeout): node_1.replace_config('/etc/clickhouse-server/users.d/users1.xml', config.format(setting=sleep_setting_name, sleep=sleep_timeout)) # Restart node to make new config relevant node_1.restart_clickhouse(sleep_timeout + 1) # Without hedged requests select query will last more than sleep_timeout seconds, - # with hedged requests it will last just over receive_timeout seconds + # with hedged requests it will last just over receive_timeout node.query("SET {setting}={value}".format(setting=receive_timeout_name, value=receive_timeout)) @@ -65,14 +62,11 @@ def process_test(sleep_setting_name, receive_timeout_name): print(query_time) - # Check that query time is not long - # assert query_time < sleep_timeout - def test(started_cluster): node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") - process_test("sleep_before_send_hello", "receive_hello_timeout") - process_test("sleep_before_send_tables_status", "receive_tables_status_timeout") - process_test("sleep_before_send_data", "receive_data_timeout") + process_test("sleep_before_send_hello", "receive_hello_timeout", 1000, 30) + process_test("sleep_before_send_tables_status", "receive_tables_status_timeout", 1000, 30) + process_test("sleep_before_send_data", "receive_data_timeout", 1, 30) diff --git a/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml b/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml new file mode 100644 index 00000000000..9d753ca2b6a --- /dev/null +++ b/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml @@ -0,0 +1,22 @@ + + + + + true + + node_1 + 9000 + + + node_2 + 9000 + + + node_3 + 9000 + + + + + + diff --git a/tests/integration/test_hedged_requests_parallel/configs/users.xml b/tests/integration/test_hedged_requests_parallel/configs/users.xml new file mode 100644 index 00000000000..0007089f326 --- /dev/null +++ b/tests/integration/test_hedged_requests_parallel/configs/users.xml @@ -0,0 +1,11 @@ + + + + + in_order + 1 + 0 + 2 + + + diff --git a/tests/integration/test_hedged_requests_parallel/configs/users1.xml b/tests/integration/test_hedged_requests_parallel/configs/users1.xml new file mode 100644 index 00000000000..5fe444b94ff --- /dev/null +++ b/tests/integration/test_hedged_requests_parallel/configs/users1.xml @@ -0,0 +1,8 @@ + + + + + 30 + + + diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py new file mode 100644 index 00000000000..65e44095ded --- /dev/null +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -0,0 +1,55 @@ +import os +import sys +import time + +import pytest + +sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) + +from helpers.cluster import ClickHouseCluster +from helpers.network import PartitionManager + +cluster = ClickHouseCluster(__file__) + +# Cluster with 1 shard of 2 replicas. node is the instance with Distributed table. +node = cluster.add_instance( + 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) +node_1 = cluster.add_instance('node_1', with_zookeeper=True, user_configs=['configs/users1.xml']) +node_2 = cluster.add_instance('node_2', with_zookeeper=True) +node_3 = cluster.add_instance('node_3', with_zookeeper=True) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + node_1.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_1') ORDER BY id PARTITION BY toYYYYMM(date)''') + + node_2.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_2') ORDER BY id PARTITION BY toYYYYMM(date)''') + + node_3.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_3') ORDER BY id PARTITION BY toYYYYMM(date)''') + + node.query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = + Distributed('test_cluster', 'default', 'replicated')''') + + yield cluster + + finally: + cluster.shutdown() + +def test(started_cluster): + node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") + + # Without hedged requests select query will last more 30 seconds, + # with hedged requests it will last just over 2 seconds + + start = time.time() + node.query("SELECT * FROM distributed"); + query_time = time.time() - start + + print(query_time) + From 02cc43502edd12e90577b093d59c9fdbfb9b4c75 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Feb 2021 15:17:06 +0300 Subject: [PATCH 047/510] Remove LOG_DEBUG --- src/Client/ConnectionPoolWithFailover.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index af4f8bb2d25..8a67d59925a 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -132,8 +132,6 @@ std::vector ConnectionPoolWithFailover::getMany(const Co const Settings * settings, PoolMode pool_mode) { - LOG_DEBUG(log, "ConnectionPoolWithFailover getMany"); - TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) { return tryGetEntry(pool, timeouts, fail_message, settings); @@ -166,9 +164,6 @@ std::vector ConnectionPoolWithFailover::g const Settings * settings, PoolMode pool_mode, const QualifiedTableName & table_to_check) { - - LOG_DEBUG(log, "ConnectionPoolWithFailover getManyChecked"); - TryGetEntryFunc try_get_entry = [&](NestedPool & pool, std::string & fail_message) { return tryGetEntry(pool, timeouts, fail_message, settings, &table_to_check); @@ -219,7 +214,6 @@ std::vector ConnectionPoolWithFailover::g PoolMode pool_mode, const TryGetEntryFunc & try_get_entry) { - LOG_DEBUG(log, "ConnectionPoolWithFailover getManyImpl"); size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; size_t max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : @@ -258,11 +252,8 @@ ConnectionPoolWithFailover::tryGetEntry( TryResult result; try { - LOG_DEBUG(log, "ConnectionPoolWithFailover tryGetEntry"); result.entry = pool.get(timeouts, settings, /* force_connected = */ false); - LOG_DEBUG(log, "ConnectionPoolWithFailover isConnected {}", result.entry->isConnected()); - UInt64 server_revision = 0; if (table_to_check) server_revision = result.entry->getServerRevision(timeouts); From cc14cb11f9edc20ad634c62de5d9ec959ea9fd80 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Feb 2021 15:20:13 +0300 Subject: [PATCH 048/510] Update test --- tests/integration/test_hedged_requests_parallel/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 65e44095ded..b662fc9d80c 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -11,7 +11,7 @@ from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) -# Cluster with 1 shard of 2 replicas. node is the instance with Distributed table. +# Cluster with 1 shard of 3 replicas. node is the instance with Distributed table. node = cluster.add_instance( 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) node_1 = cluster.add_instance('node_1', with_zookeeper=True, user_configs=['configs/users1.xml']) From ed3de186a4c34fd9c39656b6723f89b3cafc4d40 Mon Sep 17 00:00:00 2001 From: benbiti Date: Tue, 2 Feb 2021 20:26:36 +0800 Subject: [PATCH 049/510] [Docs]fix mistype in avg --- docs/en/sql-reference/aggregate-functions/reference/avg.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/avg.md b/docs/en/sql-reference/aggregate-functions/reference/avg.md index e2e6aace734..0b80a1be704 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/avg.md +++ b/docs/en/sql-reference/aggregate-functions/reference/avg.md @@ -9,7 +9,7 @@ Calculates the arithmetic mean. **Syntax** ``` sql -avgWeighted(x) +avg(x) ``` **Parameter** From 0b4a9ed87a56f8bd83eb4079ab7296784c693942 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 2 Feb 2021 17:38:42 +0300 Subject: [PATCH 050/510] Fix gcc-9 build --- src/Common/TimerDescriptor.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index 6138ed8f395..debf7cdc899 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -39,7 +39,5 @@ public: void setType(int type_) { type = type_; } }; -using TimerDescriptorPtr = std::shared_ptr; - } #endif From dd9af192c56668bb8a323671fce5c11d27ecf254 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Feb 2021 18:18:05 +0300 Subject: [PATCH 051/510] Update test --- .../test_hedged_requests/configs/users.xml | 2 -- .../integration/test_hedged_requests/test.py | 20 +++++++++---------- .../configs/users.xml | 2 -- 3 files changed, 9 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_hedged_requests/configs/users.xml b/tests/integration/test_hedged_requests/configs/users.xml index 0cf32bf9e1a..c95d73a92ed 100644 --- a/tests/integration/test_hedged_requests/configs/users.xml +++ b/tests/integration/test_hedged_requests/configs/users.xml @@ -3,8 +3,6 @@ in_order - 1 - 1 diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 5e63d92b6c5..992590b516f 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -20,7 +20,7 @@ node_2 = cluster.add_instance('node_2', with_zookeeper=True) config = ''' - <{setting}>{sleep} + <{setting}>30 ''' @@ -45,16 +45,14 @@ def started_cluster(): finally: cluster.shutdown() -def process_test(sleep_setting_name, receive_timeout_name, receive_timeout, sleep_timeout): - node_1.replace_config('/etc/clickhouse-server/users.d/users1.xml', config.format(setting=sleep_setting_name, sleep=sleep_timeout)) +def process_test(sleep_setting_name, receive_timeout_name): + node_1.replace_config('/etc/clickhouse-server/users.d/users1.xml', config.format(setting=sleep_setting_name)) # Restart node to make new config relevant - node_1.restart_clickhouse(sleep_timeout + 1) + node_1.restart_clickhouse(30) - # Without hedged requests select query will last more than sleep_timeout seconds, - # with hedged requests it will last just over receive_timeout - - node.query("SET {setting}={value}".format(setting=receive_timeout_name, value=receive_timeout)) + # Without hedged requests select query will last more than 30 seconds, + # with hedged requests it will last just around 1-2 second start = time.time() node.query("SELECT * FROM distributed"); @@ -66,7 +64,7 @@ def process_test(sleep_setting_name, receive_timeout_name, receive_timeout, slee def test(started_cluster): node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") - process_test("sleep_before_send_hello", "receive_hello_timeout", 1000, 30) - process_test("sleep_before_send_tables_status", "receive_tables_status_timeout", 1000, 30) - process_test("sleep_before_send_data", "receive_data_timeout", 1, 30) + process_test("sleep_before_send_hello", "receive_hello_timeout") + process_test("sleep_before_send_tables_status", "receive_tables_status_timeout") + process_test("sleep_before_send_data", "receive_data_timeout") diff --git a/tests/integration/test_hedged_requests_parallel/configs/users.xml b/tests/integration/test_hedged_requests_parallel/configs/users.xml index 0007089f326..c3ba59294a5 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users.xml @@ -3,8 +3,6 @@ in_order - 1 - 0 2 From 2c928f11e1c18ee1cb78e33ff08025297256632e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Feb 2021 19:39:30 +0300 Subject: [PATCH 052/510] Remove code duplication --- src/Client/ConnectionPoolWithFailover.cpp | 91 ++++------------------- src/Client/ConnectionPoolWithFailover.h | 8 +- src/Client/GetHedgedConnections.cpp | 2 +- 3 files changed, 20 insertions(+), 81 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 8a67d59925a..a7120f16b4d 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -249,77 +249,10 @@ ConnectionPoolWithFailover::tryGetEntry( const Settings * settings, const QualifiedTableName * table_to_check) { - TryResult result; - try - { - result.entry = pool.get(timeouts, settings, /* force_connected = */ false); - - UInt64 server_revision = 0; - if (table_to_check) - server_revision = result.entry->getServerRevision(timeouts); - - if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) - { - result.entry->forceConnected(timeouts); - result.is_usable = true; - result.is_up_to_date = true; - return result; - } - - /// Only status of the remote table corresponding to the Distributed table is taken into account. - /// TODO: request status for joined tables also. - TablesStatusRequest status_request; - status_request.tables.emplace(*table_to_check); - - TablesStatusResponse status_response = result.entry->getTablesStatus(timeouts, status_request); - auto table_status_it = status_response.table_states_by_id.find(*table_to_check); - if (table_status_it == status_response.table_states_by_id.end()) - { - const char * message_pattern = "There is no table {}.{} on server: {}"; - fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); - LOG_WARNING(log, fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); - - return result; - } - - result.is_usable = true; - - UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; - if (!max_allowed_delay) - { - result.is_up_to_date = true; - return result; - } - - UInt32 delay = table_status_it->second.absolute_delay; - - if (delay < max_allowed_delay) - result.is_up_to_date = true; - else - { - result.is_up_to_date = false; - result.staleness = delay; - - LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); - ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - } - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT - && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - throw; - - fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); - - if (!result.entry.isNull()) - { - result.entry->disconnect(); - result.reset(); - } - } - return result; + TryGetConnection try_get_connection(&pool, &timeouts, settings, table_to_check, log, false); + try_get_connection.run(); + fail_message = try_get_connection.fail_message; + return try_get_connection.result; } std::vector ConnectionPoolWithFailover::getShuffledPools(const Settings * settings) @@ -333,10 +266,11 @@ TryGetConnection::TryGetConnection( IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings * settings_, - std::shared_ptr table_to_check_, - Poco::Logger * log_) : + const QualifiedTableName * table_to_check_, + Poco::Logger * log_, + bool non_blocking_) : pool(pool_), timeouts(timeouts_), settings(settings_), - table_to_check(table_to_check_), log(log_), stage(Stage::CONNECT), socket_fd(-1) + table_to_check(table_to_check_), log(log_), stage(Stage::CONNECT), socket_fd(-1), non_blocking(non_blocking_) { } @@ -386,7 +320,8 @@ void TryGetConnection::run() result.entry->sendHello(); stage = Stage::RECEIVE_HELLO; /// We are waiting for hello from replica. - return; + if (non_blocking) + return; } socket_fd = result.entry->getSocket()->impl()->sockfd(); @@ -411,7 +346,8 @@ void TryGetConnection::run() result.is_usable = true; result.is_up_to_date = true; stage = FINISHED; - return; + if (non_blocking) + return; } TablesStatusRequest status_request; @@ -420,7 +356,8 @@ void TryGetConnection::run() result.entry->sendTablesStatusRequest(status_request); stage = Stage::RECEIVE_TABLES_STATUS; /// We are waiting for tables status response. - return; + if (non_blocking) + return; } if (stage == Stage::RECEIVE_TABLES_STATUS) diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index f235c5b2e5f..c4248effa81 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -50,8 +50,9 @@ public: TryGetConnection(IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings * settings_, - std::shared_ptr table_to_check = nullptr, - Poco::Logger * log_ = nullptr); + const QualifiedTableName * table_to_check = nullptr, + Poco::Logger * log_ = nullptr, + bool non_blocking_ = true); /// Continue connecting to replica from previous stage. Initial stage is CONNECT. void run(); @@ -72,11 +73,12 @@ public: const ConnectionTimeouts * timeouts; std::string fail_message; const Settings * settings; - std::shared_ptr table_to_check; + const QualifiedTableName * table_to_check; Poco::Logger * log; TryResult result; Stage stage; int socket_fd; + bool non_blocking; std::function action_before_disconnect; }; diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp index a9283a75105..093b4bc930c 100644 --- a/src/Client/GetHedgedConnections.cpp +++ b/src/Client/GetHedgedConnections.cpp @@ -23,7 +23,7 @@ GetHedgedConnections::GetHedgedConnections( { shuffled_pools = pool->getShuffledPools(settings); for (size_t i = 0; i != shuffled_pools.size(); ++i) - try_get_connections.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check, log); + try_get_connections.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); From b8ae9caa619a42e65fe39ea24d21ffbbee1859b1 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 3 Feb 2021 14:27:26 +0300 Subject: [PATCH 053/510] Fix style --- src/Client/ConnectionPoolWithFailover.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index a7120f16b4d..a0dfe1a1a8c 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -24,8 +24,6 @@ namespace DB namespace ErrorCodes { extern const int ATTEMPT_TO_READ_AFTER_EOF; - extern const int NETWORK_ERROR; - extern const int SOCKET_TIMEOUT; extern const int LOGICAL_ERROR; } From 3fc8b294e8275294fd3aaafaa1d1f22aae4d8a03 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 3 Feb 2021 15:56:42 +0300 Subject: [PATCH 054/510] Reset changes in tryGetEntry --- src/Client/ConnectionPoolWithFailover.cpp | 91 +++++++++++++++++++---- src/Client/ConnectionPoolWithFailover.h | 4 +- 2 files changed, 79 insertions(+), 16 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index a0dfe1a1a8c..3e41c26fb65 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -24,6 +24,8 @@ namespace DB namespace ErrorCodes { extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int NETWORK_ERROR; + extern const int SOCKET_TIMEOUT; extern const int LOGICAL_ERROR; } @@ -247,10 +249,77 @@ ConnectionPoolWithFailover::tryGetEntry( const Settings * settings, const QualifiedTableName * table_to_check) { - TryGetConnection try_get_connection(&pool, &timeouts, settings, table_to_check, log, false); - try_get_connection.run(); - fail_message = try_get_connection.fail_message; - return try_get_connection.result; + TryResult result; + try + { + result.entry = pool.get(timeouts, settings, /* force_connected = */ false); + + UInt64 server_revision = 0; + if (table_to_check) + server_revision = result.entry->getServerRevision(timeouts); + + if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) + { + result.entry->forceConnected(timeouts); + result.is_usable = true; + result.is_up_to_date = true; + return result; + } + + /// Only status of the remote table corresponding to the Distributed table is taken into account. + /// TODO: request status for joined tables also. + TablesStatusRequest status_request; + status_request.tables.emplace(*table_to_check); + + TablesStatusResponse status_response = result.entry->getTablesStatus(timeouts, status_request); + auto table_status_it = status_response.table_states_by_id.find(*table_to_check); + if (table_status_it == status_response.table_states_by_id.end()) + { + const char * message_pattern = "There is no table {}.{} on server: {}"; + fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); + LOG_WARNING(log, fail_message); + ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); + + return result; + } + + result.is_usable = true; + + UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; + if (!max_allowed_delay) + { + result.is_up_to_date = true; + return result; + } + + UInt32 delay = table_status_it->second.absolute_delay; + + if (delay < max_allowed_delay) + result.is_up_to_date = true; + else + { + result.is_up_to_date = false; + result.staleness = delay; + + LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); + ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); + } + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT + && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throw; + + fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); + + if (!result.entry.isNull()) + { + result.entry->disconnect(); + result.reset(); + } + } + return result; } std::vector ConnectionPoolWithFailover::getShuffledPools(const Settings * settings) @@ -265,10 +334,9 @@ TryGetConnection::TryGetConnection( const ConnectionTimeouts * timeouts_, const Settings * settings_, const QualifiedTableName * table_to_check_, - Poco::Logger * log_, - bool non_blocking_) : + Poco::Logger * log_) : pool(pool_), timeouts(timeouts_), settings(settings_), - table_to_check(table_to_check_), log(log_), stage(Stage::CONNECT), socket_fd(-1), non_blocking(non_blocking_) + table_to_check(table_to_check_), log(log_), stage(Stage::CONNECT), socket_fd(-1) { } @@ -318,8 +386,7 @@ void TryGetConnection::run() result.entry->sendHello(); stage = Stage::RECEIVE_HELLO; /// We are waiting for hello from replica. - if (non_blocking) - return; + return; } socket_fd = result.entry->getSocket()->impl()->sockfd(); @@ -344,8 +411,7 @@ void TryGetConnection::run() result.is_usable = true; result.is_up_to_date = true; stage = FINISHED; - if (non_blocking) - return; + return; } TablesStatusRequest status_request; @@ -354,8 +420,7 @@ void TryGetConnection::run() result.entry->sendTablesStatusRequest(status_request); stage = Stage::RECEIVE_TABLES_STATUS; /// We are waiting for tables status response. - if (non_blocking) - return; + return; } if (stage == Stage::RECEIVE_TABLES_STATUS) diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index c4248effa81..a6c0b9e8070 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -51,8 +51,7 @@ public: const ConnectionTimeouts * timeouts_, const Settings * settings_, const QualifiedTableName * table_to_check = nullptr, - Poco::Logger * log_ = nullptr, - bool non_blocking_ = true); + Poco::Logger * log_ = nullptr); /// Continue connecting to replica from previous stage. Initial stage is CONNECT. void run(); @@ -78,7 +77,6 @@ public: TryResult result; Stage stage; int socket_fd; - bool non_blocking; std::function action_before_disconnect; }; From 1bd80f6c521432c916d08c9f4d91bc3c45cd0589 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 4 Feb 2021 23:59:00 +0300 Subject: [PATCH 055/510] Update docs/ru/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/date-time-functions.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index b23862ccce2..aa03874d54f 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -688,7 +688,9 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g'); Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). -**Пример** +**Примеры** + +Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). Запрос: From c1328a963885058eec375f527500c40f5b121973 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 4 Feb 2021 23:59:18 +0300 Subject: [PATCH 056/510] Update docs/ru/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/date-time-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index aa03874d54f..14c7ebc7ae9 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -708,7 +708,6 @@ SELECT FROM_UNIXTIME(423543535); В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). -**Пример** Запрос: From d7098e56782187e7740fadaca93304ca2eb6310e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Fri, 5 Feb 2021 00:00:04 +0300 Subject: [PATCH 057/510] Update docs/ru/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 14c7ebc7ae9..0acb9e3cd39 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -686,7 +686,7 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g'); ## FROM\_UNIXTIME {#fromunixtime} -Функция преобразует метку времени Unix в дату. Если указан только один аргумент типа [Integer](../../sql-reference/data-types/int-uint.md), то функция действует так же, как [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime), и возвращает тип [DateTime](../../sql-reference/data-types/datetime.md). +Функция преобразует метку времени Unix в дату. **Примеры** From 0fbb3473079e171d3d9903c06b326e5cc9d84627 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 5 Feb 2021 00:39:14 +0300 Subject: [PATCH 058/510] DOCSUP-5266: Fix ticket comments. --- .../data-types/simpleaggregatefunction.md | 20 +++++++++++++----- .../data-types/simpleaggregatefunction.md | 21 +++++++++++++------ 2 files changed, 30 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index 155a7e1f858..9ea5a586981 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -32,14 +32,24 @@ The following aggregate functions are supported: - Name of the aggregate function. - Types of the aggregate function arguments. -**Syntax** +**Example** + +Query: ``` sql -CREATE TABLE t +CREATE TABLE simple (id UInt64,val SimpleAggregateFunction(sum,Double)) ENGINE=AggregatingMergeTree ORDER BY id; +``` + +Result: + +``` text +CREATE TABLE simple ( - column1 SimpleAggregateFunction(sum, UInt64), - column2 SimpleAggregateFunction(any, String) -) ENGINE = ... + `id` UInt64, + `val` SimpleAggregateFunction(sum, Double) +) +ENGINE = AggregatingMergeTree +ORDER BY id ``` [Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 9605706442e..7441ceae655 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -25,20 +25,29 @@ `SimpleAggregateFunction` имеет лучшую производительность, чем `AggregateFunction` с той же агрегатной функцией. - **Параметры** - `func` — имя агрегатной функции. - `type` — типы аргументов агрегатной функции. -**Синтаксис** +**Пример** + +Запрос: ``` sql -CREATE TABLE t +CREATE TABLE simple (id UInt64,val SimpleAggregateFunction(sum,Double)) ENGINE=AggregatingMergeTree ORDER BY id; +``` + +Ответ: + +``` text +CREATE TABLE simple ( - column1 SimpleAggregateFunction(sum, UInt64), - column2 SimpleAggregateFunction(any, String) -) ENGINE = ... + `id` UInt64, + `val` SimpleAggregateFunction(sum, Double) +) +ENGINE = AggregatingMergeTree +ORDER BY id ``` [Оригинальная статья](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From 9a9138d0380ddf67cceda85eb26f8c4d2c978b63 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 5 Feb 2021 01:37:59 +0300 Subject: [PATCH 059/510] DOCSUP-5266: Fix ticket comments. --- .../functions/type-conversion-functions.md | 119 ++++++++++---- .../functions/type-conversion-functions.md | 149 +++++++++++++----- 2 files changed, 194 insertions(+), 74 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 047b3b1cbea..b2ede6ba6ec 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -36,10 +36,14 @@ The behavior of functions for the [NaN and Inf](../../sql-reference/data-types/f **Example** +Query: + ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) +SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); ``` +Result: + ``` text ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ @@ -52,10 +56,14 @@ It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 3 **Example** +Query: + ``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123') +select toInt64OrZero('123123'), toInt8OrZero('123qwe123'); ``` +Result: + ``` text ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ @@ -68,10 +76,14 @@ It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 3 **Example** +Query: + ``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123') +select toInt64OrNull('123123'), toInt8OrNull('123qwe123'); ``` +String: + ``` text ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ @@ -102,10 +114,14 @@ The behavior of functions for negative agruments and for the [NaN and Inf](../.. **Example** +Query: + ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) +SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); ``` +Result: + ``` text ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ @@ -168,20 +184,28 @@ A value in the `Nullable(Decimal(P,S))` data type. The value contains: **Examples** +Query: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); ``` +Result: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.11100 │ Nullable(Decimal(9, 5)) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Query: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); ``` +Result: + ``` text ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ @@ -348,7 +372,7 @@ String to UUID. Query: ``` sql -SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); ``` Result: @@ -381,9 +405,11 @@ Result: ## CAST(x, T) {#type_conversion_function-cast} -Converts ‘x’ to the ‘t’ data type. The syntax CAST(x AS t) is also supported. +Converts unput value `x` to the `T` data type. The syntax `CAST(x AS t)` is also supported. -Example: +**Example** + +Query: ``` sql SELECT @@ -394,6 +420,8 @@ SELECT CAST(timestamp, 'FixedString(22)') AS fixed_string ``` +Result: + ``` text ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ @@ -402,12 +430,18 @@ SELECT Conversion to FixedString(N) only works for arguments of type String or FixedString(N). -Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported. Example: +Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported. + +**Examples** + +Query: ``` sql -SELECT toTypeName(x) FROM t_null +SELECT toTypeName(x) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(x)─┐ │ Int8 │ @@ -415,10 +449,14 @@ SELECT toTypeName(x) FROM t_null └───────────────┘ ``` +Query: + ``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null +SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ @@ -432,15 +470,18 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null ## accurateCast(x, T) {#type_conversion_function-accurate-cast} -Converts ‘x’ to the ‘t’ data type. The differente from cast(x, T) is that accurateCast -does not allow overflow of numeric types during cast if type value x does not fit -bounds of type T. +Converts `x` to the `T` data type. The differente from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` +does not allow overflow of numeric types during cast if type value `x` does not fit bounds of type `T`. + +**Examples** + +Query: -Example ``` sql -SELECT cast(-1, 'UInt8') as uint8; +SELECT cast(-1, 'UInt8') as uint8; ``` +Result: ``` text ┌─uint8─┐ @@ -448,10 +489,14 @@ SELECT cast(-1, 'UInt8') as uint8; └───────┘ ``` +Query: + ```sql SELECT accurateCast(-1, 'UInt8') as uint8; ``` +Result: + ``` text Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in column Int8 cannot be safely converted into type UInt8: While processing accurateCast(-1, 'UInt8') AS uint8. @@ -472,7 +517,7 @@ accurateCastOrNull(x, T) - `x` — Input value. - `T` — The name of the returned data type. -**Example** +**Examples** Query: @@ -488,6 +533,8 @@ Result: └────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT cast(-1, 'UInt8') as uint8, @@ -530,6 +577,8 @@ toIntervalYear(number) **Example** +Query: + ``` sql WITH toDate('2019-01-01') AS date, @@ -537,9 +586,11 @@ WITH toIntervalWeek(1) AS interval_to_week SELECT date + interval_week, - date + interval_to_week + date + interval_to_week; ``` +Result: + ``` text ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ @@ -598,7 +649,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Result: @@ -613,7 +664,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Result: @@ -628,7 +679,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Result: @@ -642,7 +693,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffort('10 20:19') +SELECT parseDateTimeBestEffort('10 20:19'); ``` Result: @@ -667,7 +718,7 @@ This function is similar to [‘parseDateTimeBestEffort’](#parsedatetimebestef **Syntax** ``` sql -parseDateTimeBestEffortUS(time_string [, time_zone]); +parseDateTimeBestEffortUS(time_string [, time_zone]) ``` **Parameters** @@ -769,7 +820,7 @@ Type: `LowCardinality(expr_result_type)` Query: ``` sql -SELECT toLowCardinality('1') +SELECT toLowCardinality('1'); ``` Result: @@ -808,7 +859,7 @@ Query: ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Milli(dt64) +SELECT toUnixTimestamp64Milli(dt64); ``` Result: @@ -819,9 +870,11 @@ Result: └──────────────────────────────┘ ``` +Query: + ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Nano(dt64) +SELECT toUnixTimestamp64Nano(dt64); ``` Result: @@ -855,13 +908,17 @@ fromUnixTimestamp64Milli(value [, ti]) - `value` converted to the `DateTime64` data type. -**Examples** +**Example** + +Query: ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC') +SELECT fromUnixTimestamp64Milli(i64, 'UTC'); ``` +Result: + ``` text ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ │ 2009-02-13 23:31:31.011 │ @@ -893,7 +950,7 @@ Query: ``` sql SELECT formatRow('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` Result: @@ -934,7 +991,7 @@ Query: ``` sql SELECT formatRowNoNewline('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` Result: diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 40fdbc6f5a0..ee3e8583504 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -36,10 +36,14 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438\u0020\u043f\u0440\u0435\u **Пример** +Запрос: + ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) +SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); ``` +Результат: + ``` text ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ @@ -52,10 +56,14 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) **Пример** +Запрос: + ``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123') +select toInt64OrZero('123123'), toInt8OrZero('123qwe123'); ``` +Результат: + ``` text ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ @@ -68,10 +76,14 @@ select toInt64OrZero('123123'), toInt8OrZero('123qwe123') **Пример** +Запрос: + ``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123') +select toInt64OrNull('123123'), toInt8OrNull('123qwe123'); ``` +Результат: + ``` text ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ @@ -102,10 +114,14 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123') **Пример** +Запрос: + ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) +SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); ``` +Результат: + ``` text ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ @@ -168,20 +184,28 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) **Примеры** +Запрос: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.11100 │ Nullable(Decimal(9, 5)) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Запрос: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ @@ -211,22 +235,30 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) - Число с `S` десятичными знаками, если ClickHouse распознал число во входной строке. - 0 c `S` десятичными знаками, если ClickHouse не смог распознать число во входной строке или входное число содержит больше чем `S` десятичных знаков. -**Пример** +**Примеры** + +Запрос: ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ │ -1.11100 │ Decimal(9, 5) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Запрос: + ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ │ 0.00 │ Decimal(9, 2) │ @@ -258,12 +290,18 @@ YYYY-MM-DD hh:mm:ss Дополнительно, функция toString от аргумента типа DateTime может принимать второй аргумент String - имя тайм-зоны. Пример: `Asia/Yekaterinburg` В этом случае, форматирование времени производится согласно указанной тайм-зоне. +**Пример** + +Запрос: + ``` sql SELECT now() AS now_local, - toString(now(), 'Asia/Yekaterinburg') AS now_yekat + toString(now(), 'Asia/Yekaterinburg') AS now_yekat; ``` +Результат: + ``` text ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ @@ -281,22 +319,30 @@ SELECT Принимает аргумент типа String или FixedString. Возвращает String, вырезая содержимое строки до первого найденного нулевого байта. -Пример: +**Примеры** + +Запрос: ``` sql -SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut +SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut; ``` +Результат: + ``` text ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ ``` +Запрос: + ``` sql -SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut +SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; ``` +Результат: + ``` text ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ @@ -344,7 +390,7 @@ reinterpretAsUUID(fixed_string) Запрос: ``` sql -SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); ``` Результат: @@ -377,10 +423,11 @@ SELECT uuid = uuid2; ## CAST(x, T) {#type_conversion_function-cast} -Преобразует x в тип данных t. -Поддерживается также синтаксис CAST(x AS t). +Преобразует входное значение `x` в тип данных `T`. Поддерживается также синтаксис `CAST(x AS t)`. -Пример: +**Пример** + +Запрос: ``` sql SELECT @@ -388,9 +435,11 @@ SELECT CAST(timestamp AS DateTime) AS datetime, CAST(timestamp AS Date) AS date, CAST(timestamp, 'String') AS string, - CAST(timestamp, 'FixedString(22)') AS fixed_string + CAST(timestamp, 'FixedString(22)') AS fixed_string; ``` +Результат: + ``` text ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ @@ -399,12 +448,18 @@ SELECT Преобразование в FixedString(N) работает только для аргументов типа String или FixedString(N). -Поддержано преобразование к типу [Nullable](../../sql-reference/functions/type-conversion-functions.md) и обратно. Пример: +Поддерживается преобразование к типу [Nullable](../../sql-reference/functions/type-conversion-functions.md) и обратно. + +**Примеры** + +Запрос: ``` sql -SELECT toTypeName(x) FROM t_null +SELECT toTypeName(x) FROM t_null; ``` +Результат: + ``` text ┌─toTypeName(x)─┐ │ Int8 │ @@ -412,10 +467,14 @@ SELECT toTypeName(x) FROM t_null └───────────────┘ ``` +Запрос: + ``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null +SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; ``` +Результат: + ``` text ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ @@ -442,7 +501,7 @@ accurateCastOrNull(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. -**Пример** +**Примеры** Запрос: @@ -502,6 +561,8 @@ toIntervalYear(number) **Пример** +Запрос: + ``` sql WITH toDate('2019-01-01') AS date, @@ -509,9 +570,11 @@ WITH toIntervalWeek(1) AS interval_to_week SELECT date + interval_week, - date + interval_to_week + date + interval_to_week; ``` +Результат: + ``` text ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ @@ -527,7 +590,7 @@ SELECT **Синтаксис** ``` sql -parseDateTimeBestEffort(time_string[, time_zone]); +parseDateTimeBestEffort(time_string[, time_zone]) ``` **Параметры** @@ -570,7 +633,7 @@ AS parseDateTimeBestEffort; ``` sql SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Результат: @@ -585,7 +648,7 @@ AS parseDateTimeBestEffort ``` sql SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Результат: @@ -600,7 +663,7 @@ AS parseDateTimeBestEffort ``` sql SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Результат: @@ -614,7 +677,7 @@ AS parseDateTimeBestEffort Запрос: ``` sql -SELECT parseDateTimeBestEffort('10 20:19') +SELECT parseDateTimeBestEffort('10 20:19'); ``` Результат: @@ -639,7 +702,7 @@ SELECT parseDateTimeBestEffort('10 20:19') **Синтаксис** ``` sql -parseDateTimeBestEffortUS(time_string [, time_zone]); +parseDateTimeBestEffortUS(time_string [, time_zone]) ``` **Параметры** @@ -668,7 +731,7 @@ SELECT parseDateTimeBestEffortUS('09/12/2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Ответ: +Результат: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -683,7 +746,7 @@ SELECT parseDateTimeBestEffortUS('09-12-2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Ответ: +Результат: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -698,7 +761,7 @@ SELECT parseDateTimeBestEffortUS('09.12.2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Ответ: +Результат: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -733,10 +796,10 @@ toUnixTimestamp64Milli(value) ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Milli(dt64) +SELECT toUnixTimestamp64Milli(dt64); ``` -Ответ: +Результат: ``` text ┌─toUnixTimestamp64Milli(dt64)─┐ @@ -748,10 +811,10 @@ SELECT toUnixTimestamp64Milli(dt64) ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Nano(dt64) +SELECT toUnixTimestamp64Nano(dt64); ``` -Ответ: +Результат: ``` text ┌─toUnixTimestamp64Nano(dt64)─┐ @@ -786,10 +849,10 @@ fromUnixTimestamp64Milli(value [, ti]) ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC') +SELECT fromUnixTimestamp64Milli(i64, 'UTC'); ``` -Ответ: +Результат: ``` text ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ @@ -820,7 +883,7 @@ toLowCardinality(expr) Тип: `LowCardinality(expr_result_type)` -**Example** +**Пример** Запрос: @@ -861,10 +924,10 @@ formatRow(format, x, y, ...) ``` sql SELECT formatRow('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` -Ответ: +Результат: ``` text ┌─formatRow('CSV', number, 'good')─┐ @@ -902,10 +965,10 @@ formatRowNoNewline(format, x, y, ...) ``` sql SELECT formatRowNoNewline('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` -Ответ: +Результат: ``` text ┌─formatRowNoNewline('CSV', number, 'good')─┐ From a3721ef0ac77046bc0db336b0bb71aa274b2fe97 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 5 Feb 2021 01:44:18 +0300 Subject: [PATCH 060/510] Revert "DOCSUP-5266: Fix ticket comments." This reverts commit 9a9138d0380ddf67cceda85eb26f8c4d2c978b63. --- .../functions/type-conversion-functions.md | 119 ++++---------- .../functions/type-conversion-functions.md | 149 +++++------------- 2 files changed, 74 insertions(+), 194 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index b2ede6ba6ec..047b3b1cbea 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -36,14 +36,10 @@ The behavior of functions for the [NaN and Inf](../../sql-reference/data-types/f **Example** -Query: - ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); +SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) ``` -Result: - ``` text ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ @@ -56,14 +52,10 @@ It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 3 **Example** -Query: - ``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123'); +select toInt64OrZero('123123'), toInt8OrZero('123qwe123') ``` -Result: - ``` text ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ @@ -76,14 +68,10 @@ It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 3 **Example** -Query: - ``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123'); +select toInt64OrNull('123123'), toInt8OrNull('123qwe123') ``` -String: - ``` text ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ @@ -114,14 +102,10 @@ The behavior of functions for negative agruments and for the [NaN and Inf](../.. **Example** -Query: - ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); +SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) ``` -Result: - ``` text ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ @@ -184,28 +168,20 @@ A value in the `Nullable(Decimal(P,S))` data type. The value contains: **Examples** -Query: - ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); +SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) ``` -Result: - ``` text ┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.11100 │ Nullable(Decimal(9, 5)) │ └──────────┴────────────────────────────────────────────────────┘ ``` -Query: - ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); +SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) ``` -Result: - ``` text ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ @@ -372,7 +348,7 @@ String to UUID. Query: ``` sql -SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) ``` Result: @@ -405,11 +381,9 @@ Result: ## CAST(x, T) {#type_conversion_function-cast} -Converts unput value `x` to the `T` data type. The syntax `CAST(x AS t)` is also supported. +Converts ‘x’ to the ‘t’ data type. The syntax CAST(x AS t) is also supported. -**Example** - -Query: +Example: ``` sql SELECT @@ -420,8 +394,6 @@ SELECT CAST(timestamp, 'FixedString(22)') AS fixed_string ``` -Result: - ``` text ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ @@ -430,18 +402,12 @@ Result: Conversion to FixedString(N) only works for arguments of type String or FixedString(N). -Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported. - -**Examples** - -Query: +Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported. Example: ``` sql -SELECT toTypeName(x) FROM t_null; +SELECT toTypeName(x) FROM t_null ``` -Result: - ``` text ┌─toTypeName(x)─┐ │ Int8 │ @@ -449,14 +415,10 @@ Result: └───────────────┘ ``` -Query: - ``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; +SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null ``` -Result: - ``` text ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ @@ -470,18 +432,15 @@ Result: ## accurateCast(x, T) {#type_conversion_function-accurate-cast} -Converts `x` to the `T` data type. The differente from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` -does not allow overflow of numeric types during cast if type value `x` does not fit bounds of type `T`. - -**Examples** - -Query: +Converts ‘x’ to the ‘t’ data type. The differente from cast(x, T) is that accurateCast +does not allow overflow of numeric types during cast if type value x does not fit +bounds of type T. +Example ``` sql -SELECT cast(-1, 'UInt8') as uint8; +SELECT cast(-1, 'UInt8') as uint8; ``` -Result: ``` text ┌─uint8─┐ @@ -489,14 +448,10 @@ Result: └───────┘ ``` -Query: - ```sql SELECT accurateCast(-1, 'UInt8') as uint8; ``` -Result: - ``` text Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in column Int8 cannot be safely converted into type UInt8: While processing accurateCast(-1, 'UInt8') AS uint8. @@ -517,7 +472,7 @@ accurateCastOrNull(x, T) - `x` — Input value. - `T` — The name of the returned data type. -**Examples** +**Example** Query: @@ -533,8 +488,6 @@ Result: └────────────────────────────────────────────┘ ``` -Query: - ``` sql SELECT cast(-1, 'UInt8') as uint8, @@ -577,8 +530,6 @@ toIntervalYear(number) **Example** -Query: - ``` sql WITH toDate('2019-01-01') AS date, @@ -586,11 +537,9 @@ WITH toIntervalWeek(1) AS interval_to_week SELECT date + interval_week, - date + interval_to_week; + date + interval_to_week ``` -Result: - ``` text ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ @@ -649,7 +598,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort; +AS parseDateTimeBestEffort ``` Result: @@ -664,7 +613,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort; +AS parseDateTimeBestEffort ``` Result: @@ -679,7 +628,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort; +AS parseDateTimeBestEffort ``` Result: @@ -693,7 +642,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffort('10 20:19'); +SELECT parseDateTimeBestEffort('10 20:19') ``` Result: @@ -718,7 +667,7 @@ This function is similar to [‘parseDateTimeBestEffort’](#parsedatetimebestef **Syntax** ``` sql -parseDateTimeBestEffortUS(time_string [, time_zone]) +parseDateTimeBestEffortUS(time_string [, time_zone]); ``` **Parameters** @@ -820,7 +769,7 @@ Type: `LowCardinality(expr_result_type)` Query: ``` sql -SELECT toLowCardinality('1'); +SELECT toLowCardinality('1') ``` Result: @@ -859,7 +808,7 @@ Query: ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Milli(dt64); +SELECT toUnixTimestamp64Milli(dt64) ``` Result: @@ -870,11 +819,9 @@ Result: └──────────────────────────────┘ ``` -Query: - ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Nano(dt64); +SELECT toUnixTimestamp64Nano(dt64) ``` Result: @@ -908,17 +855,13 @@ fromUnixTimestamp64Milli(value [, ti]) - `value` converted to the `DateTime64` data type. -**Example** - -Query: +**Examples** ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC'); +SELECT fromUnixTimestamp64Milli(i64, 'UTC') ``` -Result: - ``` text ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ │ 2009-02-13 23:31:31.011 │ @@ -950,7 +893,7 @@ Query: ``` sql SELECT formatRow('CSV', number, 'good') -FROM numbers(3); +FROM numbers(3) ``` Result: @@ -991,7 +934,7 @@ Query: ``` sql SELECT formatRowNoNewline('CSV', number, 'good') -FROM numbers(3); +FROM numbers(3) ``` Result: diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index ee3e8583504..40fdbc6f5a0 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -36,14 +36,10 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438\u0020\u043f\u0440\u0435\u **Пример** -Запрос: - ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); +SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) ``` -Результат: - ``` text ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ @@ -56,14 +52,10 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); **Пример** -Запрос: - ``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123'); +select toInt64OrZero('123123'), toInt8OrZero('123qwe123') ``` -Результат: - ``` text ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ @@ -76,14 +68,10 @@ select toInt64OrZero('123123'), toInt8OrZero('123qwe123'); **Пример** -Запрос: - ``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123'); +select toInt64OrNull('123123'), toInt8OrNull('123qwe123') ``` -Результат: - ``` text ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ @@ -114,14 +102,10 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123'); **Пример** -Запрос: - ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); +SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) ``` -Результат: - ``` text ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ @@ -184,28 +168,20 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); **Примеры** -Запрос: - ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); +SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) ``` -Результат: - ``` text ┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.11100 │ Nullable(Decimal(9, 5)) │ └──────────┴────────────────────────────────────────────────────┘ ``` -Запрос: - ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); +SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) ``` -Результат: - ``` text ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ @@ -235,30 +211,22 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); - Число с `S` десятичными знаками, если ClickHouse распознал число во входной строке. - 0 c `S` десятичными знаками, если ClickHouse не смог распознать число во входной строке или входное число содержит больше чем `S` десятичных знаков. -**Примеры** - -Запрос: +**Пример** ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); +SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val) ``` -Результат: - ``` text ┌──────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ │ -1.11100 │ Decimal(9, 5) │ └──────────┴────────────────────────────────────────────────────┘ ``` -Запрос: - ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); +SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val) ``` -Результат: - ``` text ┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ │ 0.00 │ Decimal(9, 2) │ @@ -290,18 +258,12 @@ YYYY-MM-DD hh:mm:ss Дополнительно, функция toString от аргумента типа DateTime может принимать второй аргумент String - имя тайм-зоны. Пример: `Asia/Yekaterinburg` В этом случае, форматирование времени производится согласно указанной тайм-зоне. -**Пример** - -Запрос: - ``` sql SELECT now() AS now_local, - toString(now(), 'Asia/Yekaterinburg') AS now_yekat; + toString(now(), 'Asia/Yekaterinburg') AS now_yekat ``` -Результат: - ``` text ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ @@ -319,30 +281,22 @@ SELECT Принимает аргумент типа String или FixedString. Возвращает String, вырезая содержимое строки до первого найденного нулевого байта. -**Примеры** - -Запрос: +Пример: ``` sql -SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut; +SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut ``` -Результат: - ``` text ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ ``` -Запрос: - ``` sql -SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; +SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut ``` -Результат: - ``` text ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ @@ -390,7 +344,7 @@ reinterpretAsUUID(fixed_string) Запрос: ``` sql -SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) ``` Результат: @@ -423,11 +377,10 @@ SELECT uuid = uuid2; ## CAST(x, T) {#type_conversion_function-cast} -Преобразует входное значение `x` в тип данных `T`. Поддерживается также синтаксис `CAST(x AS t)`. +Преобразует x в тип данных t. +Поддерживается также синтаксис CAST(x AS t). -**Пример** - -Запрос: +Пример: ``` sql SELECT @@ -435,11 +388,9 @@ SELECT CAST(timestamp AS DateTime) AS datetime, CAST(timestamp AS Date) AS date, CAST(timestamp, 'String') AS string, - CAST(timestamp, 'FixedString(22)') AS fixed_string; + CAST(timestamp, 'FixedString(22)') AS fixed_string ``` -Результат: - ``` text ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ @@ -448,18 +399,12 @@ SELECT Преобразование в FixedString(N) работает только для аргументов типа String или FixedString(N). -Поддерживается преобразование к типу [Nullable](../../sql-reference/functions/type-conversion-functions.md) и обратно. - -**Примеры** - -Запрос: +Поддержано преобразование к типу [Nullable](../../sql-reference/functions/type-conversion-functions.md) и обратно. Пример: ``` sql -SELECT toTypeName(x) FROM t_null; +SELECT toTypeName(x) FROM t_null ``` -Результат: - ``` text ┌─toTypeName(x)─┐ │ Int8 │ @@ -467,14 +412,10 @@ SELECT toTypeName(x) FROM t_null; └───────────────┘ ``` -Запрос: - ``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; +SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null ``` -Результат: - ``` text ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ @@ -501,7 +442,7 @@ accurateCastOrNull(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. -**Примеры** +**Пример** Запрос: @@ -561,8 +502,6 @@ toIntervalYear(number) **Пример** -Запрос: - ``` sql WITH toDate('2019-01-01') AS date, @@ -570,11 +509,9 @@ WITH toIntervalWeek(1) AS interval_to_week SELECT date + interval_week, - date + interval_to_week; + date + interval_to_week ``` -Результат: - ``` text ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ @@ -590,7 +527,7 @@ SELECT **Синтаксис** ``` sql -parseDateTimeBestEffort(time_string[, time_zone]) +parseDateTimeBestEffort(time_string[, time_zone]); ``` **Параметры** @@ -633,7 +570,7 @@ AS parseDateTimeBestEffort; ``` sql SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort; +AS parseDateTimeBestEffort ``` Результат: @@ -648,7 +585,7 @@ AS parseDateTimeBestEffort; ``` sql SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort; +AS parseDateTimeBestEffort ``` Результат: @@ -663,7 +600,7 @@ AS parseDateTimeBestEffort; ``` sql SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort; +AS parseDateTimeBestEffort ``` Результат: @@ -677,7 +614,7 @@ AS parseDateTimeBestEffort; Запрос: ``` sql -SELECT parseDateTimeBestEffort('10 20:19'); +SELECT parseDateTimeBestEffort('10 20:19') ``` Результат: @@ -702,7 +639,7 @@ SELECT parseDateTimeBestEffort('10 20:19'); **Синтаксис** ``` sql -parseDateTimeBestEffortUS(time_string [, time_zone]) +parseDateTimeBestEffortUS(time_string [, time_zone]); ``` **Параметры** @@ -731,7 +668,7 @@ SELECT parseDateTimeBestEffortUS('09/12/2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Результат: +Ответ: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -746,7 +683,7 @@ SELECT parseDateTimeBestEffortUS('09-12-2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Результат: +Ответ: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -761,7 +698,7 @@ SELECT parseDateTimeBestEffortUS('09.12.2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Результат: +Ответ: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -796,10 +733,10 @@ toUnixTimestamp64Milli(value) ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Milli(dt64); +SELECT toUnixTimestamp64Milli(dt64) ``` -Результат: +Ответ: ``` text ┌─toUnixTimestamp64Milli(dt64)─┐ @@ -811,10 +748,10 @@ SELECT toUnixTimestamp64Milli(dt64); ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Nano(dt64); +SELECT toUnixTimestamp64Nano(dt64) ``` -Результат: +Ответ: ``` text ┌─toUnixTimestamp64Nano(dt64)─┐ @@ -849,10 +786,10 @@ fromUnixTimestamp64Milli(value [, ti]) ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC'); +SELECT fromUnixTimestamp64Milli(i64, 'UTC') ``` -Результат: +Ответ: ``` text ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ @@ -883,7 +820,7 @@ toLowCardinality(expr) Тип: `LowCardinality(expr_result_type)` -**Пример** +**Example** Запрос: @@ -924,10 +861,10 @@ formatRow(format, x, y, ...) ``` sql SELECT formatRow('CSV', number, 'good') -FROM numbers(3); +FROM numbers(3) ``` -Результат: +Ответ: ``` text ┌─formatRow('CSV', number, 'good')─┐ @@ -965,10 +902,10 @@ formatRowNoNewline(format, x, y, ...) ``` sql SELECT formatRowNoNewline('CSV', number, 'good') -FROM numbers(3); +FROM numbers(3) ``` -Результат: +Ответ: ``` text ┌─formatRowNoNewline('CSV', number, 'good')─┐ From cc17edbc99ed060f870331f5eb9da93baf5e1e03 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Fri, 5 Feb 2021 13:29:31 +0300 Subject: [PATCH 061/510] DOCSUP-5822: Add function documentation and fix all file examples. --- .../functions/type-conversion-functions.md | 154 +++++++++++++----- .../functions/type-conversion-functions.md | 146 ++++++++++++----- 2 files changed, 220 insertions(+), 80 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 047b3b1cbea..1742f6b8888 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -36,10 +36,14 @@ The behavior of functions for the [NaN and Inf](../../sql-reference/data-types/f **Example** +Query: + ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) +SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); ``` +Result: + ``` text ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ @@ -52,10 +56,14 @@ It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 3 **Example** +Query: + ``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123') +SELECT toInt64OrZero('123123'), toInt8OrZero('123qwe123'); ``` +Result: + ``` text ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ @@ -68,10 +76,14 @@ It takes an argument of type String and tries to parse it into Int (8 \| 16 \| 3 **Example** +Query: + ``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123') +SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); ``` +Result: + ``` text ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ @@ -102,10 +114,14 @@ The behavior of functions for negative agruments and for the [NaN and Inf](../.. **Example** +Query: + ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) +SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); ``` +Result: + ``` text ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ @@ -168,20 +184,28 @@ A value in the `Nullable(Decimal(P,S))` data type. The value contains: **Examples** +Query: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); ``` +Result: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.11100 │ Nullable(Decimal(9, 5)) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Query: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); ``` +Result: + ``` text ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ @@ -213,20 +237,28 @@ A value in the `Nullable(Decimal(P,S))` data type. The value contains: **Example** +Query: + ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); ``` +Result: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ │ -1.11100 │ Decimal(9, 5) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Query: + ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); ``` +Result: + ``` text ┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ │ 0.00 │ Decimal(9, 2) │ @@ -258,12 +290,18 @@ Conversion between numeric types uses the same rules as assignments between diff Additionally, the toString function of the DateTime argument can take a second String argument containing the name of the time zone. Example: `Asia/Yekaterinburg` In this case, the time is formatted according to the specified time zone. +**Example** + +Query: + ``` sql SELECT now() AS now_local, - toString(now(), 'Asia/Yekaterinburg') AS now_yekat + toString(now(), 'Asia/Yekaterinburg') AS now_yekat; ``` +Result: + ``` text ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ @@ -281,22 +319,30 @@ If the string has fewer bytes than N, it is padded with null bytes to the right. Accepts a String or FixedString argument. Returns the String with the content truncated at the first zero byte found. -Example: +**Example** + +Query: ``` sql -SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut +SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut; ``` +Result: + ``` text ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ ``` +Query: + ``` sql -SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut +SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; ``` +Result: + ``` text ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ @@ -348,7 +394,7 @@ String to UUID. Query: ``` sql -SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); ``` Result: @@ -381,9 +427,11 @@ Result: ## CAST(x, T) {#type_conversion_function-cast} -Converts ‘x’ to the ‘t’ data type. The syntax CAST(x AS t) is also supported. +Converts input value `x` to the `T` data type. The syntax `CAST(x AS t)` is also supported. -Example: +**Example** + +Query: ``` sql SELECT @@ -391,9 +439,11 @@ SELECT CAST(timestamp AS DateTime) AS datetime, CAST(timestamp AS Date) AS date, CAST(timestamp, 'String') AS string, - CAST(timestamp, 'FixedString(22)') AS fixed_string + CAST(timestamp, 'FixedString(22)') AS fixed_string; ``` +Result: + ``` text ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ @@ -402,12 +452,18 @@ SELECT Conversion to FixedString(N) only works for arguments of type String or FixedString(N). -Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported. Example: +Type conversion to [Nullable](../../sql-reference/data-types/nullable.md) and back is supported. + +**Example** + +Query: ``` sql -SELECT toTypeName(x) FROM t_null +SELECT toTypeName(x) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(x)─┐ │ Int8 │ @@ -415,10 +471,14 @@ SELECT toTypeName(x) FROM t_null └───────────────┘ ``` +Query: + ``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null +SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; ``` +Result: + ``` text ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ @@ -432,15 +492,18 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null ## accurateCast(x, T) {#type_conversion_function-accurate-cast} -Converts ‘x’ to the ‘t’ data type. The differente from cast(x, T) is that accurateCast -does not allow overflow of numeric types during cast if type value x does not fit -bounds of type T. +Converts `x` to the `T` data type. The differente from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` +does not allow overflow of numeric types during cast if type value `x` does not fit bounds of type `T`. + +**Example** + +Query: -Example ``` sql SELECT cast(-1, 'UInt8') as uint8; ``` +Result: ``` text ┌─uint8─┐ @@ -448,13 +511,16 @@ SELECT cast(-1, 'UInt8') as uint8; └───────┘ ``` +Query: + ```sql SELECT accurateCast(-1, 'UInt8') as uint8; ``` +Result: + ``` text Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in column Int8 cannot be safely converted into type UInt8: While processing accurateCast(-1, 'UInt8') AS uint8. - ``` ## accurateCastOrNull(x, T) {#type_conversion_function-accurate-cast_or_null} @@ -488,6 +554,8 @@ Result: └────────────────────────────────────────────┘ ``` +Query: + ``` sql SELECT cast(-1, 'UInt8') as uint8, @@ -530,6 +598,8 @@ toIntervalYear(number) **Example** +Query: + ``` sql WITH toDate('2019-01-01') AS date, @@ -537,9 +607,11 @@ WITH toIntervalWeek(1) AS interval_to_week SELECT date + interval_week, - date + interval_to_week + date + interval_to_week; ``` +Result: + ``` text ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ @@ -555,7 +627,7 @@ The function parses [ISO 8601](https://en.wikipedia.org/wiki/ISO_8601), [RFC 112 **Syntax** ``` sql -parseDateTimeBestEffort(time_string [, time_zone]); +parseDateTimeBestEffort(time_string [, time_zone]) ``` **Parameters** @@ -598,7 +670,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Result: @@ -613,7 +685,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Result: @@ -628,7 +700,7 @@ Query: ``` sql SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Result: @@ -642,7 +714,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffort('10 20:19') +SELECT parseDateTimeBestEffort('10 20:19'); ``` Result: @@ -662,12 +734,12 @@ Result: ## parseDateTimeBestEffortUS {#parsedatetimebesteffortUS} -This function is similar to [‘parseDateTimeBestEffort’](#parsedatetimebesteffort), the only difference is that this function prefers US date format (`MM/DD/YYYY` etc.) in case of ambiguity. +This function is similar to [parseDateTimeBestEffort](#parsedatetimebesteffort), the only difference is that this function prefers US date format (`MM/DD/YYYY` etc.) in case of ambiguity. **Syntax** ``` sql -parseDateTimeBestEffortUS(time_string [, time_zone]); +parseDateTimeBestEffortUS(time_string [, time_zone]) ``` **Parameters** @@ -769,7 +841,7 @@ Type: `LowCardinality(expr_result_type)` Query: ``` sql -SELECT toLowCardinality('1') +SELECT toLowCardinality('1'); ``` Result: @@ -808,7 +880,7 @@ Query: ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Milli(dt64) +SELECT toUnixTimestamp64Milli(dt64); ``` Result: @@ -821,7 +893,7 @@ Result: ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Nano(dt64) +SELECT toUnixTimestamp64Nano(dt64); ``` Result: @@ -855,13 +927,17 @@ fromUnixTimestamp64Milli(value [, ti]) - `value` converted to the `DateTime64` data type. -**Examples** +**Example** + +Query: ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC') +SELECT fromUnixTimestamp64Milli(i64, 'UTC'); ``` +Result: + ``` text ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ │ 2009-02-13 23:31:31.011 │ @@ -893,7 +969,7 @@ Query: ``` sql SELECT formatRow('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` Result: @@ -934,7 +1010,7 @@ Query: ``` sql SELECT formatRowNoNewline('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` Result: diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 40fdbc6f5a0..aa55e015c61 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -36,10 +36,14 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438\u0020\u043f\u0440\u0435\u **Пример** +Запрос: + ``` sql -SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) +SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8); ``` +Результат: + ``` text ┌─────────toInt64(nan)─┬─toInt32(32)─┬─toInt16('16')─┬─toInt8(8.8)─┐ │ -9223372036854775808 │ 32 │ 16 │ 8 │ @@ -52,10 +56,14 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8) **Пример** +Запрос: + ``` sql -select toInt64OrZero('123123'), toInt8OrZero('123qwe123') +SELECT toInt64OrZero('123123'), toInt8OrZero('123qwe123'); ``` +Результат: + ``` text ┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐ │ 123123 │ 0 │ @@ -68,10 +76,14 @@ select toInt64OrZero('123123'), toInt8OrZero('123qwe123') **Пример** +Запрос: + ``` sql -select toInt64OrNull('123123'), toInt8OrNull('123qwe123') +SELECT toInt64OrNull('123123'), toInt8OrNull('123qwe123'); ``` +Результат: + ``` text ┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐ │ 123123 │ ᴺᵁᴸᴸ │ @@ -102,10 +114,14 @@ select toInt64OrNull('123123'), toInt8OrNull('123qwe123') **Пример** +Запрос: + ``` sql -SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) +SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8); ``` +Результат: + ``` text ┌───────toUInt64(nan)─┬─toUInt32(-32)─┬─toUInt16('16')─┬─toUInt8(8.8)─┐ │ 9223372036854775808 │ 4294967264 │ 16 │ 8 │ @@ -168,20 +184,28 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) **Примеры** +Запрос: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 5) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 5))─┐ │ -1.11100 │ Nullable(Decimal(9, 5)) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Запрос: + ``` sql -SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──val─┬─toTypeName(toDecimal32OrNull(toString(-1.111), 2))─┐ │ ᴺᵁᴸᴸ │ Nullable(Decimal(9, 2)) │ @@ -213,20 +237,28 @@ SELECT toDecimal32OrNull(toString(-1.111), 2) AS val, toTypeName(val) **Пример** +Запрос: + ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val) +SELECT toDecimal32OrZero(toString(-1.111), 5) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──────val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 5))─┐ │ -1.11100 │ Decimal(9, 5) │ └──────────┴────────────────────────────────────────────────────┘ ``` +Запрос: + ``` sql -SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val) +SELECT toDecimal32OrZero(toString(-1.111), 2) AS val, toTypeName(val); ``` +Результат: + ``` text ┌──val─┬─toTypeName(toDecimal32OrZero(toString(-1.111), 2))─┐ │ 0.00 │ Decimal(9, 2) │ @@ -258,12 +290,18 @@ YYYY-MM-DD hh:mm:ss Дополнительно, функция toString от аргумента типа DateTime может принимать второй аргумент String - имя тайм-зоны. Пример: `Asia/Yekaterinburg` В этом случае, форматирование времени производится согласно указанной тайм-зоне. +**Пример** + +Запрос: + ``` sql SELECT now() AS now_local, - toString(now(), 'Asia/Yekaterinburg') AS now_yekat + toString(now(), 'Asia/Yekaterinburg') AS now_yekat; ``` +Результат: + ``` text ┌───────────now_local─┬─now_yekat───────────┐ │ 2016-06-15 00:11:21 │ 2016-06-15 02:11:21 │ @@ -281,22 +319,30 @@ SELECT Принимает аргумент типа String или FixedString. Возвращает String, вырезая содержимое строки до первого найденного нулевого байта. -Пример: +**Примеры** + +Запрос: ``` sql -SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut +SELECT toFixedString('foo', 8) AS s, toStringCutToZero(s) AS s_cut; ``` +Результат: + ``` text ┌─s─────────────┬─s_cut─┐ │ foo\0\0\0\0\0 │ foo │ └───────────────┴───────┘ ``` +Запрос: + ``` sql -SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut +SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; ``` +Результат: + ``` text ┌─s──────────┬─s_cut─┐ │ foo\0bar\0 │ foo │ @@ -344,7 +390,7 @@ reinterpretAsUUID(fixed_string) Запрос: ``` sql -SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))) +SELECT reinterpretAsUUID(reverse(unhex('000102030405060708090a0b0c0d0e0f'))); ``` Результат: @@ -380,7 +426,9 @@ SELECT uuid = uuid2; Преобразует x в тип данных t. Поддерживается также синтаксис CAST(x AS t). -Пример: +**Пример** + +Запрос: ``` sql SELECT @@ -388,9 +436,11 @@ SELECT CAST(timestamp AS DateTime) AS datetime, CAST(timestamp AS Date) AS date, CAST(timestamp, 'String') AS string, - CAST(timestamp, 'FixedString(22)') AS fixed_string + CAST(timestamp, 'FixedString(22)') AS fixed_string; ``` +Результат: + ``` text ┌─timestamp───────────┬────────────datetime─┬───────date─┬─string──────────────┬─fixed_string──────────────┐ │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00 │ 2016-06-15 │ 2016-06-15 23:00:00 │ 2016-06-15 23:00:00\0\0\0 │ @@ -399,12 +449,18 @@ SELECT Преобразование в FixedString(N) работает только для аргументов типа String или FixedString(N). -Поддержано преобразование к типу [Nullable](../../sql-reference/functions/type-conversion-functions.md) и обратно. Пример: +Поддержано преобразование к типу [Nullable](../../sql-reference/functions/type-conversion-functions.md) и обратно. + +**Примеры** + +Запрос: ``` sql -SELECT toTypeName(x) FROM t_null +SELECT toTypeName(x) FROM t_null; ``` +Результат: + ``` text ┌─toTypeName(x)─┐ │ Int8 │ @@ -412,10 +468,14 @@ SELECT toTypeName(x) FROM t_null └───────────────┘ ``` +Запрос: + ``` sql -SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null +SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; ``` +Результат: + ``` text ┌─toTypeName(CAST(x, 'Nullable(UInt16)'))─┐ │ Nullable(UInt16) │ @@ -442,7 +502,7 @@ accurateCastOrNull(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. -**Пример** +**Примеры** Запрос: @@ -502,6 +562,8 @@ toIntervalYear(number) **Пример** +Запрос: + ``` sql WITH toDate('2019-01-01') AS date, @@ -509,9 +571,11 @@ WITH toIntervalWeek(1) AS interval_to_week SELECT date + interval_week, - date + interval_to_week + date + interval_to_week; ``` +Результат: + ``` text ┌─plus(date, interval_week)─┬─plus(date, interval_to_week)─┐ │ 2019-01-08 │ 2019-01-08 │ @@ -527,7 +591,7 @@ SELECT **Синтаксис** ``` sql -parseDateTimeBestEffort(time_string[, time_zone]); +parseDateTimeBestEffort(time_string[, time_zone]) ``` **Параметры** @@ -570,7 +634,7 @@ AS parseDateTimeBestEffort; ``` sql SELECT parseDateTimeBestEffort('Sat, 18 Aug 2018 07:22:16 GMT', 'Europe/Moscow') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Результат: @@ -585,7 +649,7 @@ AS parseDateTimeBestEffort ``` sql SELECT parseDateTimeBestEffort('1284101485') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Результат: @@ -600,7 +664,7 @@ AS parseDateTimeBestEffort ``` sql SELECT parseDateTimeBestEffort('2018-12-12 10:12:12') -AS parseDateTimeBestEffort +AS parseDateTimeBestEffort; ``` Результат: @@ -614,7 +678,7 @@ AS parseDateTimeBestEffort Запрос: ``` sql -SELECT parseDateTimeBestEffort('10 20:19') +SELECT parseDateTimeBestEffort('10 20:19'); ``` Результат: @@ -639,7 +703,7 @@ SELECT parseDateTimeBestEffort('10 20:19') **Синтаксис** ``` sql -parseDateTimeBestEffortUS(time_string [, time_zone]); +parseDateTimeBestEffortUS(time_string [, time_zone]) ``` **Параметры** @@ -668,7 +732,7 @@ SELECT parseDateTimeBestEffortUS('09/12/2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Ответ: +Результат: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -683,7 +747,7 @@ SELECT parseDateTimeBestEffortUS('09-12-2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Ответ: +Результат: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -698,7 +762,7 @@ SELECT parseDateTimeBestEffortUS('09.12.2020 12:12:57') AS parseDateTimeBestEffortUS; ``` -Ответ: +Результат: ``` text ┌─parseDateTimeBestEffortUS─┐ @@ -733,10 +797,10 @@ toUnixTimestamp64Milli(value) ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Milli(dt64) +SELECT toUnixTimestamp64Milli(dt64); ``` -Ответ: +Результат: ``` text ┌─toUnixTimestamp64Milli(dt64)─┐ @@ -748,10 +812,10 @@ SELECT toUnixTimestamp64Milli(dt64) ``` sql WITH toDateTime64('2019-09-16 19:20:12.345678910', 6) AS dt64 -SELECT toUnixTimestamp64Nano(dt64) +SELECT toUnixTimestamp64Nano(dt64); ``` -Ответ: +Результат: ``` text ┌─toUnixTimestamp64Nano(dt64)─┐ @@ -786,10 +850,10 @@ fromUnixTimestamp64Milli(value [, ti]) ``` sql WITH CAST(1234567891011, 'Int64') AS i64 -SELECT fromUnixTimestamp64Milli(i64, 'UTC') +SELECT fromUnixTimestamp64Milli(i64, 'UTC'); ``` -Ответ: +Результат: ``` text ┌─fromUnixTimestamp64Milli(i64, 'UTC')─┐ @@ -820,12 +884,12 @@ toLowCardinality(expr) Тип: `LowCardinality(expr_result_type)` -**Example** +**Пример** Запрос: ```sql -SELECT toLowCardinality('1') +SELECT toLowCardinality('1'); ``` Результат: @@ -861,10 +925,10 @@ formatRow(format, x, y, ...) ``` sql SELECT formatRow('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` -Ответ: +Результат: ``` text ┌─formatRow('CSV', number, 'good')─┐ @@ -902,10 +966,10 @@ formatRowNoNewline(format, x, y, ...) ``` sql SELECT formatRowNoNewline('CSV', number, 'good') -FROM numbers(3) +FROM numbers(3); ``` -Ответ: +Результат: ``` text ┌─formatRowNoNewline('CSV', number, 'good')─┐ From 5472eb5bd99aa712821a30b4e6aa1a73dfb6d40b Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Fri, 5 Feb 2021 10:39:58 +0000 Subject: [PATCH 062/510] Allow to drop readonly tables This check doesn't seem to be necessary. There seem to be a deadlock due to a logical race of drop with restarting thread. Seen in https://clickhouse-test-reports.s3.yandex.net/20088/4ebb44bb9936ed1daa330cb38f343664ca83751c/integration_tests_flaky_check_(asan).html#fail1 --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 69cbe0d7062..cb5f4dd5185 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -751,7 +751,7 @@ void StorageReplicatedMergeTree::drop() auto zookeeper = global_context.getZooKeeper(); /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. - if (is_readonly || !zookeeper) + if (!zookeeper) throw Exception("Can't drop readonly replicated table (need to drop data in ZooKeeper as well)", ErrorCodes::TABLE_IS_READ_ONLY); shutdown(); From e051423584855ef75bbe7d41d1b6db8a649f7bee Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Fri, 5 Feb 2021 22:14:52 +0300 Subject: [PATCH 063/510] add RU docs --- .../example-datasets/brown-benchmark.md | 416 ++++++++++++++++++ .../functions/array-functions.md | 150 +++++++ .../ru/sql-reference/table-functions/mysql.md | 63 ++- 3 files changed, 605 insertions(+), 24 deletions(-) create mode 100644 docs/ru/getting-started/example-datasets/brown-benchmark.md diff --git a/docs/ru/getting-started/example-datasets/brown-benchmark.md b/docs/ru/getting-started/example-datasets/brown-benchmark.md new file mode 100644 index 00000000000..b3f2285093a --- /dev/null +++ b/docs/ru/getting-started/example-datasets/brown-benchmark.md @@ -0,0 +1,416 @@ +--- +toc_priority: 20 +toc_title: Brown University Benchmark +--- + +# Brown University Benchmark + +`MgBench` — это новый аналитический бенчмарк для сгенерированного журнала событий, разработанный [Andrew Crotty](http://cs.brown.edu/people/acrotty/). + +Скачать данные: +``` +wget https://datasets.clickhouse.tech/mgbench{1..3}.csv.xz +``` + +Распаковать данные: +``` +xz -v -d mgbench{1..3}.csv.xz +``` + +Создание таблиц: +``` +CREATE DATABASE mgbench; + + +CREATE TABLE mgbench.logs1 ( + log_time DateTime, + machine_name LowCardinality(String), + machine_group LowCardinality(String), + cpu_idle Nullable(Float32), + cpu_nice Nullable(Float32), + cpu_system Nullable(Float32), + cpu_user Nullable(Float32), + cpu_wio Nullable(Float32), + disk_free Nullable(Float32), + disk_total Nullable(Float32), + part_max_used Nullable(Float32), + load_fifteen Nullable(Float32), + load_five Nullable(Float32), + load_one Nullable(Float32), + mem_buffers Nullable(Float32), + mem_cached Nullable(Float32), + mem_free Nullable(Float32), + mem_shared Nullable(Float32), + swap_free Nullable(Float32), + bytes_in Nullable(Float32), + bytes_out Nullable(Float32) +) +ENGINE = MergeTree() +ORDER BY (machine_group, machine_name, log_time); + + +CREATE TABLE mgbench.logs2 ( + log_time DateTime, + client_ip IPv4, + request String, + status_code UInt16, + object_size UInt64 +) +ENGINE = MergeTree() +ORDER BY log_time; + + +CREATE TABLE mgbench.logs3 ( + log_time DateTime64, + device_id FixedString(15), + device_name LowCardinality(String), + device_type LowCardinality(String), + device_floor UInt8, + event_type LowCardinality(String), + event_unit FixedString(1), + event_value Nullable(Float32) +) +ENGINE = MergeTree() +ORDER BY (event_type, log_time); +``` + +Insert data: + +``` +clickhouse-client --query "INSERT INTO mgbench.logs1 FORMAT CSVWithNames" < mgbench1.csv +clickhouse-client --query "INSERT INTO mgbench.logs2 FORMAT CSVWithNames" < mgbench2.csv +clickhouse-client --query "INSERT INTO mgbench.logs3 FORMAT CSVWithNames" < mgbench3.csv +``` + +Run benchmark queries: +``` +-- Q1.1: What is the CPU/network utilization for each web server since midnight? + +SELECT machine_name, + MIN(cpu) AS cpu_min, + MAX(cpu) AS cpu_max, + AVG(cpu) AS cpu_avg, + MIN(net_in) AS net_in_min, + MAX(net_in) AS net_in_max, + AVG(net_in) AS net_in_avg, + MIN(net_out) AS net_out_min, + MAX(net_out) AS net_out_max, + AVG(net_out) AS net_out_avg +FROM ( + SELECT machine_name, + COALESCE(cpu_user, 0.0) AS cpu, + COALESCE(bytes_in, 0.0) AS net_in, + COALESCE(bytes_out, 0.0) AS net_out + FROM logs1 + WHERE machine_name IN ('anansi','aragog','urd') + AND log_time >= TIMESTAMP '2017-01-11 00:00:00' +) AS r +GROUP BY machine_name; + + +-- Q1.2: Which computer lab machines have been offline in the past day? + +SELECT machine_name, + log_time +FROM logs1 +WHERE (machine_name LIKE 'cslab%' OR + machine_name LIKE 'mslab%') + AND load_one IS NULL + AND log_time >= TIMESTAMP '2017-01-10 00:00:00' +ORDER BY machine_name, + log_time; + + +-- Q1.3: What are the hourly average metrics during the past 10 days for a specific workstation? + +SELECT dt, + hr, + AVG(load_fifteen) AS load_fifteen_avg, + AVG(load_five) AS load_five_avg, + AVG(load_one) AS load_one_avg, + AVG(mem_free) AS mem_free_avg, + AVG(swap_free) AS swap_free_avg +FROM ( + SELECT CAST(log_time AS DATE) AS dt, + EXTRACT(HOUR FROM log_time) AS hr, + load_fifteen, + load_five, + load_one, + mem_free, + swap_free + FROM logs1 + WHERE machine_name = 'babbage' + AND load_fifteen IS NOT NULL + AND load_five IS NOT NULL + AND load_one IS NOT NULL + AND mem_free IS NOT NULL + AND swap_free IS NOT NULL + AND log_time >= TIMESTAMP '2017-01-01 00:00:00' +) AS r +GROUP BY dt, + hr +ORDER BY dt, + hr; + + +-- Q1.4: Over 1 month, how often was each server blocked on disk I/O? + +SELECT machine_name, + COUNT(*) AS spikes +FROM logs1 +WHERE machine_group = 'Servers' + AND cpu_wio > 0.99 + AND log_time >= TIMESTAMP '2016-12-01 00:00:00' + AND log_time < TIMESTAMP '2017-01-01 00:00:00' +GROUP BY machine_name +ORDER BY spikes DESC +LIMIT 10; + + +-- Q1.5: Which externally reachable VMs have run low on memory? + +SELECT machine_name, + dt, + MIN(mem_free) AS mem_free_min +FROM ( + SELECT machine_name, + CAST(log_time AS DATE) AS dt, + mem_free + FROM logs1 + WHERE machine_group = 'DMZ' + AND mem_free IS NOT NULL +) AS r +GROUP BY machine_name, + dt +HAVING MIN(mem_free) < 10000 +ORDER BY machine_name, + dt; + + +-- Q1.6: What is the total hourly network traffic across all file servers? + +SELECT dt, + hr, + SUM(net_in) AS net_in_sum, + SUM(net_out) AS net_out_sum, + SUM(net_in) + SUM(net_out) AS both_sum +FROM ( + SELECT CAST(log_time AS DATE) AS dt, + EXTRACT(HOUR FROM log_time) AS hr, + COALESCE(bytes_in, 0.0) / 1000000000.0 AS net_in, + COALESCE(bytes_out, 0.0) / 1000000000.0 AS net_out + FROM logs1 + WHERE machine_name IN ('allsorts','andes','bigred','blackjack','bonbon', + 'cadbury','chiclets','cotton','crows','dove','fireball','hearts','huey', + 'lindt','milkduds','milkyway','mnm','necco','nerds','orbit','peeps', + 'poprocks','razzles','runts','smarties','smuggler','spree','stride', + 'tootsie','trident','wrigley','york') +) AS r +GROUP BY dt, + hr +ORDER BY both_sum DESC +LIMIT 10; + + +-- Q2.1: Which requests have caused server errors within the past 2 weeks? + +SELECT * +FROM logs2 +WHERE status_code >= 500 + AND log_time >= TIMESTAMP '2012-12-18 00:00:00' +ORDER BY log_time; + + +-- Q2.2: During a specific 2-week period, was the user password file leaked? + +SELECT * +FROM logs2 +WHERE status_code >= 200 + AND status_code < 300 + AND request LIKE '%/etc/passwd%' + AND log_time >= TIMESTAMP '2012-05-06 00:00:00' + AND log_time < TIMESTAMP '2012-05-20 00:00:00'; + + +-- Q2.3: What was the average path depth for top-level requests in the past month? + +SELECT top_level, + AVG(LENGTH(request) - LENGTH(REPLACE(request, '/', ''))) AS depth_avg +FROM ( + SELECT SUBSTRING(request FROM 1 FOR len) AS top_level, + request + FROM ( + SELECT POSITION(SUBSTRING(request FROM 2), '/') AS len, + request + FROM logs2 + WHERE status_code >= 200 + AND status_code < 300 + AND log_time >= TIMESTAMP '2012-12-01 00:00:00' + ) AS r + WHERE len > 0 +) AS s +WHERE top_level IN ('/about','/courses','/degrees','/events', + '/grad','/industry','/news','/people', + '/publications','/research','/teaching','/ugrad') +GROUP BY top_level +ORDER BY top_level; + + +-- Q2.4: During the last 3 months, which clients have made an excessive number of requests? + +SELECT client_ip, + COUNT(*) AS num_requests +FROM logs2 +WHERE log_time >= TIMESTAMP '2012-10-01 00:00:00' +GROUP BY client_ip +HAVING COUNT(*) >= 100000 +ORDER BY num_requests DESC; + + +-- Q2.5: What are the daily unique visitors? + +SELECT dt, + COUNT(DISTINCT client_ip) +FROM ( + SELECT CAST(log_time AS DATE) AS dt, + client_ip + FROM logs2 +) AS r +GROUP BY dt +ORDER BY dt; + + +-- Q2.6: What are the average and maximum data transfer rates (Gbps)? + +SELECT AVG(transfer) / 125000000.0 AS transfer_avg, + MAX(transfer) / 125000000.0 AS transfer_max +FROM ( + SELECT log_time, + SUM(object_size) AS transfer + FROM logs2 + GROUP BY log_time +) AS r; + + +-- Q3.1: Did the indoor temperature reach freezing over the weekend? + +SELECT * +FROM logs3 +WHERE event_type = 'temperature' + AND event_value <= 32.0 + AND log_time >= '2019-11-29 17:00:00.000'; + + +-- Q3.4: Over the past 6 months, how frequently were each door opened? + +SELECT device_name, + device_floor, + COUNT(*) AS ct +FROM logs3 +WHERE event_type = 'door_open' + AND log_time >= '2019-06-01 00:00:00.000' +GROUP BY device_name, + device_floor +ORDER BY ct DESC; + + +-- Q3.5: Where in the building do large temperature variations occur in winter and summer? + +WITH temperature AS ( + SELECT dt, + device_name, + device_type, + device_floor + FROM ( + SELECT dt, + hr, + device_name, + device_type, + device_floor, + AVG(event_value) AS temperature_hourly_avg + FROM ( + SELECT CAST(log_time AS DATE) AS dt, + EXTRACT(HOUR FROM log_time) AS hr, + device_name, + device_type, + device_floor, + event_value + FROM logs3 + WHERE event_type = 'temperature' + ) AS r + GROUP BY dt, + hr, + device_name, + device_type, + device_floor + ) AS s + GROUP BY dt, + device_name, + device_type, + device_floor + HAVING MAX(temperature_hourly_avg) - MIN(temperature_hourly_avg) >= 25.0 +) +SELECT DISTINCT device_name, + device_type, + device_floor, + 'WINTER' +FROM temperature +WHERE dt >= DATE '2018-12-01' + AND dt < DATE '2019-03-01' +UNION +SELECT DISTINCT device_name, + device_type, + device_floor, + 'SUMMER' +FROM temperature +WHERE dt >= DATE '2019-06-01' + AND dt < DATE '2019-09-01'; + + +-- Q3.6: For each device category, what are the monthly power consumption metrics? + +SELECT yr, + mo, + SUM(coffee_hourly_avg) AS coffee_monthly_sum, + AVG(coffee_hourly_avg) AS coffee_monthly_avg, + SUM(printer_hourly_avg) AS printer_monthly_sum, + AVG(printer_hourly_avg) AS printer_monthly_avg, + SUM(projector_hourly_avg) AS projector_monthly_sum, + AVG(projector_hourly_avg) AS projector_monthly_avg, + SUM(vending_hourly_avg) AS vending_monthly_sum, + AVG(vending_hourly_avg) AS vending_monthly_avg +FROM ( + SELECT dt, + yr, + mo, + hr, + AVG(coffee) AS coffee_hourly_avg, + AVG(printer) AS printer_hourly_avg, + AVG(projector) AS projector_hourly_avg, + AVG(vending) AS vending_hourly_avg + FROM ( + SELECT CAST(log_time AS DATE) AS dt, + EXTRACT(YEAR FROM log_time) AS yr, + EXTRACT(MONTH FROM log_time) AS mo, + EXTRACT(HOUR FROM log_time) AS hr, + CASE WHEN device_name LIKE 'coffee%' THEN event_value END AS coffee, + CASE WHEN device_name LIKE 'printer%' THEN event_value END AS printer, + CASE WHEN device_name LIKE 'projector%' THEN event_value END AS projector, + CASE WHEN device_name LIKE 'vending%' THEN event_value END AS vending + FROM logs3 + WHERE device_type = 'meter' + ) AS r + GROUP BY dt, + yr, + mo, + hr +) AS s +GROUP BY yr, + mo +ORDER BY yr, + mo; +``` + +Данные также доступны для работы с интерактивными запросами через [Playground](https://gh-api.clickhouse.tech/play?user=play), [пример](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1hY2hpbmVfbmFtZSwKICAgICAgIE1JTihjcHUpIEFTIGNwdV9taW4sCiAgICAgICBNQVgoY3B1KSBBUyBjcHVfbWF4LAogICAgICAgQVZHKGNwdSkgQVMgY3B1X2F2ZywKICAgICAgIE1JTihuZXRfaW4pIEFTIG5ldF9pbl9taW4sCiAgICAgICBNQVgobmV0X2luKSBBUyBuZXRfaW5fbWF4LAogICAgICAgQVZHKG5ldF9pbikgQVMgbmV0X2luX2F2ZywKICAgICAgIE1JTihuZXRfb3V0KSBBUyBuZXRfb3V0X21pbiwKICAgICAgIE1BWChuZXRfb3V0KSBBUyBuZXRfb3V0X21heCwKICAgICAgIEFWRyhuZXRfb3V0KSBBUyBuZXRfb3V0X2F2ZwpGUk9NICgKICBTRUxFQ1QgbWFjaGluZV9uYW1lLAogICAgICAgICBDT0FMRVNDRShjcHVfdXNlciwgMC4wKSBBUyBjcHUsCiAgICAgICAgIENPQUxFU0NFKGJ5dGVzX2luLCAwLjApIEFTIG5ldF9pbiwKICAgICAgICAgQ09BTEVTQ0UoYnl0ZXNfb3V0LCAwLjApIEFTIG5ldF9vdXQKICBGUk9NIG1nYmVuY2gubG9nczEKICBXSEVSRSBtYWNoaW5lX25hbWUgSU4gKCdhbmFuc2knLCdhcmFnb2cnLCd1cmQnKQogICAgQU5EIGxvZ190aW1lID49IFRJTUVTVEFNUCAnMjAxNy0wMS0xMSAwMDowMDowMCcKKSBBUyByCkdST1VQIEJZIG1hY2hpbmVfbmFtZQ==). + +[Оригинальная статья](https://clickhouse.tech/docs/ru/getting_started/example_datasets/brown-benchmark/) diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 015d14b9de5..7afd9da471e 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -1135,12 +1135,162 @@ SELECT Функция `arrayFirstIndex` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию, и этот аргумент не может быть опущен. +## arrayMin(\[func,\] arr1, …) {#array-min} + +Возвращает минимальное значение функции `func`. Если функция не указана, возвращает минимальный из элементов массива. + +Функция `arrayMin` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию. + +**Синтаксис** + +``` sql +arrayMin(arr) +``` + +**Возвращаемое значение** + +- Число. + +Тип: [Int](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). + +**Параметры** + +- `arr` — [Массив](../../sql-reference/data-types/array.md). + +**Примеры** + +Запрос: + +``` sql +SELECT arrayMin([1, 2, 4]) AS res +``` + +Результат: + +``` text +┌─res─┐ +│ 1 │ +└─────┘ +``` + +Запрос: + +``` sql +SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res +``` + +Результат: + +``` text +┌─res─┐ +│ -4 │ +└─────┘ +``` + +## arrayMax(\[func,\] arr1, …) {#array-max} + +Возвращает максимальное значение функции `func`. Если функция не указана, возвращает максимальный из элементов массива. + +Функция `arrayMax` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) — в качестве первого аргумента ей нужно передать лямбда-функцию. + +**Синтаксис** + +``` sql +arrayMax(arr) +``` + +**Возвращаемое значение** + +- Число. + +Тип: [Int](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). + +**Параметры** + +- `arr` — [Массив](../../sql-reference/data-types/array.md). + +**Примеры** + +Запрос: + +```sql +SELECT arrayMax([1, 2, 4]) AS res +``` + +Результат: + +``` text +┌─res─┐ +│ 4 │ +└─────┘ +``` + +Запрос: + +``` sql +SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res +``` + +Результат: + +``` text +┌─res─┐ +│ -1 │ +└─────┘ +``` + ## arraySum(\[func,\] arr1, …) {#array-sum} Возвращает сумму значений функции `func`. Если функция не указана - просто возвращает сумму элементов массива. Функция `arraySum` является [функцией высшего порядка](../../sql-reference/functions/index.md#higher-order-functions) - в качестве первого аргумента ей можно передать лямбда-функцию. +**Синтаксис** + +``` sql +arraySum(arr) +``` + +**Возвращаемое значение** + +- Число. + +Тип: [Int](../../sql-reference/data-types/int-uint.md) или [Float](../../sql-reference/data-types/float.md). + +**Параметры** + +- `arr` — [Массив](../../sql-reference/data-types/array.md). + +**Примеры** + +Запрос: + +```sql +SELECT arraySum([2,3]) AS res +``` + +Результат: + +``` text +┌─res─┐ +│ 5 │ +└─────┘ +``` + +Запрос: + +``` sql +SELECT arraySum(x -> x*x, [2, 3]) AS res +``` + +Результат: + +``` text +┌─res─┐ +│ 13 │ +└─────┘ +``` + ## arrayCumSum(\[func,\] arr1, …) {#arraycumsumfunc-arr1} Возвращает массив из частичных сумм элементов исходного массива (сумма с накоплением). Если указана функция `func`, то значения элементов массива преобразуются этой функцией перед суммированием. diff --git a/docs/ru/sql-reference/table-functions/mysql.md b/docs/ru/sql-reference/table-functions/mysql.md index 21841eee67a..18b34d0bf6c 100644 --- a/docs/ru/sql-reference/table-functions/mysql.md +++ b/docs/ru/sql-reference/table-functions/mysql.md @@ -7,6 +7,8 @@ toc_title: mysql Позволяет выполнять запросы `SELECT` над данными, хранящимися на удалённом MySQL сервере. +**Синтаксис** + ``` sql mysql('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_duplicate_clause']); ``` @@ -23,13 +25,13 @@ mysql('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_ - `password` — пароль пользователя. -- `replace_query` — флаг, отвечающий за преобразование запросов `INSERT INTO` в `REPLACE INTO`. Если `replace_query=1`, то запрос заменяется. +- `replace_query` — флаг, отвечающий за преобразование запросов `INSERT INTO` в `REPLACE INTO`. Возможные значения: + - `0` - выполняется запрос `INSERT INTO`. + - `1` - выполняется запрос `REPLACE INTO`. -- `on_duplicate_clause` — выражение `ON DUPLICATE KEY on_duplicate_clause`, добавляемое в запрос `INSERT`. +- `on_duplicate_clause` — выражение `ON DUPLICATE KEY on_duplicate_clause`, добавляемое в запрос `INSERT`. Может быть передано только с помощью `replace_query = 0` (если вы одновременно передадите `replace_query = 1` и `on_duplicate_clause`, будет сгенерировано исключение). - Пример: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, где `on_duplicate_clause` это `UPDATE c2 = c2 + 1`. Чтобы узнать какие `on_duplicate_clause` можно использовать с секцией `ON DUPLICATE KEY` обратитесь к документации MySQL. - - Чтобы указать `'on_duplicate_clause'` необходимо передать `0` в параметр `replace_query`. Если одновременно передать `replace_query = 1` и `'on_duplicate_clause'`, то ClickHouse сгенерирует исключение. + Пример: `INSERT INTO t (c1,c2) VALUES ('a', 2) ON DUPLICATE KEY UPDATE c2 = c2 + 1`, где `on_duplicate_clause` это `UPDATE c2 = c2 + 1;` Простые условия `WHERE` такие как `=, !=, >, >=, <, =` выполняются на стороне сервера MySQL. @@ -39,46 +41,59 @@ mysql('host:port', 'database', 'table', 'user', 'password'[, replace_query, 'on_ Объект таблицы с теми же столбцами, что и в исходной таблице MySQL. -## Пример использования {#primer-ispolzovaniia} +!!! note "Примечание" + Чтобы отличить табличную функцию `mysql (...)` в запросе `INSERT` от имени таблицы со списком имен столбцов, используйте ключевые слова `FUNCTION` или `TABLE FUNCTION`. См. примеры ниже. + +**Примеры** Таблица в MySQL: ``` text mysql> CREATE TABLE `test`.`test` ( -> `int_id` INT NOT NULL AUTO_INCREMENT, - -> `int_nullable` INT NULL DEFAULT NULL, -> `float` FLOAT NOT NULL, - -> `float_nullable` FLOAT NULL DEFAULT NULL, -> PRIMARY KEY (`int_id`)); -Query OK, 0 rows affected (0,09 sec) -mysql> insert into test (`int_id`, `float`) VALUES (1,2); -Query OK, 1 row affected (0,00 sec) +mysql> INSERT INTO test (`int_id`, `float`) VALUES (1,2); -mysql> select * from test; -+--------+--------------+-------+----------------+ -| int_id | int_nullable | float | float_nullable | -+--------+--------------+-------+----------------+ -| 1 | NULL | 2 | NULL | -+--------+--------------+-------+----------------+ -1 row in set (0,00 sec) +mysql> SELECT * FROM test; ++--------+-------+ +| int_id | float | ++--------+-------+ +| 1 | 2 | ++--------+-------+ ``` Получение данных в ClickHouse: ``` sql -SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123') +SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123'); ``` ``` text -┌─int_id─┬─int_nullable─┬─float─┬─float_nullable─┐ -│ 1 │ ᴺᵁᴸᴸ │ 2 │ ᴺᵁᴸᴸ │ -└────────┴──────────────┴───────┴────────────────┘ +┌─int_id─┬─float─┐ +│ 1 │ 2 │ +└────────┴───────┘ ``` -## Смотрите также {#smotrite-takzhe} +Замена и вставка: + +```sql +INSERT INTO FUNCTION mysql('localhost:3306', 'test', 'test', 'bayonet', '123', 1) (int_id, float) VALUES (1, 3); +INSERT INTO TABLE FUNCTION mysql('localhost:3306', 'test', 'test', 'bayonet', '123', 0, 'UPDATE int_id = int_id + 1') (int_id, float) VALUES (1, 4); +SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123'); +``` + +``` text +┌─int_id─┬─float─┐ +│ 1 │ 3 │ +│ 2 │ 4 │ +└────────┴───────┘ +``` + +**Смотрите также** - [Движок таблиц ‘MySQL’](../../sql-reference/table-functions/mysql.md) - [Использование MySQL как источника данных для внешнего словаря](../../sql-reference/table-functions/mysql.md#dicts-external_dicts_dict_sources-mysql) -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/mysql/) +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table_functions/mysql/) From c285dafb5d2f6655fdf62febd0c9177f0bee5c1e Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Fri, 5 Feb 2021 22:20:07 +0300 Subject: [PATCH 064/510] edited brown benchmark --- docs/en/getting-started/example-datasets/brown-benchmark.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/getting-started/example-datasets/brown-benchmark.md b/docs/en/getting-started/example-datasets/brown-benchmark.md index effae6d5adb..c9b74a84a54 100644 --- a/docs/en/getting-started/example-datasets/brown-benchmark.md +++ b/docs/en/getting-started/example-datasets/brown-benchmark.md @@ -412,3 +412,5 @@ ORDER BY yr, ``` The data is also available for interactive queries in the [Playground](https://gh-api.clickhouse.tech/play?user=play), [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1hY2hpbmVfbmFtZSwKICAgICAgIE1JTihjcHUpIEFTIGNwdV9taW4sCiAgICAgICBNQVgoY3B1KSBBUyBjcHVfbWF4LAogICAgICAgQVZHKGNwdSkgQVMgY3B1X2F2ZywKICAgICAgIE1JTihuZXRfaW4pIEFTIG5ldF9pbl9taW4sCiAgICAgICBNQVgobmV0X2luKSBBUyBuZXRfaW5fbWF4LAogICAgICAgQVZHKG5ldF9pbikgQVMgbmV0X2luX2F2ZywKICAgICAgIE1JTihuZXRfb3V0KSBBUyBuZXRfb3V0X21pbiwKICAgICAgIE1BWChuZXRfb3V0KSBBUyBuZXRfb3V0X21heCwKICAgICAgIEFWRyhuZXRfb3V0KSBBUyBuZXRfb3V0X2F2ZwpGUk9NICgKICBTRUxFQ1QgbWFjaGluZV9uYW1lLAogICAgICAgICBDT0FMRVNDRShjcHVfdXNlciwgMC4wKSBBUyBjcHUsCiAgICAgICAgIENPQUxFU0NFKGJ5dGVzX2luLCAwLjApIEFTIG5ldF9pbiwKICAgICAgICAgQ09BTEVTQ0UoYnl0ZXNfb3V0LCAwLjApIEFTIG5ldF9vdXQKICBGUk9NIG1nYmVuY2gubG9nczEKICBXSEVSRSBtYWNoaW5lX25hbWUgSU4gKCdhbmFuc2knLCdhcmFnb2cnLCd1cmQnKQogICAgQU5EIGxvZ190aW1lID49IFRJTUVTVEFNUCAnMjAxNy0wMS0xMSAwMDowMDowMCcKKSBBUyByCkdST1VQIEJZIG1hY2hpbmVfbmFtZQ==). + +[Original article](https://clickhouse.tech/docs/en/getting_started/example_datasets/brown-benchmark/) From 44714c3fa895d0b827f771e0e3b9fcd876651d81 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Fri, 5 Feb 2021 22:34:26 +0300 Subject: [PATCH 065/510] edited RU brown benchmark --- docs/ru/getting-started/example-datasets/brown-benchmark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/getting-started/example-datasets/brown-benchmark.md b/docs/ru/getting-started/example-datasets/brown-benchmark.md index b3f2285093a..e4fe00ace93 100644 --- a/docs/ru/getting-started/example-datasets/brown-benchmark.md +++ b/docs/ru/getting-started/example-datasets/brown-benchmark.md @@ -5,7 +5,7 @@ toc_title: Brown University Benchmark # Brown University Benchmark -`MgBench` — это новый аналитический бенчмарк для сгенерированного журнала событий, разработанный [Andrew Crotty](http://cs.brown.edu/people/acrotty/). +`MgBench` — это новый аналитический бенчмарк для данных журнала событий, сгенерированных машиной. Бенчмарк разработан [Andrew Crotty](http://cs.brown.edu/people/acrotty/). Скачать данные: ``` From 0704d3cf27239ec0aa07ee88f256ccc40b891b7e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 6 Feb 2021 03:54:27 +0300 Subject: [PATCH 066/510] Refactor --- src/Client/ConnectionPoolWithFailover.cpp | 10 +- src/Client/ConnectionPoolWithFailover.h | 6 +- src/Client/GetHedgedConnections.cpp | 491 ------------------ src/Client/GetHedgedConnections.h | 173 ------ src/Client/HedgedConnections.cpp | 300 +++++++---- src/Client/HedgedConnections.h | 85 ++- src/Client/HedgedConnectionsFactory.cpp | 475 +++++++++++++++++ src/Client/HedgedConnectionsFactory.h | 167 ++++++ src/Client/ya.make | 2 +- src/Common/Epoll.cpp | 24 +- src/Common/Epoll.h | 12 +- src/Common/TimerDescriptor.h | 12 - .../RemoteQueryExecutorReadContext.cpp | 15 +- src/IO/ConnectionTimeouts.h | 18 +- src/IO/ReadBufferFromPocoSocket.cpp | 8 +- src/IO/ReadBufferFromPocoSocket.h | 1 + 16 files changed, 938 insertions(+), 861 deletions(-) delete mode 100644 src/Client/GetHedgedConnections.cpp delete mode 100644 src/Client/GetHedgedConnections.h create mode 100644 src/Client/HedgedConnectionsFactory.cpp create mode 100644 src/Client/HedgedConnectionsFactory.h diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 3e41c26fb65..15344b3b18b 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -329,7 +329,7 @@ std::vector ConnectionPoolWithFa return Base::getShuffledPools(max_ignored_errors, get_priority); } -TryGetConnection::TryGetConnection( +ConnectionEstablisher::ConnectionEstablisher( IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings * settings_, @@ -340,7 +340,7 @@ TryGetConnection::TryGetConnection( { } -void TryGetConnection::reset() +void ConnectionEstablisher::reset() { resetResult(); stage = Stage::CONNECT; @@ -349,7 +349,7 @@ void TryGetConnection::reset() fail_message.clear(); } -void TryGetConnection::resetResult() +void ConnectionEstablisher::resetResult() { if (!result.entry.isNull()) { @@ -358,7 +358,7 @@ void TryGetConnection::resetResult() } } -void TryGetConnection::processFail(bool add_description) +void ConnectionEstablisher::processFail(bool add_description) { if (action_before_disconnect) action_before_disconnect(socket_fd); @@ -371,7 +371,7 @@ void TryGetConnection::processFail(bool add_description) stage = Stage::FAILED; } -void TryGetConnection::run() +void ConnectionEstablisher::run() { try { diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index a6c0b9e8070..44b06e871ec 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -31,8 +31,8 @@ enum class PoolMode GET_ALL }; -/// Class for establishing connection with replica without blocking. -class TryGetConnection +/// Class for establishing connection with replica without blocking using different stages. +class ConnectionEstablisher { public: enum Stage @@ -47,7 +47,7 @@ public: using TryResult = PoolWithFailoverBase::TryResult; - TryGetConnection(IConnectionPool * pool_, + ConnectionEstablisher(IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings * settings_, const QualifiedTableName * table_to_check = nullptr, diff --git a/src/Client/GetHedgedConnections.cpp b/src/Client/GetHedgedConnections.cpp deleted file mode 100644 index 093b4bc930c..00000000000 --- a/src/Client/GetHedgedConnections.cpp +++ /dev/null @@ -1,491 +0,0 @@ -#if defined(OS_LINUX) - -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; - extern const int LOGICAL_ERROR; - extern const int ALL_CONNECTION_TRIES_FAILED; - extern const int ALL_REPLICAS_ARE_STALE; -} - -GetHedgedConnections::GetHedgedConnections( - const ConnectionPoolWithFailoverPtr & pool_, - const Settings * settings_, - const ConnectionTimeouts & timeouts_, - std::shared_ptr table_to_check_) - : pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(&Poco::Logger::get("GetHedgedConnections")) -{ - shuffled_pools = pool->getShuffledPools(settings); - for (size_t i = 0; i != shuffled_pools.size(); ++i) - try_get_connections.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); - - max_tries - = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); - - fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries : false; - entries_count = 0; - usable_count = 0; - failed_pools_count = 0; -} - -GetHedgedConnections::~GetHedgedConnections() -{ - pool->updateSharedError(shuffled_pools); -} - -std::vector GetHedgedConnections::getManyConnections(PoolMode pool_mode) -{ - size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; - - size_t max_entries; - if (pool_mode == PoolMode::GET_ALL) - { - min_entries = shuffled_pools.size(); - max_entries = shuffled_pools.size(); - } - else if (pool_mode == PoolMode::GET_ONE) - max_entries = 1; - else if (pool_mode == PoolMode::GET_MANY) - max_entries = settings ? size_t(settings->max_parallel_replicas) : 1; - else - throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR); - - std::vector replicas; - replicas.reserve(max_entries); - for (size_t i = 0; i != max_entries; ++i) - { - auto replica = getNextConnection(false); - if (replica->isCannotChoose()) - { - if (replicas.size() >= min_entries) - break; - - /// Determine the reason of not enough replicas. - if (!fallback_to_stale_replicas && usable_count >= min_entries) - throw DB::Exception( - "Could not find enough connections to up-to-date replicas. Got: " + std::to_string(replicas.size()) - + ", needed: " + std::to_string(min_entries), - DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); - - throw DB::NetException( - "All connection tries failed. Log: \n\n" + fail_messages + "\n", - DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); - } - - replicas.push_back(replica); - } - - return replicas; -} - -GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::getNextConnection(bool non_blocking) -{ - ReplicaStatePtr replica = createNewReplica(); - int index; - - /// Check if it's the first time. - if (epoll.empty() && ready_indexes.empty()) - { - index = 0; - last_used_index = 0; - } - else - index = getNextIndex(); - - bool is_first = true; - - while (index != -1 || !epoll.empty()) - { - /// Prevent blocking after receiving timeout when there is no new replica to connect - /// (processEpollEvents can return EMPTY replica after timeout processing to start new connection). - if (index == -1 && !is_first && non_blocking) - { - replica->state = State::NOT_READY; - return replica; - } - - if (is_first) - is_first = false; - - if (index != -1) - { - Action action = startTryGetConnection(index, replica); - - if (action == Action::FINISH) - return replica; - - if (action == Action::TRY_NEXT_REPLICA) - { - index = getNextIndex(); - continue; - } - - if (action == Action::PROCESS_EPOLL_EVENTS && non_blocking) - return replica; - } - - replica = processEpollEvents(non_blocking); - if (replica->isReady() || (replica->isNotReady() && non_blocking)) - return replica; - - index = getNextIndex(); - } - - /// We reach this point only if there was no free up to date replica. - /// We will try to use usable replica. - - /// Check if we are not allowed to use usable replicas or there is no even a free usable replica. - if (!fallback_to_stale_replicas || !canGetNewConnection()) - { - replica->state = State::CANNOT_CHOOSE; - return replica; - } - - setBestUsableReplica(replica); - return replica; -} - -void GetHedgedConnections::stopChoosingReplicas() -{ - for (auto & [fd, replica] : fd_to_replica) - { - removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); - epoll.remove(fd); - try_get_connections[replica->index].reset(); - replica->reset(); - } - - fd_to_replica.clear(); -} - -int GetHedgedConnections::getNextIndex() -{ - /// Check if there is no free replica. - if (entries_count + indexes_in_process.size() + failed_pools_count >= shuffled_pools.size()) - return -1; - - bool finish = false; - int next_index = last_used_index; - while (!finish) - { - next_index = (next_index + 1) % shuffled_pools.size(); - - /// Check if we can try this replica. - if (indexes_in_process.find(next_index) == indexes_in_process.end() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) - && try_get_connections[next_index].stage != TryGetConnection::Stage::FINISHED) - finish = true; - - /// If we made a complete round, there is no replica to connect. - else if (next_index == last_used_index) - return -1; - } - - last_used_index = next_index; - return next_index; -} - -GetHedgedConnections::Action GetHedgedConnections::startTryGetConnection(int index, ReplicaStatePtr & replica) -{ - TryGetConnection & try_get_connection = try_get_connections[index]; - - replica->state = State::NOT_READY; - replica->index = index; - indexes_in_process.insert(index); - - try_get_connection.reset(); - try_get_connection.run(); - - if (try_get_connection.stage != TryGetConnection::Stage::FAILED) - { - replica->fd = try_get_connection.socket_fd; - replica->connection = &*try_get_connection.result.entry; - } - - Action action = processTryGetConnectionStage(replica); - - if (action == Action::PROCESS_EPOLL_EVENTS) - { - epoll.add(try_get_connection.socket_fd); - fd_to_replica[try_get_connection.socket_fd] = replica; - try_get_connection.setActionBeforeDisconnect( - [&](int fd) - { - epoll.remove(fd); - fd_to_replica.erase(fd); - }); - addTimeouts(replica); - } - - return action; -} - -GetHedgedConnections::Action -GetHedgedConnections::processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll) -{ - TryGetConnection & try_get_connection = try_get_connections[replica->index]; - - if (try_get_connection.stage == TryGetConnection::Stage::FINISHED) - { - indexes_in_process.erase(replica->index); - ++entries_count; - - if (remove_from_epoll) - { - epoll.remove(try_get_connection.socket_fd); - fd_to_replica.erase(try_get_connection.socket_fd); - } - - if (try_get_connection.result.is_usable) - { - ++usable_count; - if (try_get_connection.result.is_up_to_date) - { - replica->state = State::READY; - ready_indexes.insert(replica->index); - return Action::FINISH; - } - } - - /// This replica is not up to date, we will try to find up to date. - replica->reset(); - return Action::TRY_NEXT_REPLICA; - } - else if (try_get_connection.stage == TryGetConnection::Stage::FAILED) - { - processFailedConnection(replica); - return Action::TRY_NEXT_REPLICA; - } - - /// Get connection process is not finished. - return Action::PROCESS_EPOLL_EVENTS; -} - -void GetHedgedConnections::processFailedConnection(ReplicaStatePtr & replica) -{ - ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; - LOG_WARNING( - log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), try_get_connections[replica->index].fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); - - shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); - - if (shuffled_pool.error_count >= max_tries) - { - ++failed_pools_count; - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); - } - - std::string & fail_message = try_get_connections[replica->index].fail_message; - if (!fail_message.empty()) - fail_messages += fail_message + "\n"; - - indexes_in_process.erase(replica->index); - replica->reset(); -} - -void GetHedgedConnections::addTimeouts(ReplicaStatePtr & replica) -{ - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); - - auto stage = try_get_connections[replica->index].stage; - if (stage == TryGetConnection::Stage::RECEIVE_HELLO) - addTimeoutToReplica(TimerTypes::RECEIVE_HELLO_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); - else if (stage == TryGetConnection::Stage::RECEIVE_TABLES_STATUS) - addTimeoutToReplica(TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT, replica, epoll, timeout_fd_to_replica, timeouts); -} - -GetHedgedConnections::ReplicaStatePtr GetHedgedConnections::processEpollEvents(bool non_blocking) -{ - int event_fd; - ReplicaStatePtr replica = nullptr; - bool finish = false; - while (!finish) - { - event_fd = getReadyFileDescriptor(); - - if (fd_to_replica.find(event_fd) != fd_to_replica.end()) - { - replica = fd_to_replica[event_fd]; - finish = processReplicaEvent(replica, non_blocking); - } - else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) - { - replica = timeout_fd_to_replica[event_fd]; - finish = processTimeoutEvent(replica, replica->active_timeouts[event_fd], non_blocking); - } - else - throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); - } - - return replica; -} - -int GetHedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) -{ - for (auto & [fd, replica] : fd_to_replica) - if (replica->connection->hasReadPendingData()) - return replica->fd; - - return epoll.getReady(std::move(async_callback)).data.fd; -} - -bool GetHedgedConnections::processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking) -{ - removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); - try_get_connections[replica->index].run(); - Action action = processTryGetConnectionStage(replica, true); - if (action == Action::PROCESS_EPOLL_EVENTS) - { - addTimeouts(replica); - return non_blocking; - } - - return true; -} - -bool GetHedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking) -{ - epoll.remove(timeout_descriptor->getDescriptor()); - replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); - timeout_fd_to_replica[timeout_descriptor->getDescriptor()]; - - if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) - { - removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); - epoll.remove(replica->fd); - fd_to_replica.erase(replica->fd); - - TryGetConnection & try_get_connection = try_get_connections[replica->index]; - try_get_connection.fail_message = "Receive timeout expired (" + try_get_connection.result.entry->getDescription() + ")"; - try_get_connection.resetResult(); - try_get_connection.stage = TryGetConnection::Stage::FAILED; - processFailedConnection(replica); - - return true; - } - else if ((timeout_descriptor->getType() == TimerTypes::RECEIVE_HELLO_TIMEOUT - || timeout_descriptor->getType() == TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT) - && entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size()) - { - replica = createNewReplica(); - return true; - } - - return non_blocking; -} - -void GetHedgedConnections::setBestUsableReplica(ReplicaStatePtr & replica) -{ - std::vector indexes(try_get_connections.size()); - for (size_t i = 0; i != indexes.size(); ++i) - indexes[i] = i; - - /// Remove unusable, failed replicas and replicas that are ready or in process. - indexes.erase( - std::remove_if( - indexes.begin(), - indexes.end(), - [&](int i) - { - return try_get_connections[i].result.entry.isNull() || !try_get_connections[i].result.is_usable || - indexes_in_process.find(i) != indexes_in_process.end() || ready_indexes.find(i) != ready_indexes.end(); - }), - indexes.end()); - - if (indexes.empty()) - { - replica->state = State::CANNOT_CHOOSE; - return; - } - - /// Sort replicas by staleness. - std::stable_sort( - indexes.begin(), - indexes.end(), - [&](size_t lhs, size_t rhs) - { - return try_get_connections[lhs].result.staleness < try_get_connections[rhs].result.staleness; - }); - - replica->index = indexes[0]; - replica->connection = &*try_get_connections[indexes[0]].result.entry; - replica->state = State::READY; - replica->fd = replica->connection->getSocket()->impl()->sockfd(); - ready_indexes.insert(replica->index); -} - -void addTimeoutToReplica( - int type, - GetHedgedConnections::ReplicaStatePtr & replica, - Epoll & epoll, - std::unordered_map & timeout_fd_to_replica, - const ConnectionTimeouts & timeouts) -{ - Poco::Timespan timeout; - switch (type) - { - case TimerTypes::RECEIVE_HELLO_TIMEOUT: - timeout = timeouts.receive_hello_timeout; - break; - case TimerTypes::RECEIVE_TABLES_STATUS_TIMEOUT: - timeout = timeouts.receive_tables_status_timeout; - break; - case TimerTypes::RECEIVE_DATA_TIMEOUT: - timeout = timeouts.receive_data_timeout; - break; - case TimerTypes::RECEIVE_TIMEOUT: - timeout = timeouts.receive_timeout; - break; - default: - throw Exception("Unknown timeout type", ErrorCodes::BAD_ARGUMENTS); - } - - TimerDescriptorPtr timeout_descriptor = std::make_shared(); - timeout_descriptor->setType(type); - timeout_descriptor->setRelative(timeout); - epoll.add(timeout_descriptor->getDescriptor()); - timeout_fd_to_replica[timeout_descriptor->getDescriptor()] = replica; - replica->active_timeouts[timeout_descriptor->getDescriptor()] = std::move(timeout_descriptor); -} - -void removeTimeoutsFromReplica( - GetHedgedConnections::ReplicaStatePtr & replica, - Epoll & epoll, - std::unordered_map & timeout_fd_to_replica) -{ - for (auto & [fd, _] : replica->active_timeouts) - { - epoll.remove(fd); - timeout_fd_to_replica.erase(fd); - } - replica->active_timeouts.clear(); -} - -void removeTimeoutFromReplica( - int type, - GetHedgedConnections::ReplicaStatePtr & replica, - Epoll & epoll, - std::unordered_map & timeout_fd_to_replica) -{ - auto it = std::find_if( - replica->active_timeouts.begin(), - replica->active_timeouts.end(), - [type](auto & value){ return value.second->getType() == type; } - ); - - if (it != replica->active_timeouts.end()) - { - epoll.remove(it->first); - timeout_fd_to_replica.erase(it->first); - replica->active_timeouts.erase(it); - } -} - -} -#endif diff --git a/src/Client/GetHedgedConnections.h b/src/Client/GetHedgedConnections.h deleted file mode 100644 index 8638367e184..00000000000 --- a/src/Client/GetHedgedConnections.h +++ /dev/null @@ -1,173 +0,0 @@ -#pragma once - -#if defined(OS_LINUX) - -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -using TimerDescriptorPtr = std::shared_ptr; - -/// Class for establishing hedged connections with replicas. -/// It works with multiple replicas simultaneously without blocking by using epoll. -class GetHedgedConnections -{ -public: - using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool; - - enum State - { - EMPTY = 0, - READY = 1, - NOT_READY = 2, - CANNOT_CHOOSE = 3, - }; - - struct ReplicaState - { - Connection * connection = nullptr; - State state = State::EMPTY; - int index = -1; - int fd = -1; - size_t parallel_replica_offset = 0; - std::unordered_map> active_timeouts; - - void reset() - { - connection = nullptr; - state = State::EMPTY; - index = -1; - fd = -1; - parallel_replica_offset = 0; - active_timeouts.clear(); - } - - bool isReady() const { return state == State::READY; } - bool isNotReady() const { return state == State::NOT_READY; } - bool isEmpty() const { return state == State::EMPTY; } - bool isCannotChoose() const { return state == State::CANNOT_CHOOSE; } - }; - - using ReplicaStatePtr = std::shared_ptr; - - GetHedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, - const Settings * settings_, - const ConnectionTimeouts & timeouts_, - std::shared_ptr table_to_check_ = nullptr); - - /// Create and return connections according to pool_mode. - std::vector getManyConnections(PoolMode pool_mode); - - /// Try to establish connection to the new replica. If non_blocking is false, this function will block - /// until establishing connection to the new replica (returned replica state might be READY or CANNOT_CHOOSE). - /// If non_blocking is true, this function will try to establish connection to the new replica without blocking - /// (returned replica state might be READY, NOT_READY and CANNOT_CHOOSE). - ReplicaStatePtr getNextConnection(bool non_blocking); - - /// Check if we can try to produce new READY replica. - bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); } - - /// Stop working with all replicas that are not READY. - void stopChoosingReplicas(); - - bool hasEventsInProcess() const { return epoll.size() > 0; } - - int getFileDescriptor() const { return epoll.getFileDescriptor(); } - - const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; } - - ~GetHedgedConnections(); - -private: - - enum Action - { - FINISH = 0, - PROCESS_EPOLL_EVENTS = 1, - TRY_NEXT_REPLICA = 2, - }; - - Action startTryGetConnection(int index, ReplicaStatePtr & replica); - - Action processTryGetConnectionStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); - - /// Find an index of the next free replica to start connection. - /// Return -1 if there is no free replica. - int getNextIndex(); - - int getReadyFileDescriptor(AsyncCallback async_callback = {}); - - void addTimeouts(ReplicaStatePtr & replica); - - void processFailedConnection(ReplicaStatePtr & replica); - - void processReceiveTimeout(ReplicaStatePtr & replica); - - bool processReplicaEvent(ReplicaStatePtr & replica, bool non_blocking); - - bool processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor, bool non_blocking); - - ReplicaStatePtr processEpollEvents(bool non_blocking = false); - - void setBestUsableReplica(ReplicaStatePtr & replica); - - ReplicaStatePtr createNewReplica() { return std::make_shared(); } - - const ConnectionPoolWithFailoverPtr pool; - const Settings * settings; - const ConnectionTimeouts timeouts; - std::shared_ptr table_to_check; - - std::vector try_get_connections; - std::vector shuffled_pools; - - /// Map socket file descriptor to replica. - std::unordered_map fd_to_replica; - /// Map timeout file descriptor to replica. - std::unordered_map timeout_fd_to_replica; - - /// Indexes of replicas, that are in process of connection. - std::unordered_set indexes_in_process; - /// Indexes of ready replicas. - std::unordered_set ready_indexes; - - int last_used_index; - bool fallback_to_stale_replicas; - Epoll epoll; - Poco::Logger * log; - std::string fail_messages; - size_t entries_count; - size_t usable_count; - size_t failed_pools_count; - size_t max_tries; -}; - -/// Add timeout with particular type to replica and add it to epoll. -void addTimeoutToReplica( - int type, - GetHedgedConnections::ReplicaStatePtr & replica, - Epoll & epoll, - std::unordered_map & timeout_fd_to_replica, - const ConnectionTimeouts & timeouts); - -/// Remove timeout with particular type from replica and epoll. -void removeTimeoutFromReplica( - int type, - GetHedgedConnections::ReplicaStatePtr & replica, - Epoll & epoll, - std::unordered_map & timeout_fd_to_replica); - -/// Remove all timeouts from replica and epoll. -void removeTimeoutsFromReplica( - GetHedgedConnections::ReplicaStatePtr & replica, - Epoll & epoll, - std::unordered_map & timeout_fd_to_replica); - -} -#endif diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index f4810a7d79c..a6ffc3cbd1d 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -10,6 +10,7 @@ namespace ErrorCodes extern const int MISMATCH_REPLICAS_DATA_SOURCES; extern const int LOGICAL_ERROR; extern const int SOCKET_TIMEOUT; + extern const int ALL_CONNECTION_TRIES_FAILED; } HedgedConnections::HedgedConnections( @@ -19,29 +20,35 @@ HedgedConnections::HedgedConnections( const ThrottlerPtr & throttler_, PoolMode pool_mode, std::shared_ptr table_to_check_) - : get_hedged_connections(pool_, &settings_, timeouts_, table_to_check_), settings(settings_), throttler(throttler_), log(&Poco::Logger::get("HedgedConnections")) + : hedged_connections_factory(pool_, &settings_, timeouts_, table_to_check_) + , settings(settings_) + , throttler(throttler_) + , log(&Poco::Logger::get("HedgedConnections")) { - std::vector replicas_states = get_hedged_connections.getManyConnections(pool_mode); + std::vector connections = hedged_connections_factory.getManyConnections(pool_mode); - for (size_t i = 0; i != replicas_states.size(); ++i) + ReplicaState replica; + for (size_t i = 0; i != connections.size(); ++i) { - replicas_states[i]->parallel_replica_offset = i; - replicas_states[i]->connection->setThrottler(throttler_); - epoll.add(replicas_states[i]->fd); - fd_to_replica[replicas_states[i]->fd] = replicas_states[i]; - replicas.push_back({std::move(replicas_states[i])}); - active_connections_count_by_offset[i] = 1; + replica.connection = connections[i]; + replica.connection->setThrottler(throttler_); + int socket_fd = replica.connection->getSocket()->impl()->sockfd(); + epoll.add(socket_fd); + fd_to_replica_location[socket_fd] = ReplicaLocation{i, 0}; + offset_states.push_back(OffsetState{{replica}, 1, false}); } - pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr & replica_){ replica_->connection->setThrottler(throttler_); }); + active_connection_count = connections.size(); + offsets_with_received_first_data_packet = 0; + pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); }); } -void HedgedConnections::Pipeline::add(std::function send_function) +void HedgedConnections::Pipeline::add(std::function send_function) { pipeline.push_back(send_function); } -void HedgedConnections::Pipeline::run(ReplicaStatePtr & replica) +void HedgedConnections::Pipeline::run(ReplicaState & replica) { for (auto & send_func : pipeline) send_func(replica); @@ -54,11 +61,11 @@ void HedgedConnections::sendScalarsData(Scalars & data) if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); - auto send_scalars_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendScalarsData(data); }; + auto send_scalars_data = [&data](ReplicaState & replica) { replica.connection->sendScalarsData(data); }; - for (auto & replicas_with_same_offset : replicas) - for (auto & replica : replicas_with_same_offset) - if (replica->isReady()) + for (auto & offset_state : offset_states) + for (auto & replica : offset_state.replicas) + if (replica.connection) send_scalars_data(replica); pipeline_for_new_replicas.add(send_scalars_data); @@ -74,11 +81,11 @@ void HedgedConnections::sendExternalTablesData(std::vector & if (data.size() != size()) throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES); - auto send_external_tables_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendExternalTablesData(data[0]); }; + auto send_external_tables_data = [&data](ReplicaState & replica) { replica.connection->sendExternalTablesData(data[0]); }; - for (auto & replicas_with_same_offset : replicas) - for (auto & replica : replicas_with_same_offset) - if (replica->isReady()) + for (auto & offset_state : offset_states) + for (auto & replica : offset_state.replicas) + if (replica.connection) send_external_tables_data(replica); pipeline_for_new_replicas.add(send_external_tables_data); @@ -97,11 +104,11 @@ void HedgedConnections::sendQuery( if (sent_query) throw Exception("Query already sent.", ErrorCodes::LOGICAL_ERROR); - for (auto & replicas_with_same_offset : replicas) + for (auto & offset_state : offset_states) { - for (auto & replica : replicas_with_same_offset) + for (auto & replica : offset_state.replicas) { - if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + if (replica.connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) { disable_two_level_aggregation = true; break; @@ -111,30 +118,29 @@ void HedgedConnections::sendQuery( break; } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica) - { - Settings modified_settings = this->settings; + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) { + Settings modified_settings = settings; - if (this->disable_two_level_aggregation) + if (disable_two_level_aggregation) { /// Disable two-level aggregation due to version incompatibility. modified_settings.group_by_two_level_threshold = 0; modified_settings.group_by_two_level_threshold_bytes = 0; } - if (this->replicas.size() > 1) + if (offset_states.size() > 1) { - modified_settings.parallel_replicas_count = this->replicas.size(); - modified_settings.parallel_replica_offset = replica->parallel_replica_offset; + modified_settings.parallel_replicas_count = offset_states.size(); + modified_settings.parallel_replica_offset = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()].offset; } - replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); - addTimeoutToReplica(TimerTypes::RECEIVE_DATA_TIMEOUT, replica, this->epoll, this->timeout_fd_to_replica, timeouts); + replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, replica); }; - for (auto & replicas_with_same_offset : replicas) - for (auto & replica : replicas_with_same_offset) + for (auto & offset_status : offset_states) + for (auto & replica : offset_status.replicas) send_query(replica); pipeline_for_new_replicas.add(send_query); @@ -145,16 +151,20 @@ void HedgedConnections::disconnect() { std::lock_guard lock(cancel_mutex); - for (auto & replicas_with_same_offset : replicas) - for (auto & replica : replicas_with_same_offset) - if (replica->isReady()) + for (auto & offset_status : offset_states) + for (auto & replica : offset_status.replicas) + if (replica.connection) finishProcessReplica(replica, true); - if (get_hedged_connections.hasEventsInProcess()) + if (hedged_connections_factory.hasEventsInProcess()) { - get_hedged_connections.stopChoosingReplicas(); if (next_replica_in_process) - epoll.remove(get_hedged_connections.getFileDescriptor()); + { + epoll.remove(hedged_connections_factory.getFileDescriptor()); + next_replica_in_process = false; + } + + hedged_connections_factory.stopChoosingReplicas(); } } @@ -165,13 +175,13 @@ std::string HedgedConnections::dumpAddresses() const std::string addresses; bool is_first = true; - for (const auto & replicas_with_same_offset : replicas) + for (const auto & offset_state : offset_states) { - for (const auto & replica : replicas_with_same_offset) + for (const auto & replica : offset_state.replicas) { - if (replica->isReady()) + if (replica.connection) { - addresses += (is_first ? "" : "; ") + replica->connection->getDescription(); + addresses += (is_first ? "" : "; ") + replica.connection->getDescription(); is_first = false; } } @@ -187,15 +197,14 @@ void HedgedConnections::sendCancel() if (!sent_query || cancelled) throw Exception("Cannot cancel. Either no query sent or already cancelled.", ErrorCodes::LOGICAL_ERROR); - for (auto & replicas_with_same_offset : replicas) - for (auto & replica : replicas_with_same_offset) - if (replica->isReady()) - replica->connection->sendCancel(); + for (auto & offset_status : offset_states) + for (auto & replica : offset_status.replicas) + if (replica.connection) + replica.connection->sendCancel(); cancelled = true; } - Packet HedgedConnections::drain() { std::lock_guard lock(cancel_mutex); @@ -252,26 +261,24 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { int event_fd; - ReplicaStatePtr replica = nullptr; Packet packet; bool finish = false; while (!finish) { event_fd = getReadyFileDescriptor(async_callback); - if (fd_to_replica.find(event_fd) != fd_to_replica.end()) + if (fd_to_replica_location.contains(event_fd)) { - replica = fd_to_replica[event_fd]; - packet = receivePacketFromReplica(replica, async_callback); + packet = receivePacketFromReplica(fd_to_replica_location[event_fd], async_callback); finish = true; } - else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) + else if (timeout_fd_to_replica_location.contains(event_fd)) { - replica = timeout_fd_to_replica[event_fd]; - processTimeoutEvent(replica, replica->active_timeouts[event_fd]); + ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; + processTimeoutEvent(location, offset_states[location.offset].replicas[location.index].active_timeouts[event_fd]); } - else if (event_fd == get_hedged_connections.getFileDescriptor()) - tryGetNewReplica(); + else if (event_fd == hedged_connections_factory.getFileDescriptor()) + tryGetNewReplica(false); else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } @@ -281,30 +288,34 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { - for (auto & [fd, replica] : fd_to_replica) - if (replica->connection->hasReadPendingData()) - return replica->fd; + for (auto & [fd, location] : fd_to_replica_location) + { + ReplicaState & replica = offset_states[location.offset].replicas[location.index]; + if (replica.connection->hasReadPendingData()) + return replica.connection->getSocket()->impl()->sockfd(); + } - return epoll.getReady(std::move(async_callback)).data.fd; + return epoll.getReady(true, std::move(async_callback)).data.fd; } -Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) +Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback) { - Packet packet = replica->connection->receivePacket(std::move(async_callback)); + ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; + removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + Packet packet = replica.connection->receivePacket(std::move(async_callback)); switch (packet.type) { case Protocol::Server::Data: - removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); - processReceiveData(replica); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, get_hedged_connections.getConnectionTimeouts()); + if (!offset_states[replica_location.offset].first_packet_of_data_received) + processReceivedFirstDataPacket(replica_location); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); break; case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: - removeTimeoutFromReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica); - addTimeoutToReplica(TimerTypes::RECEIVE_TIMEOUT, replica, epoll, timeout_fd_to_replica, get_hedged_connections.getConnectionTimeouts()); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); break; case Protocol::Server::EndOfStream: @@ -320,96 +331,155 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, As return packet; } -void HedgedConnections::processReceiveData(ReplicaStatePtr & replica) +void HedgedConnections::processReceivedFirstDataPacket(ReplicaLocation & replica_location) { /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. - offsets_with_received_data.insert(replica->parallel_replica_offset); + OffsetState & offset_state = offset_states[replica_location.offset]; + removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, offset_state.replicas[replica_location.index]); + ++offsets_with_received_first_data_packet; + offset_state.first_packet_of_data_received = true; - for (auto & other_replica : replicas[replica->parallel_replica_offset]) + for (size_t i = 0; i != offset_state.replicas.size(); ++i) { - if (other_replica->isReady() && other_replica != replica) + if (i != replica_location.index && offset_state.replicas[i].connection) { - other_replica->connection->sendCancel(); - finishProcessReplica(other_replica, true); + offset_state.replicas[i].connection->sendCancel(); + finishProcessReplica(offset_state.replicas[i], true); } } /// If we received data from replicas with all offsets, we need to stop choosing new replicas. - if (get_hedged_connections.hasEventsInProcess() && offsets_with_received_data.size() == replicas.size()) + if (hedged_connections_factory.hasEventsInProcess() && offsets_with_received_first_data_packet == offset_states.size()) { - get_hedged_connections.stopChoosingReplicas(); if (next_replica_in_process) - epoll.remove(get_hedged_connections.getFileDescriptor()); + { + epoll.remove(hedged_connections_factory.getFileDescriptor()); + next_replica_in_process = false; + } + hedged_connections_factory.stopChoosingReplicas(); } } -void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor) +void HedgedConnections::processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor) { - epoll.remove(timeout_descriptor->getDescriptor()); - replica->active_timeouts.erase(timeout_descriptor->getDescriptor()); - timeout_fd_to_replica.erase(timeout_descriptor->getDescriptor()); + ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; + epoll.remove(timeout_descriptor->timer.getDescriptor()); + replica.active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica_location.erase(timeout_descriptor->timer.getDescriptor()); - if (timeout_descriptor->getType() == TimerTypes::RECEIVE_TIMEOUT) + if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) { - size_t offset = replica->parallel_replica_offset; finishProcessReplica(replica, true); - /// Check if there is no active connections with the same offset. - if (active_connections_count_by_offset[offset] == 0) + /// Check if there is no active connections with the same offset and there is no new replica in process. + if (offset_states[replica_location.offset].active_connection_count == 0 && !next_replica_in_process) throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); } - else if (timeout_descriptor->getType() == TimerTypes::RECEIVE_DATA_TIMEOUT) + else if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT) { - offsets_queue.push(replica->parallel_replica_offset); - tryGetNewReplica(); + offsets_queue.push(replica_location.offset); + tryGetNewReplica(true); } } -void HedgedConnections::tryGetNewReplica() +void HedgedConnections::tryGetNewReplica(bool start_new_connection) { - ReplicaStatePtr new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); + Connection * connection = nullptr; + HedgedConnectionsFactory::State state = hedged_connections_factory.getNextConnection(start_new_connection, connection); /// Skip replicas that doesn't support two-level aggregation if we didn't disable it in sendQuery. - while (new_replica->isReady() && !disable_two_level_aggregation - && new_replica->connection->getServerRevision(get_hedged_connections.getConnectionTimeouts()) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) - new_replica = get_hedged_connections.getNextConnection(/*non_blocking*/ true); + while (state == HedgedConnectionsFactory::State::READY && !disable_two_level_aggregation + && connection->getServerRevision(hedged_connections_factory.getConnectionTimeouts()) + < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + state = hedged_connections_factory.getNextConnection(true, connection); - if (new_replica->isReady()) + if (state == HedgedConnectionsFactory::State::READY) { - new_replica->parallel_replica_offset = offsets_queue.front(); + size_t offset = offsets_queue.front(); offsets_queue.pop(); - replicas[new_replica->parallel_replica_offset].push_back(new_replica); - epoll.add(new_replica->fd); - fd_to_replica[new_replica->fd] = new_replica; - ++active_connections_count_by_offset[new_replica->parallel_replica_offset]; - pipeline_for_new_replicas.run(new_replica); + size_t index = offset_states[offset].replicas.size(); + + ReplicaState replica; + replica.connection = connection; + int socket_fd = replica.connection->getSocket()->impl()->sockfd(); + epoll.add(socket_fd); + fd_to_replica_location[socket_fd] = ReplicaLocation{offset, index}; + offset_states[offset].replicas.push_back(replica); + ++offset_states[offset].active_connection_count; + ++active_connection_count; + pipeline_for_new_replicas.run(replica); } - else if (new_replica->isNotReady() && !next_replica_in_process) + else if (state == HedgedConnectionsFactory::State::NOT_READY && !next_replica_in_process) { - epoll.add(get_hedged_connections.getFileDescriptor()); + epoll.add(hedged_connections_factory.getFileDescriptor()); next_replica_in_process = true; } - if (next_replica_in_process && (new_replica->isCannotChoose() || offsets_queue.empty())) + /// Check if we cannot get new replica and there is no active replica with needed offsets. + else if (state == HedgedConnectionsFactory::State::CANNOT_CHOOSE) { - epoll.remove(get_hedged_connections.getFileDescriptor()); + while (!offsets_queue.empty()) + { + if (offset_states[offsets_queue.front()].active_connection_count == 0) + throw Exception("Cannot find enough connections to replicas", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + offsets_queue.pop(); + } + } + + /// Check if we don't need to listen hedged_connections_factory file descriptor in epoll anymore. + if (next_replica_in_process && (state == HedgedConnectionsFactory::State::CANNOT_CHOOSE || offsets_queue.empty())) + { + epoll.remove(hedged_connections_factory.getFileDescriptor()); next_replica_in_process = false; } } -void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect) +void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { - removeTimeoutsFromReplica(replica, epoll, timeout_fd_to_replica); - epoll.remove(replica->fd); - fd_to_replica.erase(replica->fd); - --active_connections_count_by_offset[replica->parallel_replica_offset]; - if (active_connections_count_by_offset[replica->parallel_replica_offset] == 0) - active_connections_count_by_offset.erase(replica->parallel_replica_offset); + removeTimeoutsFromReplica(replica); + int socket_fd = replica.connection->getSocket()->impl()->sockfd(); + epoll.remove(socket_fd); + --offset_states[fd_to_replica_location[socket_fd].offset].active_connection_count; + fd_to_replica_location.erase(socket_fd); + --active_connection_count; if (disconnect) - replica->connection->disconnect(); - replica->reset(); + replica.connection->disconnect(); + replica.connection = nullptr; +} + +void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica) +{ + ConnectionTimeoutDescriptorPtr timeout_descriptor + = createConnectionTimeoutDescriptor(type, hedged_connections_factory.getConnectionTimeouts()); + epoll.add(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica_location[timeout_descriptor->timer.getDescriptor()] + = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()]; + replica.active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); +} + +void HedgedConnections::removeTimeoutsFromReplica(ReplicaState & replica) +{ + for (auto & [fd, _] : replica.active_timeouts) + { + epoll.remove(fd); + timeout_fd_to_replica_location.erase(fd); + } + replica.active_timeouts.clear(); +} + +void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica) +{ + auto it = std::find_if( + replica.active_timeouts.begin(), replica.active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); + + if (it != replica.active_timeouts.end()) + { + epoll.remove(it->first); + timeout_fd_to_replica_location.erase(it->first); + replica.active_timeouts.erase(it); + } } } diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 8081fa6739d..6931db9ede6 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -1,18 +1,41 @@ #pragma once #if defined(OS_LINUX) -#include -#include #include #include +#include +#include namespace DB { +/** To receive data from multiple replicas (connections) from one shard asynchronously, + * The principe of Hedged Connections is used to reduce tail latency: + * (if we don't receive data from replica for a long time, we try to get new replica + * and send query to it, without cancelling working with previous replica). This class + * supports all functionality that MultipleConnections has. + */ class HedgedConnections : public IConnections { public: - using ReplicaStatePtr = GetHedgedConnections::ReplicaStatePtr; + struct ReplicaState + { + Connection * connection = nullptr; + std::unordered_map active_timeouts; + }; + + struct ReplicaLocation + { + size_t offset; + size_t index; + }; + + struct OffsetState + { + std::vector replicas; + size_t active_connection_count; + bool first_packet_of_data_received; + }; HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, const Settings & settings_, @@ -45,57 +68,67 @@ public: std::string dumpAddresses() const override; - size_t size() const override { return replicas.size(); } + size_t size() const override { return offset_states.size(); } - bool hasActiveConnections() const override { return !active_connections_count_by_offset.empty(); } + bool hasActiveConnections() const override { return active_connection_count > 0; } private: /// We will save actions with replicas in pipeline to perform them on the new replicas. class Pipeline { public: - void add(std::function send_function); + void add(std::function send_function); - void run(ReplicaStatePtr & replica); + void run(ReplicaState & replica); private: - std::vector> pipeline; + std::vector> pipeline; }; - Packet receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback = {}); Packet receivePacketImpl(AsyncCallback async_callback = {}); - void processReceiveData(ReplicaStatePtr & replica); + void processReceivedFirstDataPacket(ReplicaLocation & replica_location); - void processTimeoutEvent(ReplicaStatePtr & replica, TimerDescriptorPtr timeout_descriptor); + void processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor); - void tryGetNewReplica(); + void tryGetNewReplica(bool start_new_connection); - void finishProcessReplica(ReplicaStatePtr & replica, bool disconnect); + void finishProcessReplica(ReplicaState & replica, bool disconnect); int getReadyFileDescriptor(AsyncCallback async_callback = {}); - GetHedgedConnections get_hedged_connections; + void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica); - /// All replicas in replicas[offset] are responsible for process query + void removeTimeoutsFromReplica(ReplicaState & replica); + + void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica); + + + HedgedConnectionsFactory hedged_connections_factory; + + /// All replicas in offset_states[offset] is responsible for process query /// with setting parallel_replica_offset = offset. In common situations - /// replicas[offset].size() = 1 (like in MultiplexedConnections). - std::vector> replicas; + /// replica_states[offset].replicas.size() = 1 (like in MultiplexedConnections). + std::vector offset_states; - /// Map socket file descriptor to replica. - std::unordered_map fd_to_replica; - /// Map timeout file descriptor to replica. - std::unordered_map timeout_fd_to_replica; + /// Map socket file descriptor to replica location (it's offset and index in OffsetState.replicas). + std::unordered_map fd_to_replica_location; + /// Map timeout file descriptor to replica location (it's offset and index in OffsetState.replicas). + std::unordered_map timeout_fd_to_replica_location; /// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from /// the replica, we push it's offset to this queue and start trying to get /// new replica. std::queue offsets_queue; - /// Map offset to amount of active connections, responsible to this offset. - std::unordered_map active_connections_count_by_offset; + /// The current number of valid connections to the replicas of this shard. + size_t active_connection_count; - std::unordered_set offsets_with_received_data; + /// We count offsets which received first packet of data, + /// it's needed to cancel choosing new replicas when all offsets + /// received their first packet of data. + size_t offsets_with_received_first_data_packet; Pipeline pipeline_for_new_replicas; @@ -103,8 +136,8 @@ private: /// If we didn't disabled it, we need to skip this replica. bool disable_two_level_aggregation = false; - /// next_replica_in_process is true when get_hedged_connections.getFileDescriptor() - /// is in epoll now and false otherwise. + /// This flag means we need to get connection with new replica, but no replica is ready. + /// When it's true, hedged_connections_factory.getFileDescriptor() is in epoll. bool next_replica_in_process = false; Epoll epoll; diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp new file mode 100644 index 00000000000..22666642b4e --- /dev/null +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -0,0 +1,475 @@ +#if defined(OS_LINUX) + +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int LOGICAL_ERROR; + extern const int ALL_CONNECTION_TRIES_FAILED; + extern const int ALL_REPLICAS_ARE_STALE; +} + +HedgedConnectionsFactory::HedgedConnectionsFactory( + const ConnectionPoolWithFailoverPtr & pool_, + const Settings * settings_, + const ConnectionTimeouts & timeouts_, + std::shared_ptr table_to_check_) + : pool(pool_), settings(settings_), timeouts(timeouts_), table_to_check(table_to_check_), log(&Poco::Logger::get("HedgedConnectionsFactory")) +{ + shuffled_pools = pool->getShuffledPools(settings); + for (size_t i = 0; i != shuffled_pools.size(); ++i) + connection_establishers.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); + + max_tries + = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); + + fallback_to_stale_replicas = settings && settings->fallback_to_stale_replicas_for_distributed_queries; + entries_count = 0; + usable_count = 0; + failed_pools_count = 0; +} + +HedgedConnectionsFactory::~HedgedConnectionsFactory() +{ + pool->updateSharedError(shuffled_pools); +} + +std::vector HedgedConnectionsFactory::getManyConnections(PoolMode pool_mode) +{ + size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; + + size_t max_entries; + if (pool_mode == PoolMode::GET_ALL) + { + min_entries = shuffled_pools.size(); + max_entries = shuffled_pools.size(); + } + else if (pool_mode == PoolMode::GET_ONE) + max_entries = 1; + else if (pool_mode == PoolMode::GET_MANY) + max_entries = settings ? size_t(settings->max_parallel_replicas) : 1; + else + throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR); + + std::vector connections; + connections.reserve(max_entries); + + /// Try to start establishing connections with max_entries replicas. + int index; + for (size_t i = 0; i != max_entries; ++i) + { + index = getNextIndex(); + if (index == -1) + break; + + ReplicaStatePtr replica = startEstablishingConnection(index); + if (replica->state == State::READY) + connections.push_back(replica->connection); + } + + /// Process connections until we get enough READY connections + /// (work asynchronously with all connections we started). + Connection * connection = nullptr; + while (connections.size() < max_entries) + { + auto state = processConnections(true, connection); + if (state == State::READY) + connections.push_back(connection); + else if (state == State::CANNOT_CHOOSE) + { + if (connections.size() >= min_entries) + break; + + /// Determine the reason of not enough replicas. + if (!fallback_to_stale_replicas && usable_count >= min_entries) + throw DB::Exception( + "Could not find enough connections to up-to-date replicas. Got: " + std::to_string(connections.size()) + + ", needed: " + std::to_string(min_entries), + DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); + + throw DB::NetException( + "All connection tries failed. Log: \n\n" + fail_messages + "\n", + DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + } + } + + return connections; +} + +HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, Connection *& connection_out) +{ + if (start_new_connection) + { + /// Try to start establishing connection to the new replica. + int index = getNextIndex(); + if (index != -1) + { + ReplicaStatePtr replica = startEstablishingConnection(index); + if (replica->state == State::READY) + { + connection_out = replica->connection; + return State::READY; + } + } + } + + return processConnections(false, connection_out); +} + +HedgedConnectionsFactory::State HedgedConnectionsFactory::processConnections(bool blocking, Connection *& connection_out) +{ + ReplicaStatePtr replica = nullptr; + int index = -1; + + while (index != -1 || !epoll.empty()) + { + if (index != -1) + { + replica = startEstablishingConnection(index); + if (replica->state == State::READY) + { + connection_out = replica->connection; + return State::READY; + } + } + + if (!processEpollEvents(replica, blocking)) + return State::NOT_READY; + + if (replica->state == State::READY) + { + connection_out = replica->connection; + return State::READY; + } + + index = getNextIndex(); + } + + /// We reach this point only if there was no free up to date replica. + /// We will try to use usable replica. + + /// Check if we are not allowed to use usable replicas or there is no even a free usable replica. + if (!fallback_to_stale_replicas || !canGetNewConnection()) + return State::CANNOT_CHOOSE; + + setBestUsableReplica(replica); + connection_out = replica->connection; + return replica->state; +} + +void HedgedConnectionsFactory::stopChoosingReplicas() +{ + for (auto & [fd, replica] : fd_to_replica) + { + removeTimeoutsFromReplica(replica); + epoll.remove(fd); + connection_establishers[replica->index].reset(); + replica->reset(); + } + + fd_to_replica.clear(); +} + +int HedgedConnectionsFactory::getNextIndex() +{ + /// Check if there is no free replica. + if (entries_count + indexes_in_process.size() + failed_pools_count >= shuffled_pools.size()) + return -1; + + /// Check if it's the first time. + if (last_used_index == -1) + { + last_used_index = 0; + return 0; + } + + bool finish = false; + int next_index = last_used_index; + while (!finish) + { + next_index = (next_index + 1) % shuffled_pools.size(); + + /// Check if we can try this replica. + if (indexes_in_process.find(next_index) == indexes_in_process.end() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) + && connection_establishers[next_index].stage != ConnectionEstablisher::Stage::FINISHED) + finish = true; + + /// If we made a complete round, there is no replica to connect. + else if (next_index == last_used_index) + return -1; + } + + last_used_index = next_index; + return next_index; +} + +HedgedConnectionsFactory::ReplicaStatePtr HedgedConnectionsFactory::startEstablishingConnection(int index) +{ + ReplicaStatePtr replica = createNewReplica(); + + do + { + ConnectionEstablisher & connection_establisher = connection_establishers[index]; + + replica->state = State::NOT_READY; + replica->index = index; + indexes_in_process.insert(index); + + connection_establisher.reset(); + connection_establisher.run(); + + if (connection_establisher.stage != ConnectionEstablisher::Stage::FAILED) + replica->connection = &*connection_establisher.result.entry; + + processConnectionEstablisherStage(replica); + + if (replica->state == State::NOT_READY) + { + epoll.add(connection_establisher.socket_fd); + fd_to_replica[connection_establisher.socket_fd] = replica; + connection_establisher.setActionBeforeDisconnect([&](int fd) { + epoll.remove(fd); + fd_to_replica.erase(fd); + }); + addTimeouts(replica); + } + } + while (replica->state == State::EMPTY && (index = getNextIndex()) != -1); + + return replica; +} + +void HedgedConnectionsFactory::processConnectionEstablisherStage(ReplicaStatePtr & replica, bool remove_from_epoll) +{ + ConnectionEstablisher & connection_establisher = connection_establishers[replica->index]; + + if (connection_establisher.stage == ConnectionEstablisher::Stage::FINISHED) + { + indexes_in_process.erase(replica->index); + ++entries_count; + + if (remove_from_epoll) + { + epoll.remove(connection_establisher.socket_fd); + fd_to_replica.erase(connection_establisher.socket_fd); + } + + if (connection_establisher.result.is_usable) + { + ++usable_count; + if (connection_establisher.result.is_up_to_date) + { + replica->state = State::READY; + ready_indexes.insert(replica->index); + return; + } + } + + /// This replica is not up to date, we will try to find up to date. + replica->reset(); + } + else if (connection_establisher.stage == ConnectionEstablisher::Stage::FAILED) + processFailedConnection(replica); +} + +void HedgedConnectionsFactory::processFailedConnection(ReplicaStatePtr & replica) +{ + ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; + LOG_WARNING( + log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), connection_establishers[replica->index].fail_message); + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); + + shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); + + if (shuffled_pool.error_count >= max_tries) + { + ++failed_pools_count; + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); + } + + std::string & fail_message = connection_establishers[replica->index].fail_message; + if (!fail_message.empty()) + fail_messages += fail_message + "\n"; + + indexes_in_process.erase(replica->index); + replica->reset(); +} + +void HedgedConnectionsFactory::addTimeouts(ReplicaStatePtr & replica) +{ + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + + auto stage = connection_establishers[replica->index].stage; + if (stage == ConnectionEstablisher::Stage::RECEIVE_HELLO) + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT, replica); + else if (stage == ConnectionEstablisher::Stage::RECEIVE_TABLES_STATUS) + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT, replica); +} + +void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica) +{ + ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, timeouts); + epoll.add(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()] = replica; + replica->active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); +} + +void HedgedConnectionsFactory::removeTimeoutsFromReplica(ReplicaStatePtr & replica) +{ + for (auto & [fd, _] : replica->active_timeouts) + { + epoll.remove(fd); + timeout_fd_to_replica.erase(fd); + } + replica->active_timeouts.clear(); +} + +bool HedgedConnectionsFactory::processEpollEvents(ReplicaStatePtr & replica, bool blocking) +{ + int event_fd; + bool finish = false; + while (!finish) + { + event_fd = getReadyFileDescriptor(blocking); + + /// Check if there is no events. + if (event_fd == -1) + return false; + + if (fd_to_replica.find(event_fd) != fd_to_replica.end()) + { + replica = fd_to_replica[event_fd]; + processReplicaEvent(replica); + /// Check if replica is ready or we need to try next replica. + if (replica->state == State::READY || replica->state == State::EMPTY) + finish = true; + } + else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) + { + replica = timeout_fd_to_replica[event_fd]; + processTimeoutEvent(replica, replica->active_timeouts[event_fd]); + /// Check if we need to try next replica. + if (replica->state == State::EMPTY) + finish = true; + } + else + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); + } + + return true; +} + +int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) +{ + for (auto & [fd, replica] : fd_to_replica) + if (replica->connection->hasReadPendingData()) + return replica->connection->getSocket()->impl()->sockfd(); + + return epoll.getReady(/* blocking */blocking).data.fd; +} + +void HedgedConnectionsFactory::processReplicaEvent(ReplicaStatePtr & replica) +{ + removeTimeoutsFromReplica(replica); + connection_establishers[replica->index].run(); + processConnectionEstablisherStage(replica, true); + if (replica->state == State::NOT_READY) + addTimeouts(replica); +} + +void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) +{ + epoll.remove(timeout_descriptor->timer.getDescriptor()); + replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()]; + + if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) + { + removeTimeoutsFromReplica(replica); + int fd = replica->connection->getSocket()->impl()->sockfd(); + epoll.remove(fd); + fd_to_replica.erase(fd); + + ConnectionEstablisher & connection_establisher = connection_establishers[replica->index]; + connection_establisher.fail_message = "Receive timeout expired (" + connection_establisher.result.entry->getDescription() + ")"; + connection_establisher.resetResult(); + connection_establisher.stage = ConnectionEstablisher::Stage::FAILED; + processFailedConnection(replica); + } + else if ((timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT + || timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT) + && entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size()) + replica = createNewReplica(); +} + +void HedgedConnectionsFactory::setBestUsableReplica(ReplicaStatePtr & replica) +{ + std::vector indexes(connection_establishers.size()); + for (size_t i = 0; i != indexes.size(); ++i) + indexes[i] = i; + + /// Remove unusable, failed replicas and replicas that are ready or in process. + indexes.erase( + std::remove_if( + indexes.begin(), + indexes.end(), + [&](int i) + { + return connection_establishers[i].result.entry.isNull() || !connection_establishers[i].result.is_usable || + indexes_in_process.find(i) != indexes_in_process.end() || ready_indexes.find(i) != ready_indexes.end(); + }), + indexes.end()); + + if (indexes.empty()) + { + replica->state = State::CANNOT_CHOOSE; + return; + } + + /// Sort replicas by staleness. + std::stable_sort( + indexes.begin(), + indexes.end(), + [&](size_t lhs, size_t rhs) + { + return connection_establishers[lhs].result.staleness < connection_establishers[rhs].result.staleness; + }); + + replica->index = indexes[0]; + replica->connection = &*connection_establishers[indexes[0]].result.entry; + replica->state = State::READY; + ready_indexes.insert(replica->index); +} + +ConnectionTimeoutDescriptorPtr createConnectionTimeoutDescriptor(ConnectionTimeoutType type, const ConnectionTimeouts & timeouts) +{ + Poco::Timespan timeout; + switch (type) + { + case ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT: + timeout = timeouts.receive_hello_timeout; + break; + case ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT: + timeout = timeouts.receive_tables_status_timeout; + break; + case ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT: + timeout = timeouts.receive_data_timeout; + break; + case ConnectionTimeoutType::RECEIVE_TIMEOUT: + timeout = timeouts.receive_timeout; + break; + } + + ConnectionTimeoutDescriptorPtr timeout_descriptor = std::make_shared(); + timeout_descriptor->type = type; + timeout_descriptor->timer.setRelative(timeout); + return timeout_descriptor; +} + +} +#endif diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h new file mode 100644 index 00000000000..d1dc262d39c --- /dev/null +++ b/src/Client/HedgedConnectionsFactory.h @@ -0,0 +1,167 @@ +#pragma once + +#if defined(OS_LINUX) + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +enum class ConnectionTimeoutType +{ + RECEIVE_HELLO_TIMEOUT, + RECEIVE_TABLES_STATUS_TIMEOUT, + RECEIVE_DATA_TIMEOUT, + RECEIVE_TIMEOUT, +}; + +struct ConnectionTimeoutDescriptor +{ + ConnectionTimeoutType type; + TimerDescriptor timer; +}; + +using ConnectionTimeoutDescriptorPtr = std::shared_ptr; +using TimerDescriptorPtr = std::shared_ptr; + +/** Class for establishing hedged connections with replicas. + * The process of establishing connection is divided on stages, on each stage if + * replica doesn't respond for a long time, we start establishing connection with + * the next replica, without cancelling working with previous one. + * It works with multiple replicas simultaneously without blocking by using epoll. + */ +class HedgedConnectionsFactory +{ +public: + using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool; + + enum class State + { + EMPTY = 0, + READY = 1, + NOT_READY = 2, + CANNOT_CHOOSE = 3, + }; + + struct ReplicaState + { + Connection * connection = nullptr; + size_t index = -1; + State state = State::EMPTY; + std::unordered_map active_timeouts; + + void reset() + { + connection = nullptr; + index = -1; + state = State::EMPTY; + active_timeouts.clear(); + } + }; + + using ReplicaStatePtr = std::shared_ptr; + + HedgedConnectionsFactory(const ConnectionPoolWithFailoverPtr & pool_, + const Settings * settings_, + const ConnectionTimeouts & timeouts_, + std::shared_ptr table_to_check_ = nullptr); + + /// Create and return active connections according to pool_mode. + std::vector getManyConnections(PoolMode pool_mode); + + /// Try to get connection to the new replica without blocking. If start_new_connection is true, we start establishing connection + /// with the new replica and then call processConnections, otherwise just call processConnections. + State getNextConnection(bool start_new_connection, Connection *& connection_out); + + /// Process all current events in epoll (connections, timeouts), if there is no events in epoll and blocking is false, + /// return NOT_READY. Returned state might be READY, NOT_READY and CANNOT_CHOOSE. + /// If state is READY, replica connection will be written in connection_out. + State processConnections(bool blocking, Connection *& connection_out); + + /// Check if we can try to produce new READY replica. + bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); } + + /// Stop working with all replicas that are not READY. + void stopChoosingReplicas(); + + bool hasEventsInProcess() const { return epoll.size() > 0; } + + int getFileDescriptor() const { return epoll.getFileDescriptor(); } + + const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; } + + ~HedgedConnectionsFactory(); + +private: + ReplicaStatePtr startEstablishingConnection(int index); + + void processConnectionEstablisherStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); + + /// Find an index of the next free replica to start connection. + /// Return -1 if there is no free replica. + int getNextIndex(); + + int getReadyFileDescriptor(bool blocking); + + void addTimeouts(ReplicaStatePtr & replica); + + void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica); + + void removeTimeoutsFromReplica(ReplicaStatePtr & replica); + + void processFailedConnection(ReplicaStatePtr & replica); + + void processReceiveTimeout(ReplicaStatePtr & replica); + + void processReplicaEvent(ReplicaStatePtr & replica); + + void processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); + + /// Return false if there is no ready events, return true if replica is ready + /// or we need to try next replica. + bool processEpollEvents(ReplicaStatePtr & replica, bool blocking); + + void setBestUsableReplica(ReplicaStatePtr & replica); + + ReplicaStatePtr createNewReplica() { return std::make_shared(); } + + const ConnectionPoolWithFailoverPtr pool; + const Settings * settings; + const ConnectionTimeouts timeouts; + std::shared_ptr table_to_check; + + std::vector connection_establishers; + std::vector shuffled_pools; + std::vector replica_states; + + /// Map socket file descriptor to replica. + std::unordered_map fd_to_replica; + /// Map timeout file descriptor to replica. + std::unordered_map timeout_fd_to_replica; + + /// Indexes of replicas, that are in process of connection. + std::unordered_set indexes_in_process; + /// Indexes of ready replicas. + std::unordered_set ready_indexes; + + int last_used_index = -1; + bool fallback_to_stale_replicas; + Epoll epoll; + Poco::Logger * log; + std::string fail_messages; + size_t entries_count; + size_t usable_count; + size_t failed_pools_count; + size_t max_tries; +}; + +/// Create ConnectionTimeoutDescriptor with particular type. +ConnectionTimeoutDescriptorPtr createConnectionTimeoutDescriptor(ConnectionTimeoutType type, const ConnectionTimeouts & timeouts); + +} +#endif diff --git a/src/Client/ya.make b/src/Client/ya.make index 603e8290350..7a664f328f7 100644 --- a/src/Client/ya.make +++ b/src/Client/ya.make @@ -12,8 +12,8 @@ PEERDIR( SRCS( Connection.cpp ConnectionPoolWithFailover.cpp - GetHedgedConnections.cpp HedgedConnections.cpp + HedgedConnectionsFactory.cpp MultiplexedConnections.cpp TimeoutSetter.cpp diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index cb34f81cf36..bfd323b4f55 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -46,24 +46,24 @@ void Epoll::remove(int fd) --events_count; } -epoll_event Epoll::getReady(AsyncCallback async_callback) const +epoll_event Epoll::getReady(bool blocking, AsyncCallback async_callback) const { - std::vector events = getManyReady(1, true, std::move(async_callback)); - if (events.empty()) - throw Exception("Vector of ready events is empty", ErrorCodes::LOGICAL_ERROR); + epoll_event event; + event.data.fd = -1; + size_t ready_events_count = getManyReady(1, &event, blocking, std::move(async_callback)); + if (ready_events_count > 1) + throw Exception("Returned amount of events cannot be more than 1.", ErrorCodes::LOGICAL_ERROR); - return events[0]; + return event; } -std::vector Epoll::getManyReady(int max_events, bool blocking, AsyncCallback async_callback) const +size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback) const { - std::vector events(max_events); - int ready_size = 0; int timeout = blocking && !async_callback ? -1 : 0; - while (ready_size <= 0 && (ready_size != 0 || blocking)) + do { - ready_size = epoll_wait(epoll_fd, events.data(), max_events, timeout); + ready_size = epoll_wait(epoll_fd, events_out, max_events, timeout); if (ready_size == -1 && errno != EINTR) throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR); @@ -71,9 +71,9 @@ std::vector Epoll::getManyReady(int max_events, bool blocking, Asyn if (ready_size == 0 && blocking && async_callback) async_callback(epoll_fd, 0, "epoll"); } + while (ready_size <= 0 && (ready_size != 0 || blocking)); - events.resize(ready_size); - return events; + return ready_size; } Epoll::~Epoll() diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index 1dc65d15d08..92638715aeb 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -16,20 +16,22 @@ class Epoll : boost::noncopyable public: Epoll(); - /// Add new file descriptor to epoll. + /// Add new file descriptor to epoll. If ptr set to nullptr, epoll_event.data.fd = fd, + /// otherwise epoll_event.data.ptr = ptr. void add(int fd, void * ptr = nullptr); /// Remove file descriptor to epoll. void remove(int fd); - /// Get events from epoll. If blocking is false and there are no ready events, + /// Get events from epoll. Events are written in events_out, this function returns an amount of ready events. + /// If blocking is false and there are no ready events, /// return empty vector, otherwise wait for ready events. If blocking is true, /// async_callback is given and there is no ready events, async_callback is called /// with epoll file descriptor. - std::vector getManyReady(int max_events, bool blocking, AsyncCallback async_callback = {}) const; + size_t getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback = {}) const; - /// Get only one ready event, this function is always blocking. - epoll_event getReady(AsyncCallback async_callback = {}) const; + /// Get only one ready event, if blocking is false and there is no ready events, epoll_event.data.fd will be set to -1. + epoll_event getReady(bool blocking = true, AsyncCallback async_callback = {}) const; int getFileDescriptor() const { return epoll_fd; } diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index debf7cdc899..6f7003f6980 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -5,21 +5,11 @@ namespace DB { -enum TimerTypes -{ - DEFAULT, - RECEIVE_HELLO_TIMEOUT, - RECEIVE_TABLES_STATUS_TIMEOUT, - RECEIVE_DATA_TIMEOUT, - RECEIVE_TIMEOUT, -}; - /// Wrapper over timerfd. class TimerDescriptor { private: int timer_fd; - int type = TimerTypes::DEFAULT; public: explicit TimerDescriptor(int clockid = CLOCK_MONOTONIC, int flags = 0); @@ -31,12 +21,10 @@ public: TimerDescriptor & operator=(TimerDescriptor &&) = default; int getDescriptor() const { return timer_fd; } - int getType() const { return type; } void reset() const; void drain() const; void setRelative(const Poco::Timespan & timespan) const; - void setType(int type_) { type = type_; } }; } diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp index c77b2d48f05..e02ac1fc1b3 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -121,19 +121,22 @@ bool RemoteQueryExecutorReadContext::checkTimeout() const bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const { /// Wait for epoll will not block if it was polled externally. - std::vector events = epoll.getManyReady(epoll.size(), /* blocking = */ false); + epoll_event events[3]; + events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; + + epoll.getManyReady(3, events,/* blocking = */ false); bool is_socket_ready = false; bool is_pipe_alarmed = false; bool has_timer_alarm = false; - for (const auto & event : events) + for (int i = 0; i < 3; ++i) { - if (event.data.fd == connection_fd) + if (events[i].data.fd == connection_fd) is_socket_ready = true; - if (event.data.fd == timer.getDescriptor()) + if (events[i].data.fd == timer.getDescriptor()) has_timer_alarm = true; - if (event.data.fd == pipe_fd[0]) + if (events[i].data.fd == pipe_fd[0]) is_pipe_alarmed = true; } @@ -198,7 +201,7 @@ void RemoteQueryExecutorReadContext::cancel() RemoteQueryExecutorReadContext::~RemoteQueryExecutorReadContext() { - /// connection_fd is closed by Poco::Net::Socket + /// connection_fd is closed by Poco::Net::Socket or Epoll if (pipe_fd[0] != -1) close(pipe_fd[0]); if (pipe_fd[1] != -1) diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 01f31d6efa8..a92f75bf980 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -33,9 +33,9 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(0), http_keep_alive_timeout(0), secure_connection_timeout(connection_timeout), - receive_hello_timeout(0), - receive_tables_status_timeout(0), - receive_data_timeout(0) + receive_hello_timeout(receive_timeout_), + receive_tables_status_timeout(receive_timeout_), + receive_data_timeout(receive_timeout_) { } @@ -49,9 +49,9 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(0), secure_connection_timeout(connection_timeout), - receive_hello_timeout(0), - receive_tables_status_timeout(0), - receive_data_timeout(0) + receive_hello_timeout(receive_timeout_), + receive_tables_status_timeout(receive_timeout_), + receive_data_timeout(receive_timeout_) { } ConnectionTimeouts(const Poco::Timespan & connection_timeout_, @@ -65,9 +65,9 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(http_keep_alive_timeout_), secure_connection_timeout(connection_timeout), - receive_hello_timeout(0), - receive_tables_status_timeout(0), - receive_data_timeout(0) + receive_hello_timeout(receive_timeout_), + receive_tables_status_timeout(receive_timeout_), + receive_data_timeout(receive_timeout_) { } diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index e08b9e7c8fb..1f9c732e644 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -14,7 +14,6 @@ namespace ProfileEvents namespace DB { - namespace ErrorCodes { extern const int NETWORK_ERROR; @@ -42,7 +41,7 @@ bool ReadBufferFromPocoSocket::nextImpl() /// Note that receive timeout is not checked here. External code should check it while polling. while (bytes_read < 0 && async_callback && errno == EAGAIN) { - async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), "socket (" + socket.peerAddress().toString() + ")"); + async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), socket_description); bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), internal_buffer.size(), flags); } } @@ -74,7 +73,10 @@ bool ReadBufferFromPocoSocket::nextImpl() } ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, size_t buf_size) - : BufferWithOwnMemory(buf_size), socket(socket_), peer_address(socket.peerAddress()) + : BufferWithOwnMemory(buf_size) + , socket(socket_) + , peer_address(socket.peerAddress()) + , socket_description("socket (" + peer_address.toString() + ")") { } diff --git a/src/IO/ReadBufferFromPocoSocket.h b/src/IO/ReadBufferFromPocoSocket.h index 7fd1b646846..73e83dfb5f9 100644 --- a/src/IO/ReadBufferFromPocoSocket.h +++ b/src/IO/ReadBufferFromPocoSocket.h @@ -34,6 +34,7 @@ public: private: AsyncCallback async_callback; + std::string socket_description; }; } From 610798aa487ee1b2ef6007b9185a1c1b27a11660 Mon Sep 17 00:00:00 2001 From: keenwolf Date: Sat, 6 Feb 2021 15:32:49 +0800 Subject: [PATCH 067/510] fix the toMinute bug which will cause toDateTime or toString printing wrong time --- base/common/DateLUTImpl.h | 10 +++- src/Functions/ya.make | 1 + .../0_stateless/01698_fix_toMinute.reference | 24 ++++++++ .../0_stateless/01698_fix_toMinute.sql | 16 +++++ .../01699_timezoneOffset.reference | 58 +++++-------------- .../0_stateless/01699_timezoneOffset.sql | 3 +- 6 files changed, 65 insertions(+), 47 deletions(-) create mode 100644 tests/queries/0_stateless/01698_fix_toMinute.reference create mode 100644 tests/queries/0_stateless/01698_fix_toMinute.sql diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h index 23c78f6e7fc..8991f69d3f3 100644 --- a/base/common/DateLUTImpl.h +++ b/base/common/DateLUTImpl.h @@ -317,8 +317,14 @@ public: if (offset_is_whole_number_of_hours_everytime) return (t / 60) % 60; - UInt32 date = find(t).date; - return (UInt32(t) - date) / 60 % 60; + /// To consider the DST changing situation within this day. + /// also make the special timezones with no whole hour offset such as 'Australia/Lord_Howe' been taken into account + DayNum index = findIndex(t); + time_t res = t - lut[index].date; + if (lut[index].amount_of_offset_change != 0 && t >= lut[index].date + lut[index].time_at_offset_change) + res += lut[index].amount_of_offset_change; + + return res / 60 % 60; } inline time_t toStartOfMinute(time_t t) const { return t / 60 * 60; } diff --git a/src/Functions/ya.make b/src/Functions/ya.make index b97a4a187e9..647bbbb47cb 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -452,6 +452,7 @@ SRCS( timeSlot.cpp timeSlots.cpp timezone.cpp + timezoneOffset.cpp toColumnTypeName.cpp toCustomWeek.cpp toDayOfMonth.cpp diff --git a/tests/queries/0_stateless/01698_fix_toMinute.reference b/tests/queries/0_stateless/01698_fix_toMinute.reference new file mode 100644 index 00000000000..5df800c9fef --- /dev/null +++ b/tests/queries/0_stateless/01698_fix_toMinute.reference @@ -0,0 +1,24 @@ +Check the bug causing situation: the special Australia/Lord_Howe time zone. tooDateTime and toString functions are all tested at once +1554559200 2019-04-07 01:00:00 2019-04-07 01:00:00 +1554559800 2019-04-07 01:10:00 2019-04-07 01:10:00 +1554560400 2019-04-07 01:20:00 2019-04-07 01:20:00 +1554561000 2019-04-07 01:30:00 2019-04-07 01:30:00 +1554561600 2019-04-07 01:40:00 2019-04-07 01:40:00 +1554562200 2019-04-07 01:50:00 2019-04-07 01:50:00 +1554562800 2019-04-07 01:30:00 2019-04-07 01:30:00 +1554563400 2019-04-07 01:40:00 2019-04-07 01:40:00 +1554564000 2019-04-07 01:50:00 2019-04-07 01:50:00 +1554564600 2019-04-07 02:00:00 2019-04-07 02:00:00 +1554565200 2019-04-07 02:10:00 2019-04-07 02:10:00 +1554565800 2019-04-07 02:20:00 2019-04-07 02:20:00 +1554566400 2019-04-07 02:30:00 2019-04-07 02:30:00 +1554567000 2019-04-07 02:40:00 2019-04-07 02:40:00 +1554567600 2019-04-07 02:50:00 2019-04-07 02:50:00 +1554568200 2019-04-07 03:00:00 2019-04-07 03:00:00 +1554568800 2019-04-07 03:10:00 2019-04-07 03:10:00 +1554569400 2019-04-07 03:20:00 2019-04-07 03:20:00 +1554570000 2019-04-07 03:30:00 2019-04-07 03:30:00 +1554570600 2019-04-07 03:40:00 2019-04-07 03:40:00 +4 days test in batch comparing with manually computation result for Europe/Moscow whose timezone epoc is of whole hour: +4 days test in batch comparing with manually computation result for Asia/Tehran whose timezone epoc is of half hour: +4 days test in batch comparing with manually computation result for Australia/Lord_Howe whose timezone epoc is of half hour and also its DST offset is half hour: diff --git a/tests/queries/0_stateless/01698_fix_toMinute.sql b/tests/queries/0_stateless/01698_fix_toMinute.sql new file mode 100644 index 00000000000..293741b6957 --- /dev/null +++ b/tests/queries/0_stateless/01698_fix_toMinute.sql @@ -0,0 +1,16 @@ +/* toDateTime or toString or other functions which should call the toMinute() function will all meet this bug. tests below will verify the toDateTime and toString. */ +SELECT 'Check the bug causing situation: the special Australia/Lord_Howe time zone. tooDateTime and toString functions are all tested at once'; +SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, toString(x) as xx FROM numbers(20); + +/* The Batch Part. Test period is whole 4 days*/ +SELECT '4 days test in batch comparing with manually computation result for Europe/Moscow whose timezone epoc is of whole hour:'; +SELECT toUnixTimestamp(x) as tt, (toDateTime('1981-04-01 00:00:00', 'Europe/Moscow') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; +SELECT toUnixTimestamp(x) as tt, (toDateTime('1981-09-30 00:00:00', 'Europe/Moscow') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; + +SELECT '4 days test in batch comparing with manually computation result for Asia/Tehran whose timezone epoc is of half hour:'; +SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-03-21 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; +SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-09-20 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; + +SELECT '4 days test in batch comparing with manually computation result for Australia/Lord_Howe whose timezone epoc is of half hour and also its DST offset is half hour:'; +SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-10-04 01:40:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; +SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; diff --git a/tests/queries/0_stateless/01699_timezoneOffset.reference b/tests/queries/0_stateless/01699_timezoneOffset.reference index e70c5fa62ee..45f30314f5a 100644 --- a/tests/queries/0_stateless/01699_timezoneOffset.reference +++ b/tests/queries/0_stateless/01699_timezoneOffset.reference @@ -50,57 +50,29 @@ DST boundary test for Australia/Lord_Howe. This is a special timezone with DST o DST boundary test for Australia/Lord_Howe: 0 2020-10-04 01:40:00 37800 1601737800 1 2020-10-04 01:50:00 37800 1601738400 -2 2020-10-04 02:00:00 39600 1601739000 -3 2020-10-04 02:10:00 39600 1601739600 +2 2020-10-04 02:30:00 39600 1601739000 +3 2020-10-04 02:40:00 39600 1601739600 0 2019-04-07 01:00:00 39600 1554559200 1 2019-04-07 01:10:00 39600 1554559800 2 2019-04-07 01:20:00 39600 1554560400 3 2019-04-07 01:30:00 39600 1554561000 4 2019-04-07 01:40:00 39600 1554561600 5 2019-04-07 01:50:00 39600 1554562200 -6 2019-04-07 01:00:00 37800 1554562800 -7 2019-04-07 01:10:00 37800 1554563400 -8 2019-04-07 01:20:00 37800 1554564000 -9 2019-04-07 02:30:00 37800 1554564600 -10 2019-04-07 02:40:00 37800 1554565200 -11 2019-04-07 02:50:00 37800 1554565800 -12 2019-04-07 02:00:00 37800 1554566400 -13 2019-04-07 02:10:00 37800 1554567000 -14 2019-04-07 02:20:00 37800 1554567600 -15 2019-04-07 03:30:00 37800 1554568200 -16 2019-04-07 03:40:00 37800 1554568800 -17 2019-04-07 03:50:00 37800 1554569400 +6 2019-04-07 01:30:00 37800 1554562800 +7 2019-04-07 01:40:00 37800 1554563400 +8 2019-04-07 01:50:00 37800 1554564000 +9 2019-04-07 02:00:00 37800 1554564600 +10 2019-04-07 02:10:00 37800 1554565200 +11 2019-04-07 02:20:00 37800 1554565800 +12 2019-04-07 02:30:00 37800 1554566400 +13 2019-04-07 02:40:00 37800 1554567000 +14 2019-04-07 02:50:00 37800 1554567600 +15 2019-04-07 03:00:00 37800 1554568200 +16 2019-04-07 03:10:00 37800 1554568800 +17 2019-04-07 03:20:00 37800 1554569400 4 days test in batch comparing with manually computation result for Europe/Moscow: 4 days test in batch comparing with manually computation result for Asia/Tehran: -The result maybe wrong for toDateTime processing Australia/Lord_Howe -1601739000 2020-10-04 02:00:00 39600 37800 -1601739600 2020-10-04 02:10:00 39600 37800 -1601740200 2020-10-04 02:20:00 39600 37800 -1601740800 2020-10-04 03:30:00 39600 41400 -1601741400 2020-10-04 03:40:00 39600 41400 -1601742000 2020-10-04 03:50:00 39600 41400 -1601742600 2020-10-04 03:00:00 39600 37800 -1601743200 2020-10-04 03:10:00 39600 37800 -1601743800 2020-10-04 03:20:00 39600 37800 -1601744400 2020-10-04 04:30:00 39600 41400 -1601745000 2020-10-04 04:40:00 39600 41400 -1601745600 2020-10-04 04:50:00 39600 41400 -1601746200 2020-10-04 04:00:00 39600 37800 -1601746800 2020-10-04 04:10:00 39600 37800 -1601747400 2020-10-04 04:20:00 39600 37800 -1601748000 2020-10-04 05:30:00 39600 41400 -1554562800 2019-04-07 01:00:00 37800 36000 -1554563400 2019-04-07 01:10:00 37800 36000 -1554564000 2019-04-07 01:20:00 37800 36000 -1554564600 2019-04-07 02:30:00 37800 39600 -1554565200 2019-04-07 02:40:00 37800 39600 -1554565800 2019-04-07 02:50:00 37800 39600 -1554566400 2019-04-07 02:00:00 37800 36000 -1554567000 2019-04-07 02:10:00 37800 36000 -1554567600 2019-04-07 02:20:00 37800 36000 -1554568200 2019-04-07 03:30:00 37800 39600 -1554568800 2019-04-07 03:40:00 37800 39600 -1554569400 2019-04-07 03:50:00 37800 39600 +4 days test in batch comparing with manually computation result for Australia/Lord_Howe Moscow DST Years: 11 1981-06-01 00:00:00 14400 12 1982-06-01 00:00:00 14400 diff --git a/tests/queries/0_stateless/01699_timezoneOffset.sql b/tests/queries/0_stateless/01699_timezoneOffset.sql index 1b3f05ecdd7..8cabb23c4de 100644 --- a/tests/queries/0_stateless/01699_timezoneOffset.sql +++ b/tests/queries/0_stateless/01699_timezoneOffset.sql @@ -26,8 +26,7 @@ SELECT '4 days test in batch comparing with manually computation result for Asia SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-03-21 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-09-20 00:00:00', 'Asia/Tehran') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(576) where res != calc; -/* During this test we got unexpected result comes from the toDateTime() function when process the special time zone of 'Australia/Lord_Howe', which may be some kind of bugs. */ -SELECT 'The result maybe wrong for toDateTime processing Australia/Lord_Howe'; +SELECT '4 days test in batch comparing with manually computation result for Australia/Lord_Howe'; SELECT toUnixTimestamp(x) as tt, (toDateTime('2020-10-04 01:40:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(18) where res != calc; SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, timezoneOffset(x) as res,(toDateTime(toString(x), 'UTC') - x ) AS calc FROM numbers(18) where res != calc; From 740c1c72e6eed901e56d7256f1067304e265dcf9 Mon Sep 17 00:00:00 2001 From: keenwolf Date: Sat, 6 Feb 2021 16:55:46 +0800 Subject: [PATCH 068/510] little fix --- tests/queries/0_stateless/01698_fix_toMinute.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01698_fix_toMinute.sql b/tests/queries/0_stateless/01698_fix_toMinute.sql index 293741b6957..f582806719d 100644 --- a/tests/queries/0_stateless/01698_fix_toMinute.sql +++ b/tests/queries/0_stateless/01698_fix_toMinute.sql @@ -1,5 +1,5 @@ /* toDateTime or toString or other functions which should call the toMinute() function will all meet this bug. tests below will verify the toDateTime and toString. */ -SELECT 'Check the bug causing situation: the special Australia/Lord_Howe time zone. tooDateTime and toString functions are all tested at once'; +SELECT 'Check the bug causing situation: the special Australia/Lord_Howe time zone. toDateTime and toString functions are all tested at once'; SELECT toUnixTimestamp(x) as tt, (toDateTime('2019-04-07 01:00:00', 'Australia/Lord_Howe') + INTERVAL number * 600 SECOND) AS x, toString(x) as xx FROM numbers(20); /* The Batch Part. Test period is whole 4 days*/ From 34af94accfc03fb6335aae9b8ca27f6e6992d49d Mon Sep 17 00:00:00 2001 From: keenwolf Date: Sat, 6 Feb 2021 16:59:01 +0800 Subject: [PATCH 069/510] little fix --- tests/queries/0_stateless/01698_fix_toMinute.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01698_fix_toMinute.reference b/tests/queries/0_stateless/01698_fix_toMinute.reference index 5df800c9fef..7675aad3a57 100644 --- a/tests/queries/0_stateless/01698_fix_toMinute.reference +++ b/tests/queries/0_stateless/01698_fix_toMinute.reference @@ -1,4 +1,4 @@ -Check the bug causing situation: the special Australia/Lord_Howe time zone. tooDateTime and toString functions are all tested at once +Check the bug causing situation: the special Australia/Lord_Howe time zone. toDateTime and toString functions are all tested at once 1554559200 2019-04-07 01:00:00 2019-04-07 01:00:00 1554559800 2019-04-07 01:10:00 2019-04-07 01:10:00 1554560400 2019-04-07 01:20:00 2019-04-07 01:20:00 From 794f185442ea1da799e8c2348bbaac0b6e310aa7 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 6 Feb 2021 17:23:48 +0300 Subject: [PATCH 070/510] Fix --- src/Client/HedgedConnections.cpp | 7 +++++-- src/Client/HedgedConnectionsFactory.cpp | 12 +++++++----- src/Client/HedgedConnectionsFactory.h | 2 +- src/Common/Epoll.cpp | 11 ----------- src/Common/Epoll.h | 3 --- src/Processors/Executors/PollingQueue.cpp | 4 +++- 6 files changed, 16 insertions(+), 23 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index a6ffc3cbd1d..8a7c728146f 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -275,7 +275,7 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) else if (timeout_fd_to_replica_location.contains(event_fd)) { ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; - processTimeoutEvent(location, offset_states[location.offset].replicas[location.index].active_timeouts[event_fd]); + processTimeoutEvent(location, offset_states[location.offset].replicas[location.index].active_timeouts[event_fd]); } else if (event_fd == hedged_connections_factory.getFileDescriptor()) tryGetNewReplica(false); @@ -295,7 +295,10 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) return replica.connection->getSocket()->impl()->sockfd(); } - return epoll.getReady(true, std::move(async_callback)).data.fd; + epoll_event event; + event.data.fd = -1; + epoll.getManyReady(1, &event, true, std::move(async_callback)); + return event.data.fd; } Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 22666642b4e..01063faa2fe 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -8,7 +8,6 @@ namespace DB { namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; extern const int ALL_CONNECTION_TRIES_FAILED; extern const int ALL_REPLICAS_ARE_STALE; @@ -370,7 +369,10 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) if (replica->connection->hasReadPendingData()) return replica->connection->getSocket()->impl()->sockfd(); - return epoll.getReady(/* blocking */blocking).data.fd; + epoll_event event; + event.data.fd = -1; + epoll.getManyReady(1, &event, blocking); + return event.data.fd; } void HedgedConnectionsFactory::processReplicaEvent(ReplicaStatePtr & replica) @@ -388,7 +390,7 @@ void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr & replica, Co replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()]; - if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) + if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) { removeTimeoutsFromReplica(replica); int fd = replica->connection->getSocket()->impl()->sockfd(); @@ -401,8 +403,8 @@ void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr & replica, Co connection_establisher.stage = ConnectionEstablisher::Stage::FAILED; processFailedConnection(replica); } - else if ((timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT - || timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT) + else if ((timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT + || timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT) && entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size()) replica = createNewReplica(); } diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index d1dc262d39c..0a3ac1e7b47 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -100,7 +100,7 @@ public: private: ReplicaStatePtr startEstablishingConnection(int index); - void processConnectionEstablisherStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); + void processConnectionEstablisherStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); /// Find an index of the next free replica to start connection. /// Return -1 if there is no free replica. diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index bfd323b4f55..5a0140a06ec 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -46,17 +46,6 @@ void Epoll::remove(int fd) --events_count; } -epoll_event Epoll::getReady(bool blocking, AsyncCallback async_callback) const -{ - epoll_event event; - event.data.fd = -1; - size_t ready_events_count = getManyReady(1, &event, blocking, std::move(async_callback)); - if (ready_events_count > 1) - throw Exception("Returned amount of events cannot be more than 1.", ErrorCodes::LOGICAL_ERROR); - - return event; -} - size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback) const { int ready_size = 0; diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index 92638715aeb..3a91199799b 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -30,9 +30,6 @@ public: /// with epoll file descriptor. size_t getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback = {}) const; - /// Get only one ready event, if blocking is false and there is no ready events, epoll_event.data.fd will be set to -1. - epoll_event getReady(bool blocking = true, AsyncCallback async_callback = {}) const; - int getFileDescriptor() const { return epoll_fd; } int size() const { return events_count; } diff --git a/src/Processors/Executors/PollingQueue.cpp b/src/Processors/Executors/PollingQueue.cpp index 44941ae788a..a601d426a5d 100644 --- a/src/Processors/Executors/PollingQueue.cpp +++ b/src/Processors/Executors/PollingQueue.cpp @@ -68,7 +68,9 @@ PollingQueue::TaskData PollingQueue::wait(std::unique_lock & lock) lock.unlock(); - epoll_event event = epoll.getReady(); + epoll_event event; + event.data.ptr = nullptr; + epoll.getManyReady(1, &event, true); lock.lock(); From 8ff3dde2903c65d23ea4e26568651b707d83ba20 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 6 Feb 2021 18:23:41 +0300 Subject: [PATCH 071/510] Add sendIgnoredPartUUIDs to HedgedRequests --- src/Client/HedgedConnections.cpp | 19 +++++++++++++++++++ src/Client/HedgedConnections.h | 2 ++ src/Client/IConnections.h | 3 +++ src/Client/MultiplexedConnections.h | 2 +- src/DataStreams/RemoteQueryExecutor.cpp | 2 +- 5 files changed, 26 insertions(+), 2 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 8a7c728146f..957e4d09fe5 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -91,6 +91,23 @@ void HedgedConnections::sendExternalTablesData(std::vector & pipeline_for_new_replicas.add(send_external_tables_data); } +void HedgedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) +{ + std::lock_guard lock(cancel_mutex); + + if (sent_query) + throw Exception("Cannot send uuids after query is sent.", ErrorCodes::LOGICAL_ERROR); + + auto send_ignored_part_uuids = [&uuids](ReplicaState & replica) { replica.connection->sendIgnoredPartUUIDs(uuids); }; + + for (auto & offset_state : offset_states) + for (auto & replica : offset_state.replicas) + if (replica.connection) + send_ignored_part_uuids(replica); + + pipeline_for_new_replicas.add(send_ignored_part_uuids); +} + void HedgedConnections::sendQuery( const ConnectionTimeouts & timeouts, const String & query, @@ -220,6 +237,7 @@ Packet HedgedConnections::drain() Packet packet = receivePacketImpl(); switch (packet.type) { + case Protocol::Server::PartUUIDs: case Protocol::Server::Data: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: @@ -313,6 +331,7 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_loc processReceivedFirstDataPacket(replica_location); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); break; + case Protocol::Server::PartUUIDs: case Protocol::Server::Progress: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 6931db9ede6..eb73f2ded52 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -64,6 +64,8 @@ public: void sendCancel() override; + void sendIgnoredPartUUIDs(const std::vector & uuids) override; + Packet drain() override; std::string dumpAddresses() const override; diff --git a/src/Client/IConnections.h b/src/Client/IConnections.h index 85d1e29c243..38730922456 100644 --- a/src/Client/IConnections.h +++ b/src/Client/IConnections.h @@ -36,6 +36,9 @@ public: /// Send a request to replicas to cancel the request virtual void sendCancel() = 0; + /// Send parts' uuids to replicas to exclude them from query processing + virtual void sendIgnoredPartUUIDs(const std::vector & uuids) = 0; + /** On each replica, read and skip all packets to EndOfStream or Exception. * Returns EndOfStream if no exception has been received. Otherwise * returns the last received packet of type Exception. diff --git a/src/Client/MultiplexedConnections.h b/src/Client/MultiplexedConnections.h index a2f7f42e6b6..c04b06e525e 100644 --- a/src/Client/MultiplexedConnections.h +++ b/src/Client/MultiplexedConnections.h @@ -46,7 +46,7 @@ public: void sendCancel() override; /// Send parts' uuids to replicas to exclude them from query processing - void sendIgnoredPartUUIDs(const std::vector & uuids); + void sendIgnoredPartUUIDs(const std::vector & uuids) override; Packet drain() override; diff --git a/src/DataStreams/RemoteQueryExecutor.cpp b/src/DataStreams/RemoteQueryExecutor.cpp index 19996b563a2..9cac638ceb5 100644 --- a/src/DataStreams/RemoteQueryExecutor.cpp +++ b/src/DataStreams/RemoteQueryExecutor.cpp @@ -273,7 +273,7 @@ std::variant RemoteQueryExecutor::restartQueryWithoutDuplicatedUUIDs { /// Cancel previous query and disconnect before retry. cancel(read_context); - multiplexed_connections->disconnect(); + connections->disconnect(); /// Only resend once, otherwise throw an exception if (!resent_query) From 9048dc43d469563cbe71350fe2f51dd84f2fac62 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 6 Feb 2021 22:13:50 +0300 Subject: [PATCH 072/510] Fix style and build --- src/Common/Epoll.cpp | 3 --- src/DataStreams/RemoteQueryExecutorReadContext.cpp | 4 ++-- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 5a0140a06ec..da3a4c4c04b 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -8,12 +8,9 @@ namespace DB { - -/// TODO: add appropriate error codes namespace ErrorCodes { extern const int EPOLL_ERROR; - extern const int LOGICAL_ERROR; } Epoll::Epoll() : events_count(0) diff --git a/src/DataStreams/RemoteQueryExecutorReadContext.cpp b/src/DataStreams/RemoteQueryExecutorReadContext.cpp index e02ac1fc1b3..c2a65f02d08 100644 --- a/src/DataStreams/RemoteQueryExecutorReadContext.cpp +++ b/src/DataStreams/RemoteQueryExecutorReadContext.cpp @@ -124,13 +124,13 @@ bool RemoteQueryExecutorReadContext::checkTimeoutImpl() const epoll_event events[3]; events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - epoll.getManyReady(3, events,/* blocking = */ false); + int num_events = epoll.getManyReady(3, events,/* blocking = */ false); bool is_socket_ready = false; bool is_pipe_alarmed = false; bool has_timer_alarm = false; - for (int i = 0; i < 3; ++i) + for (int i = 0; i < num_events; ++i) { if (events[i].data.fd == connection_fd) is_socket_ready = true; From 3d8e05dc94d0b241dcd0e69c908f9b056fccd3fe Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 7 Feb 2021 00:54:29 +0300 Subject: [PATCH 073/510] Fix style --- src/Client/HedgedConnections.cpp | 3 ++- src/Client/HedgedConnectionsFactory.cpp | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 957e4d09fe5..65100a7ea41 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -135,7 +135,8 @@ void HedgedConnections::sendQuery( break; } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) { + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) + { Settings modified_settings = settings; if (disable_two_level_aggregation) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 01063faa2fe..84848949fb9 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -231,7 +231,8 @@ HedgedConnectionsFactory::ReplicaStatePtr HedgedConnectionsFactory::startEstabli { epoll.add(connection_establisher.socket_fd); fd_to_replica[connection_establisher.socket_fd] = replica; - connection_establisher.setActionBeforeDisconnect([&](int fd) { + connection_establisher.setActionBeforeDisconnect([&](int fd) + { epoll.remove(fd); fd_to_replica.erase(fd); }); From 636ef5394bd5f2977783b46b3d33ca4620220b4f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sun, 7 Feb 2021 01:45:11 +0300 Subject: [PATCH 074/510] Remove redundant field --- src/Client/HedgedConnectionsFactory.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 0a3ac1e7b47..799e16bb068 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -137,7 +137,6 @@ private: std::vector connection_establishers; std::vector shuffled_pools; - std::vector replica_states; /// Map socket file descriptor to replica. std::unordered_map fd_to_replica; From 12b925dd7a3878b9861c593fda973bd9ee54312a Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 8 Feb 2021 14:06:45 +0300 Subject: [PATCH 075/510] Small update --- src/Client/HedgedConnections.cpp | 4 ++-- src/Client/HedgedConnectionsFactory.cpp | 19 +++++++------------ src/Client/HedgedConnectionsFactory.h | 12 +++++------- .../integration/test_hedged_requests/test.py | 2 +- .../test_hedged_requests_parallel/test.py | 2 +- 5 files changed, 16 insertions(+), 23 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 65100a7ea41..bba17f9dcad 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -409,13 +409,13 @@ void HedgedConnections::processTimeoutEvent(ReplicaLocation & replica_location, void HedgedConnections::tryGetNewReplica(bool start_new_connection) { Connection * connection = nullptr; - HedgedConnectionsFactory::State state = hedged_connections_factory.getNextConnection(start_new_connection, connection); + HedgedConnectionsFactory::State state = hedged_connections_factory.getNextConnection(start_new_connection, false, connection); /// Skip replicas that doesn't support two-level aggregation if we didn't disable it in sendQuery. while (state == HedgedConnectionsFactory::State::READY && !disable_two_level_aggregation && connection->getServerRevision(hedged_connections_factory.getConnectionTimeouts()) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) - state = hedged_connections_factory.getNextConnection(true, connection); + state = hedged_connections_factory.getNextConnection(true, false, connection); if (state == HedgedConnectionsFactory::State::READY) { diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 84848949fb9..84f0384f377 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -76,7 +76,7 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode Connection * connection = nullptr; while (connections.size() < max_entries) { - auto state = processConnections(true, connection); + auto state = getNextConnection(false, true, connection); if (state == State::READY) connections.push_back(connection); else if (state == State::CANNOT_CHOOSE) @@ -100,15 +100,18 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode return connections; } -HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, Connection *& connection_out) +HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out) { + ReplicaStatePtr replica = nullptr; + int index = -1; + if (start_new_connection) { /// Try to start establishing connection to the new replica. - int index = getNextIndex(); + index = getNextIndex(); if (index != -1) { - ReplicaStatePtr replica = startEstablishingConnection(index); + replica = startEstablishingConnection(index); if (replica->state == State::READY) { connection_out = replica->connection; @@ -117,14 +120,6 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool } } - return processConnections(false, connection_out); -} - -HedgedConnectionsFactory::State HedgedConnectionsFactory::processConnections(bool blocking, Connection *& connection_out) -{ - ReplicaStatePtr replica = nullptr; - int index = -1; - while (index != -1 || !epoll.empty()) { if (index != -1) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 799e16bb068..dbafed7f3d1 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -74,14 +74,12 @@ public: /// Create and return active connections according to pool_mode. std::vector getManyConnections(PoolMode pool_mode); - /// Try to get connection to the new replica without blocking. If start_new_connection is true, we start establishing connection - /// with the new replica and then call processConnections, otherwise just call processConnections. - State getNextConnection(bool start_new_connection, Connection *& connection_out); - - /// Process all current events in epoll (connections, timeouts), if there is no events in epoll and blocking is false, - /// return NOT_READY. Returned state might be READY, NOT_READY and CANNOT_CHOOSE. + /// Try to get connection to the new replica. If start_new_connection is true, we start establishing connection + /// with the new replica. Process all current events in epoll (connections, timeouts), + /// if there is no events in epoll and blocking is false, return NOT_READY. + /// Returned state might be READY, NOT_READY and CANNOT_CHOOSE. /// If state is READY, replica connection will be written in connection_out. - State processConnections(bool blocking, Connection *& connection_out); + State getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out); /// Check if we can try to produce new READY replica. bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); } diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 992590b516f..20602b1af0a 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -61,7 +61,7 @@ def process_test(sleep_setting_name, receive_timeout_name): print(query_time) -def test(started_cluster): +def test_hedged_requests(started_cluster): node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") process_test("sleep_before_send_hello", "receive_hello_timeout") diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index b662fc9d80c..08c5c0d3cd1 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -41,7 +41,7 @@ def started_cluster(): finally: cluster.shutdown() -def test(started_cluster): +def test_hedged_requests_with_max_parallel_replicas(started_cluster): node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") # Without hedged requests select query will last more 30 seconds, From 8efee9ed9a5db0d4cc773b7bf60760160bb8b79c Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 8 Feb 2021 15:40:23 +0300 Subject: [PATCH 076/510] DOCSUP-5822: IN oper - supports diff types. --- docs/en/sql-reference/operators/in.md | 20 +++++++++++++++++++- docs/ru/sql-reference/operators/in.md | 18 +++++++++++++++++- 2 files changed, 36 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index bfa8b3d1003..5f928f12024 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -13,10 +13,28 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... If the left side is a single column that is in the index, and the right side is a set of constants, the system uses the index for processing the query. -Don’t list too many values explicitly (i.e. millions). If a data set is large, put it in a temporary table (for example, see the section “External data for query processing”), then use a subquery. +Don’t list too many values explicitly (i.e. millions). If a data set is large, put it in a temporary table (for example, see the section [External data for query processing](../../engines/table-engines/special/external-data.md)), then use a subquery. The right side of the operator can be a set of constant expressions, a set of tuples with constant expressions (shown in the examples above), or the name of a database table or SELECT subquery in brackets. +ClickHouse allows different types inside `IN` subquery. For left hand side it applies type conversion to the type of right hand side. + +**Example** + +Query: + +``` sql +SELECT '1' IN (SELECT 1); +``` + +Result: + +``` text +┌─in('1', _subquery49)─┐ +│ 1 │ +└──────────────────────┘ +``` + If the right side of the operator is the name of a table (for example, `UserID IN users`), this is equivalent to the subquery `UserID IN (SELECT * FROM users)`. Use this when working with external data that is sent along with the query. For example, the query can be sent together with a set of user IDs loaded to the ‘users’ temporary table, which should be filtered. If the right side of the operator is a table name that has the Set engine (a prepared data set that is always in RAM), the data set will not be created over again for each query. diff --git a/docs/ru/sql-reference/operators/in.md b/docs/ru/sql-reference/operators/in.md index 4c1290df166..5a4fe95f108 100644 --- a/docs/ru/sql-reference/operators/in.md +++ b/docs/ru/sql-reference/operators/in.md @@ -13,10 +13,26 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... Если слева стоит один столбец, входящий в индекс, а справа - множество констант, то при выполнении запроса, система воспользуется индексом. -Не перечисляйте слишком большое количество значений (миллионы) явно. Если множество большое - лучше загрузить его во временную таблицу (например, смотрите раздел «Внешние данные для обработки запроса»), и затем воспользоваться подзапросом. +Не перечисляйте слишком большое количество значений (миллионы) явно. Если множество большое - лучше загрузить его во временную таблицу (например, смотрите раздел [Внешние данные для обработки запроса](../../engines/table-engines/special/external-data.md)), и затем воспользоваться подзапросом. В качестве правой части оператора может быть множество константных выражений, множество кортежей с константными выражениями (показано в примерах выше), а также имя таблицы или подзапрос SELECT в скобках. +**Пример** + +Запрос: + +``` sql +SELECT '1' IN (SELECT 1); +``` + +Результат: + +``` text +┌─in('1', _subquery49)─┐ +│ 1 │ +└──────────────────────┘ +``` + Если в качестве правой части оператора указано имя таблицы (например, `UserID IN users`), то это эквивалентно подзапросу `UserID IN (SELECT * FROM users)`. Это используется при работе с внешними данными, отправляемым вместе с запросом. Например, вместе с запросом может быть отправлено множество идентификаторов посетителей, загруженное во временную таблицу users, по которому следует выполнить фильтрацию. Если в качестве правой части оператора, указано имя таблицы, имеющий движок Set (подготовленное множество, постоянно находящееся в оперативке), то множество не будет создаваться заново при каждом запросе. From 5647f0eb8c25fc302179661d77e27e8d5e7bf479 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 8 Feb 2021 15:51:33 +0300 Subject: [PATCH 077/510] DOCSUP-5822: IN oper - supports diff types. --- docs/en/sql-reference/operators/in.md | 2 +- docs/ru/sql-reference/operators/in.md | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 5f928f12024..1b6531a57f8 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -17,7 +17,7 @@ Don’t list too many values explicitly (i.e. millions). If a data set is large The right side of the operator can be a set of constant expressions, a set of tuples with constant expressions (shown in the examples above), or the name of a database table or SELECT subquery in brackets. -ClickHouse allows different types inside `IN` subquery. For left hand side it applies type conversion to the type of right hand side. +ClickHouse allows different types inside `IN` subquery. For left hand side it applies type conversion to the type of right hand side with [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). **Example** diff --git a/docs/ru/sql-reference/operators/in.md b/docs/ru/sql-reference/operators/in.md index 5a4fe95f108..d86d6f9ec57 100644 --- a/docs/ru/sql-reference/operators/in.md +++ b/docs/ru/sql-reference/operators/in.md @@ -17,6 +17,8 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... В качестве правой части оператора может быть множество константных выражений, множество кортежей с константными выражениями (показано в примерах выше), а также имя таблицы или подзапрос SELECT в скобках. +ClickHouse допускает различные типы внутри подзапроса `IN`. Для левой стороны он применяет преобразование к типу правой стороны с помощью [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). + **Пример** Запрос: From 78f5f416171a192c4c6dbad4dd79d069be389a43 Mon Sep 17 00:00:00 2001 From: romanzhukov Date: Mon, 8 Feb 2021 15:55:53 +0300 Subject: [PATCH 078/510] DOCSUP-5822: Minor text fix. --- docs/en/sql-reference/operators/in.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 1b6531a57f8..a0dd0455c4d 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -17,7 +17,7 @@ Don’t list too many values explicitly (i.e. millions). If a data set is large The right side of the operator can be a set of constant expressions, a set of tuples with constant expressions (shown in the examples above), or the name of a database table or SELECT subquery in brackets. -ClickHouse allows different types inside `IN` subquery. For left hand side it applies type conversion to the type of right hand side with [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). +ClickHouse allows different types inside `IN` subquery. For left hand side it applies conversion to the type of right hand side with [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). **Example** From d3ebf59376f42efef5e5341c04f182cdcd11f51c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 8 Feb 2021 16:08:15 +0300 Subject: [PATCH 079/510] Fix --- src/Client/HedgedConnections.cpp | 152 ++++++++++++------------ src/Client/HedgedConnections.h | 37 +++--- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Client/HedgedConnectionsFactory.h | 2 +- 4 files changed, 94 insertions(+), 99 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index bba17f9dcad..8ac79f5cb44 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -27,28 +27,30 @@ HedgedConnections::HedgedConnections( { std::vector connections = hedged_connections_factory.getManyConnections(pool_mode); - ReplicaState replica; + ReplicaStatePtr replica = nullptr; for (size_t i = 0; i != connections.size(); ++i) { - replica.connection = connections[i]; - replica.connection->setThrottler(throttler_); - int socket_fd = replica.connection->getSocket()->impl()->sockfd(); + replica = std::make_shared(); + replica->connection = connections[i]; + replica->offset = i; + replica->connection->setThrottler(throttler_); + int socket_fd = replica->connection->getSocket()->impl()->sockfd(); epoll.add(socket_fd); - fd_to_replica_location[socket_fd] = ReplicaLocation{i, 0}; - offset_states.push_back(OffsetState{{replica}, 1, false}); + fd_to_replica[socket_fd] = replica; + offset_states.push_back(OffsetState{{std::move(replica)}, 1, false}); } active_connection_count = connections.size(); offsets_with_received_first_data_packet = 0; - pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); }); + pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr & replica_) { replica_->connection->setThrottler(throttler_); }); } -void HedgedConnections::Pipeline::add(std::function send_function) +void HedgedConnections::Pipeline::add(std::function send_function) { pipeline.push_back(send_function); } -void HedgedConnections::Pipeline::run(ReplicaState & replica) +void HedgedConnections::Pipeline::run(ReplicaStatePtr & replica) { for (auto & send_func : pipeline) send_func(replica); @@ -61,11 +63,11 @@ void HedgedConnections::sendScalarsData(Scalars & data) if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); - auto send_scalars_data = [&data](ReplicaState & replica) { replica.connection->sendScalarsData(data); }; + auto send_scalars_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendScalarsData(data); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) - if (replica.connection) + if (replica->connection) send_scalars_data(replica); pipeline_for_new_replicas.add(send_scalars_data); @@ -81,11 +83,11 @@ void HedgedConnections::sendExternalTablesData(std::vector & if (data.size() != size()) throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES); - auto send_external_tables_data = [&data](ReplicaState & replica) { replica.connection->sendExternalTablesData(data[0]); }; + auto send_external_tables_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendExternalTablesData(data[0]); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) - if (replica.connection) + if (replica->connection) send_external_tables_data(replica); pipeline_for_new_replicas.add(send_external_tables_data); @@ -98,11 +100,11 @@ void HedgedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) if (sent_query) throw Exception("Cannot send uuids after query is sent.", ErrorCodes::LOGICAL_ERROR); - auto send_ignored_part_uuids = [&uuids](ReplicaState & replica) { replica.connection->sendIgnoredPartUUIDs(uuids); }; + auto send_ignored_part_uuids = [&uuids](ReplicaStatePtr & replica) { replica->connection->sendIgnoredPartUUIDs(uuids); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) - if (replica.connection) + if (replica->connection) send_ignored_part_uuids(replica); pipeline_for_new_replicas.add(send_ignored_part_uuids); @@ -125,7 +127,7 @@ void HedgedConnections::sendQuery( { for (auto & replica : offset_state.replicas) { - if (replica.connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) { disable_two_level_aggregation = true; break; @@ -135,8 +137,7 @@ void HedgedConnections::sendQuery( break; } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) - { + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica) { Settings modified_settings = settings; if (disable_two_level_aggregation) @@ -149,10 +150,10 @@ void HedgedConnections::sendQuery( if (offset_states.size() > 1) { modified_settings.parallel_replicas_count = offset_states.size(); - modified_settings.parallel_replica_offset = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()].offset; + modified_settings.parallel_replica_offset = replica->offset; } - replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); + replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, replica); }; @@ -171,7 +172,7 @@ void HedgedConnections::disconnect() for (auto & offset_status : offset_states) for (auto & replica : offset_status.replicas) - if (replica.connection) + if (replica->connection) finishProcessReplica(replica, true); if (hedged_connections_factory.hasEventsInProcess()) @@ -197,9 +198,9 @@ std::string HedgedConnections::dumpAddresses() const { for (const auto & replica : offset_state.replicas) { - if (replica.connection) + if (replica->connection) { - addresses += (is_first ? "" : "; ") + replica.connection->getDescription(); + addresses += (is_first ? "" : "; ") + replica->connection->getDescription(); is_first = false; } } @@ -217,8 +218,8 @@ void HedgedConnections::sendCancel() for (auto & offset_status : offset_states) for (auto & replica : offset_status.replicas) - if (replica.connection) - replica.connection->sendCancel(); + if (replica->connection) + replica->connection->sendCancel(); cancelled = true; } @@ -286,15 +287,15 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { event_fd = getReadyFileDescriptor(async_callback); - if (fd_to_replica_location.contains(event_fd)) + if (fd_to_replica.contains(event_fd)) { - packet = receivePacketFromReplica(fd_to_replica_location[event_fd], async_callback); + packet = receivePacketFromReplica(fd_to_replica[event_fd], async_callback); finish = true; } - else if (timeout_fd_to_replica_location.contains(event_fd)) + else if (timeout_fd_to_replica.contains(event_fd)) { - ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; - processTimeoutEvent(location, offset_states[location.offset].replicas[location.index].active_timeouts[event_fd]); + ReplicaStatePtr & replica = timeout_fd_to_replica[event_fd]; + processTimeoutEvent(replica, replica->active_timeouts[event_fd]); } else if (event_fd == hedged_connections_factory.getFileDescriptor()) tryGetNewReplica(false); @@ -307,11 +308,10 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { - for (auto & [fd, location] : fd_to_replica_location) + for (auto & [fd, replica] : fd_to_replica) { - ReplicaState & replica = offset_states[location.offset].replicas[location.index]; - if (replica.connection->hasReadPendingData()) - return replica.connection->getSocket()->impl()->sockfd(); + if (replica->connection->hasReadPendingData()) + return replica->connection->getSocket()->impl()->sockfd(); } epoll_event event; @@ -320,16 +320,15 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) return event.data.fd; } -Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback) +Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) { - ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); - Packet packet = replica.connection->receivePacket(std::move(async_callback)); + Packet packet = replica->connection->receivePacket(std::move(async_callback)); switch (packet.type) { case Protocol::Server::Data: - if (!offset_states[replica_location.offset].first_packet_of_data_received) - processReceivedFirstDataPacket(replica_location); + if (!offset_states[replica->offset].first_packet_of_data_received) + processReceivedFirstDataPacket(replica); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); break; case Protocol::Server::PartUUIDs: @@ -354,21 +353,21 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_loc return packet; } -void HedgedConnections::processReceivedFirstDataPacket(ReplicaLocation & replica_location) +void HedgedConnections::processReceivedFirstDataPacket(ReplicaStatePtr & replica) { /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. - OffsetState & offset_state = offset_states[replica_location.offset]; - removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, offset_state.replicas[replica_location.index]); + OffsetState & offset_state = offset_states[replica->offset]; + removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, replica); ++offsets_with_received_first_data_packet; offset_state.first_packet_of_data_received = true; - for (size_t i = 0; i != offset_state.replicas.size(); ++i) + for (auto & other_replica : offset_state.replicas) { - if (i != replica_location.index && offset_state.replicas[i].connection) + if (replica != other_replica && other_replica->connection) { - offset_state.replicas[i].connection->sendCancel(); - finishProcessReplica(offset_state.replicas[i], true); + other_replica->connection->sendCancel(); + finishProcessReplica(other_replica, true); } } @@ -384,24 +383,23 @@ void HedgedConnections::processReceivedFirstDataPacket(ReplicaLocation & replica } } -void HedgedConnections::processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor) +void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) { - ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; epoll.remove(timeout_descriptor->timer.getDescriptor()); - replica.active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica_location.erase(timeout_descriptor->timer.getDescriptor()); + replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica.erase(timeout_descriptor->timer.getDescriptor()); if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) { finishProcessReplica(replica, true); /// Check if there is no active connections with the same offset and there is no new replica in process. - if (offset_states[replica_location.offset].active_connection_count == 0 && !next_replica_in_process) + if (offset_states[replica->offset].active_connection_count == 0 && !next_replica_in_process) throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); } else if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT) { - offsets_queue.push(replica_location.offset); + offsets_queue.push(replica->offset); tryGetNewReplica(true); } } @@ -421,14 +419,14 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) { size_t offset = offsets_queue.front(); offsets_queue.pop(); - size_t index = offset_states[offset].replicas.size(); - ReplicaState replica; - replica.connection = connection; - int socket_fd = replica.connection->getSocket()->impl()->sockfd(); + ReplicaStatePtr replica = std::make_shared(); + replica->connection = connection; + replica->offset = offset; + int socket_fd = replica->connection->getSocket()->impl()->sockfd(); epoll.add(socket_fd); - fd_to_replica_location[socket_fd] = ReplicaLocation{offset, index}; - offset_states[offset].replicas.push_back(replica); + fd_to_replica[socket_fd] = replica; + offset_states[offset].replicas.push_back(std::move(replica)); ++offset_states[offset].active_connection_count; ++active_connection_count; pipeline_for_new_replicas.run(replica); @@ -458,50 +456,50 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) } } -void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) +void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect) { removeTimeoutsFromReplica(replica); - int socket_fd = replica.connection->getSocket()->impl()->sockfd(); + int socket_fd = replica->connection->getSocket()->impl()->sockfd(); epoll.remove(socket_fd); - --offset_states[fd_to_replica_location[socket_fd].offset].active_connection_count; - fd_to_replica_location.erase(socket_fd); + --offset_states[replica->offset].active_connection_count; + fd_to_replica.erase(socket_fd); --active_connection_count; if (disconnect) - replica.connection->disconnect(); - replica.connection = nullptr; + replica->connection->disconnect(); + replica->connection = nullptr; } -void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica) +void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica) { ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, hedged_connections_factory.getConnectionTimeouts()); epoll.add(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica_location[timeout_descriptor->timer.getDescriptor()] - = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()]; - replica.active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); + timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()] + = fd_to_replica[replica->connection->getSocket()->impl()->sockfd()]; + replica->active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); } -void HedgedConnections::removeTimeoutsFromReplica(ReplicaState & replica) +void HedgedConnections::removeTimeoutsFromReplica(ReplicaStatePtr & replica) { - for (auto & [fd, _] : replica.active_timeouts) + for (auto & [fd, _] : replica->active_timeouts) { epoll.remove(fd); - timeout_fd_to_replica_location.erase(fd); + timeout_fd_to_replica.erase(fd); } - replica.active_timeouts.clear(); + replica->active_timeouts.clear(); } -void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica) +void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica) { auto it = std::find_if( - replica.active_timeouts.begin(), replica.active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); + replica->active_timeouts.begin(), replica->active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); - if (it != replica.active_timeouts.end()) + if (it != replica->active_timeouts.end()) { epoll.remove(it->first); - timeout_fd_to_replica_location.erase(it->first); - replica.active_timeouts.erase(it); + timeout_fd_to_replica.erase(it->first); + replica->active_timeouts.erase(it); } } diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index eb73f2ded52..56eca3ffbe7 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -22,17 +22,14 @@ public: { Connection * connection = nullptr; std::unordered_map active_timeouts; + size_t offset = 0; }; - struct ReplicaLocation - { - size_t offset; - size_t index; - }; + using ReplicaStatePtr = std::shared_ptr; struct OffsetState { - std::vector replicas; + std::vector replicas; size_t active_connection_count; bool first_packet_of_data_received; }; @@ -79,32 +76,32 @@ private: class Pipeline { public: - void add(std::function send_function); + void add(std::function send_function); - void run(ReplicaState & replica); + void run(ReplicaStatePtr & replica); private: - std::vector> pipeline; + std::vector> pipeline; }; - Packet receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback = {}); Packet receivePacketImpl(AsyncCallback async_callback = {}); - void processReceivedFirstDataPacket(ReplicaLocation & replica_location); + void processReceivedFirstDataPacket(ReplicaStatePtr & replica); - void processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor); + void processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); void tryGetNewReplica(bool start_new_connection); - void finishProcessReplica(ReplicaState & replica, bool disconnect); + void finishProcessReplica(ReplicaStatePtr & replica, bool disconnect); int getReadyFileDescriptor(AsyncCallback async_callback = {}); - void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica); + void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica); - void removeTimeoutsFromReplica(ReplicaState & replica); + void removeTimeoutsFromReplica(ReplicaStatePtr & replica); - void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica); + void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica); HedgedConnectionsFactory hedged_connections_factory; @@ -114,10 +111,10 @@ private: /// replica_states[offset].replicas.size() = 1 (like in MultiplexedConnections). std::vector offset_states; - /// Map socket file descriptor to replica location (it's offset and index in OffsetState.replicas). - std::unordered_map fd_to_replica_location; - /// Map timeout file descriptor to replica location (it's offset and index in OffsetState.replicas). - std::unordered_map timeout_fd_to_replica_location; + /// Map socket file descriptor to replica. + std::unordered_map fd_to_replica; + /// Map timeout file descriptor to replica. + std::unordered_map timeout_fd_to_replica; /// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from /// the replica, we push it's offset to this queue and start trying to get diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 84f0384f377..7b49a351ff6 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -104,7 +104,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool { ReplicaStatePtr replica = nullptr; int index = -1; - + if (start_new_connection) { /// Try to start establishing connection to the new replica. diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index dbafed7f3d1..345a1f2fe3e 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -75,7 +75,7 @@ public: std::vector getManyConnections(PoolMode pool_mode); /// Try to get connection to the new replica. If start_new_connection is true, we start establishing connection - /// with the new replica. Process all current events in epoll (connections, timeouts), + /// with the new replica. Process all current events in epoll (connections, timeouts), /// if there is no events in epoll and blocking is false, return NOT_READY. /// Returned state might be READY, NOT_READY and CANNOT_CHOOSE. /// If state is READY, replica connection will be written in connection_out. From 134a686fa5f2a26bab4dae159bfd5d72bb4e1874 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 8 Feb 2021 16:16:09 +0300 Subject: [PATCH 080/510] Fix 2 --- src/Client/HedgedConnections.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 8ac79f5cb44..9fab9e6ec84 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -426,7 +426,7 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) int socket_fd = replica->connection->getSocket()->impl()->sockfd(); epoll.add(socket_fd); fd_to_replica[socket_fd] = replica; - offset_states[offset].replicas.push_back(std::move(replica)); + offset_states[offset].replicas.push_back(replica); ++offset_states[offset].active_connection_count; ++active_connection_count; pipeline_for_new_replicas.run(replica); From 7b45860b0674ae5a85979f0147de532f4da52f1a Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 8 Feb 2021 17:02:11 +0300 Subject: [PATCH 081/510] Style fix --- src/Client/HedgedConnections.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 9fab9e6ec84..b361f04f0b1 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -137,7 +137,8 @@ void HedgedConnections::sendQuery( break; } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica) { + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica) + { Settings modified_settings = settings; if (disable_two_level_aggregation) From a594c738c2f4c539065f36d160b451c7048fb670 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 8 Feb 2021 17:14:36 +0300 Subject: [PATCH 082/510] Remove code duplication --- src/Client/HedgedConnectionsFactory.cpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 7b49a351ff6..2a5abbbaf57 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -106,19 +106,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool int index = -1; if (start_new_connection) - { - /// Try to start establishing connection to the new replica. index = getNextIndex(); - if (index != -1) - { - replica = startEstablishingConnection(index); - if (replica->state == State::READY) - { - connection_out = replica->connection; - return State::READY; - } - } - } while (index != -1 || !epoll.empty()) { From e252b138420cb9621dbc26aff3ef411d43177161 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 8 Feb 2021 23:54:28 +0300 Subject: [PATCH 083/510] Update simpleaggregatefunction.md Remove output of creating table example. --- .../data-types/simpleaggregatefunction.md | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 7441ceae655..b80826803de 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -32,22 +32,8 @@ **Пример** -Запрос: - ``` sql CREATE TABLE simple (id UInt64,val SimpleAggregateFunction(sum,Double)) ENGINE=AggregatingMergeTree ORDER BY id; ``` -Ответ: - -``` text -CREATE TABLE simple -( - `id` UInt64, - `val` SimpleAggregateFunction(sum, Double) -) -ENGINE = AggregatingMergeTree -ORDER BY id -``` - [Оригинальная статья](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From 436954dc26de1263b9071d530101b9468ac8c2eb Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 8 Feb 2021 23:54:52 +0300 Subject: [PATCH 084/510] Update simpleaggregatefunction.md --- .../data-types/simpleaggregatefunction.md | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index 9ea5a586981..e25d4803613 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -34,22 +34,8 @@ The following aggregate functions are supported: **Example** -Query: - ``` sql CREATE TABLE simple (id UInt64,val SimpleAggregateFunction(sum,Double)) ENGINE=AggregatingMergeTree ORDER BY id; ``` -Result: - -``` text -CREATE TABLE simple -( - `id` UInt64, - `val` SimpleAggregateFunction(sum, Double) -) -ENGINE = AggregatingMergeTree -ORDER BY id -``` - [Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From df5dc102c47cde5112186a9f7b0d62b0e6b574ee Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 8 Feb 2021 22:31:21 +0300 Subject: [PATCH 085/510] Don't use 'ReplicaStatePtr &' to prevent use-after-free --- src/Client/HedgedConnections.cpp | 30 ++++++++++++------------- src/Client/HedgedConnections.h | 20 ++++++++--------- src/Client/HedgedConnectionsFactory.cpp | 16 ++++++------- src/Client/HedgedConnectionsFactory.h | 18 +++++++-------- 4 files changed, 42 insertions(+), 42 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index b361f04f0b1..8e547169f29 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -42,15 +42,15 @@ HedgedConnections::HedgedConnections( active_connection_count = connections.size(); offsets_with_received_first_data_packet = 0; - pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr & replica_) { replica_->connection->setThrottler(throttler_); }); + pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr replica_) { replica_->connection->setThrottler(throttler_); }); } -void HedgedConnections::Pipeline::add(std::function send_function) +void HedgedConnections::Pipeline::add(std::function send_function) { pipeline.push_back(send_function); } -void HedgedConnections::Pipeline::run(ReplicaStatePtr & replica) +void HedgedConnections::Pipeline::run(ReplicaStatePtr replica) { for (auto & send_func : pipeline) send_func(replica); @@ -63,7 +63,7 @@ void HedgedConnections::sendScalarsData(Scalars & data) if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); - auto send_scalars_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendScalarsData(data); }; + auto send_scalars_data = [&data](ReplicaStatePtr replica) { replica->connection->sendScalarsData(data); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) @@ -83,7 +83,7 @@ void HedgedConnections::sendExternalTablesData(std::vector & if (data.size() != size()) throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES); - auto send_external_tables_data = [&data](ReplicaStatePtr & replica) { replica->connection->sendExternalTablesData(data[0]); }; + auto send_external_tables_data = [&data](ReplicaStatePtr replica) { replica->connection->sendExternalTablesData(data[0]); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) @@ -100,7 +100,7 @@ void HedgedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) if (sent_query) throw Exception("Cannot send uuids after query is sent.", ErrorCodes::LOGICAL_ERROR); - auto send_ignored_part_uuids = [&uuids](ReplicaStatePtr & replica) { replica->connection->sendIgnoredPartUUIDs(uuids); }; + auto send_ignored_part_uuids = [&uuids](ReplicaStatePtr replica) { replica->connection->sendIgnoredPartUUIDs(uuids); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) @@ -137,7 +137,7 @@ void HedgedConnections::sendQuery( break; } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr & replica) + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr replica) { Settings modified_settings = settings; @@ -295,7 +295,7 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) } else if (timeout_fd_to_replica.contains(event_fd)) { - ReplicaStatePtr & replica = timeout_fd_to_replica[event_fd]; + ReplicaStatePtr replica = timeout_fd_to_replica[event_fd]; processTimeoutEvent(replica, replica->active_timeouts[event_fd]); } else if (event_fd == hedged_connections_factory.getFileDescriptor()) @@ -321,7 +321,7 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) return event.data.fd; } -Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback) +Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback) { removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); Packet packet = replica->connection->receivePacket(std::move(async_callback)); @@ -354,7 +354,7 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr & replica, As return packet; } -void HedgedConnections::processReceivedFirstDataPacket(ReplicaStatePtr & replica) +void HedgedConnections::processReceivedFirstDataPacket(ReplicaStatePtr replica) { /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. @@ -384,7 +384,7 @@ void HedgedConnections::processReceivedFirstDataPacket(ReplicaStatePtr & replica } } -void HedgedConnections::processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) +void HedgedConnections::processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) { epoll.remove(timeout_descriptor->timer.getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); @@ -457,7 +457,7 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) } } -void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool disconnect) +void HedgedConnections::finishProcessReplica(ReplicaStatePtr replica, bool disconnect) { removeTimeoutsFromReplica(replica); int socket_fd = replica->connection->getSocket()->impl()->sockfd(); @@ -471,7 +471,7 @@ void HedgedConnections::finishProcessReplica(ReplicaStatePtr & replica, bool dis replica->connection = nullptr; } -void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica) +void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica) { ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, hedged_connections_factory.getConnectionTimeouts()); @@ -481,7 +481,7 @@ void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaS replica->active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); } -void HedgedConnections::removeTimeoutsFromReplica(ReplicaStatePtr & replica) +void HedgedConnections::removeTimeoutsFromReplica(ReplicaStatePtr replica) { for (auto & [fd, _] : replica->active_timeouts) { @@ -491,7 +491,7 @@ void HedgedConnections::removeTimeoutsFromReplica(ReplicaStatePtr & replica) replica->active_timeouts.clear(); } -void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica) +void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr replica) { auto it = std::find_if( replica->active_timeouts.begin(), replica->active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 56eca3ffbe7..00145544096 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -76,32 +76,32 @@ private: class Pipeline { public: - void add(std::function send_function); + void add(std::function send_function); - void run(ReplicaStatePtr & replica); + void run(ReplicaStatePtr replica); private: - std::vector> pipeline; + std::vector> pipeline; }; - Packet receivePacketFromReplica(ReplicaStatePtr & replica, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback = {}); Packet receivePacketImpl(AsyncCallback async_callback = {}); - void processReceivedFirstDataPacket(ReplicaStatePtr & replica); + void processReceivedFirstDataPacket(ReplicaStatePtr replica); - void processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); + void processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); void tryGetNewReplica(bool start_new_connection); - void finishProcessReplica(ReplicaStatePtr & replica, bool disconnect); + void finishProcessReplica(ReplicaStatePtr replica, bool disconnect); int getReadyFileDescriptor(AsyncCallback async_callback = {}); - void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica); + void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica); - void removeTimeoutsFromReplica(ReplicaStatePtr & replica); + void removeTimeoutsFromReplica(ReplicaStatePtr replica); - void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica); + void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr replica); HedgedConnectionsFactory hedged_connections_factory; diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 2a5abbbaf57..732e1e4b7d0 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -227,7 +227,7 @@ HedgedConnectionsFactory::ReplicaStatePtr HedgedConnectionsFactory::startEstabli return replica; } -void HedgedConnectionsFactory::processConnectionEstablisherStage(ReplicaStatePtr & replica, bool remove_from_epoll) +void HedgedConnectionsFactory::processConnectionEstablisherStage(ReplicaStatePtr replica, bool remove_from_epoll) { ConnectionEstablisher & connection_establisher = connection_establishers[replica->index]; @@ -260,7 +260,7 @@ void HedgedConnectionsFactory::processConnectionEstablisherStage(ReplicaStatePtr processFailedConnection(replica); } -void HedgedConnectionsFactory::processFailedConnection(ReplicaStatePtr & replica) +void HedgedConnectionsFactory::processFailedConnection(ReplicaStatePtr replica) { ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; LOG_WARNING( @@ -283,7 +283,7 @@ void HedgedConnectionsFactory::processFailedConnection(ReplicaStatePtr & replica replica->reset(); } -void HedgedConnectionsFactory::addTimeouts(ReplicaStatePtr & replica) +void HedgedConnectionsFactory::addTimeouts(ReplicaStatePtr replica) { addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); @@ -294,7 +294,7 @@ void HedgedConnectionsFactory::addTimeouts(ReplicaStatePtr & replica) addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT, replica); } -void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica) +void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica) { ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, timeouts); epoll.add(timeout_descriptor->timer.getDescriptor()); @@ -302,7 +302,7 @@ void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, R replica->active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); } -void HedgedConnectionsFactory::removeTimeoutsFromReplica(ReplicaStatePtr & replica) +void HedgedConnectionsFactory::removeTimeoutsFromReplica(ReplicaStatePtr replica) { for (auto & [fd, _] : replica->active_timeouts) { @@ -359,7 +359,7 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) return event.data.fd; } -void HedgedConnectionsFactory::processReplicaEvent(ReplicaStatePtr & replica) +void HedgedConnectionsFactory::processReplicaEvent(ReplicaStatePtr replica) { removeTimeoutsFromReplica(replica); connection_establishers[replica->index].run(); @@ -368,7 +368,7 @@ void HedgedConnectionsFactory::processReplicaEvent(ReplicaStatePtr & replica) addTimeouts(replica); } -void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) +void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) { epoll.remove(timeout_descriptor->timer.getDescriptor()); replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); @@ -393,7 +393,7 @@ void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr & replica, Co replica = createNewReplica(); } -void HedgedConnectionsFactory::setBestUsableReplica(ReplicaStatePtr & replica) +void HedgedConnectionsFactory::setBestUsableReplica(ReplicaStatePtr replica) { std::vector indexes(connection_establishers.size()); for (size_t i = 0; i != indexes.size(); ++i) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 345a1f2fe3e..398629cf13c 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -98,7 +98,7 @@ public: private: ReplicaStatePtr startEstablishingConnection(int index); - void processConnectionEstablisherStage(ReplicaStatePtr & replica, bool remove_from_epoll = false); + void processConnectionEstablisherStage(ReplicaStatePtr replica, bool remove_from_epoll = false); /// Find an index of the next free replica to start connection. /// Return -1 if there is no free replica. @@ -106,25 +106,25 @@ private: int getReadyFileDescriptor(bool blocking); - void addTimeouts(ReplicaStatePtr & replica); + void addTimeouts(ReplicaStatePtr replica); - void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr & replica); + void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica); - void removeTimeoutsFromReplica(ReplicaStatePtr & replica); + void removeTimeoutsFromReplica(ReplicaStatePtr replica); - void processFailedConnection(ReplicaStatePtr & replica); + void processFailedConnection(ReplicaStatePtr replica); - void processReceiveTimeout(ReplicaStatePtr & replica); + void processReceiveTimeout(ReplicaStatePtr replica); - void processReplicaEvent(ReplicaStatePtr & replica); + void processReplicaEvent(ReplicaStatePtr replica); - void processTimeoutEvent(ReplicaStatePtr & replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); + void processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); /// Return false if there is no ready events, return true if replica is ready /// or we need to try next replica. bool processEpollEvents(ReplicaStatePtr & replica, bool blocking); - void setBestUsableReplica(ReplicaStatePtr & replica); + void setBestUsableReplica(ReplicaStatePtr replica); ReplicaStatePtr createNewReplica() { return std::make_shared(); } From 46840b0a4f9211f997fed85330968584502904fc Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 9 Feb 2021 00:47:13 +0300 Subject: [PATCH 086/510] Add __init__.py --- tests/integration/test_hedged_requests/__init__.py | 0 tests/integration/test_hedged_requests_parallel/__init__.py | 0 2 files changed, 0 insertions(+), 0 deletions(-) create mode 100644 tests/integration/test_hedged_requests/__init__.py create mode 100644 tests/integration/test_hedged_requests_parallel/__init__.py diff --git a/tests/integration/test_hedged_requests/__init__.py b/tests/integration/test_hedged_requests/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_hedged_requests_parallel/__init__.py b/tests/integration/test_hedged_requests_parallel/__init__.py new file mode 100644 index 00000000000..e69de29bb2d From 22b8bc6c99fafcd653884cfc9ac471ac1a81d9f4 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 9 Feb 2021 05:01:09 +0300 Subject: [PATCH 087/510] Refactor 2.0 --- src/Client/HedgedConnections.cpp | 155 ++++++++-------- src/Client/HedgedConnections.h | 38 ++-- src/Client/HedgedConnectionsFactory.cpp | 224 +++++++++++------------- src/Client/HedgedConnectionsFactory.h | 52 ++---- 4 files changed, 219 insertions(+), 250 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 8e547169f29..32a91af6179 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -23,34 +23,31 @@ HedgedConnections::HedgedConnections( : hedged_connections_factory(pool_, &settings_, timeouts_, table_to_check_) , settings(settings_) , throttler(throttler_) - , log(&Poco::Logger::get("HedgedConnections")) { std::vector connections = hedged_connections_factory.getManyConnections(pool_mode); - ReplicaStatePtr replica = nullptr; + ReplicaState replica; for (size_t i = 0; i != connections.size(); ++i) { - replica = std::make_shared(); - replica->connection = connections[i]; - replica->offset = i; - replica->connection->setThrottler(throttler_); - int socket_fd = replica->connection->getSocket()->impl()->sockfd(); + replica.connection = connections[i]; + replica.connection->setThrottler(throttler_); + int socket_fd = replica.connection->getSocket()->impl()->sockfd(); epoll.add(socket_fd); - fd_to_replica[socket_fd] = replica; - offset_states.push_back(OffsetState{{std::move(replica)}, 1, false}); + fd_to_replica_location[socket_fd] = ReplicaLocation{i, 0}; + offset_states.push_back(OffsetState{{replica}, 1, false}); } active_connection_count = connections.size(); offsets_with_received_first_data_packet = 0; - pipeline_for_new_replicas.add([throttler_](ReplicaStatePtr replica_) { replica_->connection->setThrottler(throttler_); }); + pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); }); } -void HedgedConnections::Pipeline::add(std::function send_function) +void HedgedConnections::Pipeline::add(std::function send_function) { pipeline.push_back(send_function); } -void HedgedConnections::Pipeline::run(ReplicaStatePtr replica) +void HedgedConnections::Pipeline::run(ReplicaState & replica) { for (auto & send_func : pipeline) send_func(replica); @@ -63,11 +60,11 @@ void HedgedConnections::sendScalarsData(Scalars & data) if (!sent_query) throw Exception("Cannot send scalars data: query not yet sent.", ErrorCodes::LOGICAL_ERROR); - auto send_scalars_data = [&data](ReplicaStatePtr replica) { replica->connection->sendScalarsData(data); }; + auto send_scalars_data = [&data](ReplicaState & replica) { replica.connection->sendScalarsData(data); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) - if (replica->connection) + if (replica.connection) send_scalars_data(replica); pipeline_for_new_replicas.add(send_scalars_data); @@ -83,11 +80,11 @@ void HedgedConnections::sendExternalTablesData(std::vector & if (data.size() != size()) throw Exception("Mismatch between replicas and data sources", ErrorCodes::MISMATCH_REPLICAS_DATA_SOURCES); - auto send_external_tables_data = [&data](ReplicaStatePtr replica) { replica->connection->sendExternalTablesData(data[0]); }; + auto send_external_tables_data = [&data](ReplicaState & replica) { replica.connection->sendExternalTablesData(data[0]); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) - if (replica->connection) + if (replica.connection) send_external_tables_data(replica); pipeline_for_new_replicas.add(send_external_tables_data); @@ -100,11 +97,11 @@ void HedgedConnections::sendIgnoredPartUUIDs(const std::vector & uuids) if (sent_query) throw Exception("Cannot send uuids after query is sent.", ErrorCodes::LOGICAL_ERROR); - auto send_ignored_part_uuids = [&uuids](ReplicaStatePtr replica) { replica->connection->sendIgnoredPartUUIDs(uuids); }; + auto send_ignored_part_uuids = [&uuids](ReplicaState & replica) { replica.connection->sendIgnoredPartUUIDs(uuids); }; for (auto & offset_state : offset_states) for (auto & replica : offset_state.replicas) - if (replica->connection) + if (replica.connection) send_ignored_part_uuids(replica); pipeline_for_new_replicas.add(send_ignored_part_uuids); @@ -127,7 +124,7 @@ void HedgedConnections::sendQuery( { for (auto & replica : offset_state.replicas) { - if (replica->connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + if (replica.connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) { disable_two_level_aggregation = true; break; @@ -137,7 +134,7 @@ void HedgedConnections::sendQuery( break; } - auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaStatePtr replica) + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) { Settings modified_settings = settings; @@ -151,10 +148,10 @@ void HedgedConnections::sendQuery( if (offset_states.size() > 1) { modified_settings.parallel_replicas_count = offset_states.size(); - modified_settings.parallel_replica_offset = replica->offset; + modified_settings.parallel_replica_offset = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()].offset; } - replica->connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); + replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, replica); }; @@ -173,7 +170,7 @@ void HedgedConnections::disconnect() for (auto & offset_status : offset_states) for (auto & replica : offset_status.replicas) - if (replica->connection) + if (replica.connection) finishProcessReplica(replica, true); if (hedged_connections_factory.hasEventsInProcess()) @@ -199,9 +196,9 @@ std::string HedgedConnections::dumpAddresses() const { for (const auto & replica : offset_state.replicas) { - if (replica->connection) + if (replica.connection) { - addresses += (is_first ? "" : "; ") + replica->connection->getDescription(); + addresses += (is_first ? "" : "; ") + replica.connection->getDescription(); is_first = false; } } @@ -219,8 +216,8 @@ void HedgedConnections::sendCancel() for (auto & offset_status : offset_states) for (auto & replica : offset_status.replicas) - if (replica->connection) - replica->connection->sendCancel(); + if (replica.connection) + replica.connection->sendCancel(); cancelled = true; } @@ -288,15 +285,16 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) { event_fd = getReadyFileDescriptor(async_callback); - if (fd_to_replica.contains(event_fd)) + if (fd_to_replica_location.contains(event_fd)) { - packet = receivePacketFromReplica(fd_to_replica[event_fd], async_callback); + ReplicaLocation location = fd_to_replica_location[event_fd]; + packet = receivePacketFromReplica(location, async_callback); finish = true; } - else if (timeout_fd_to_replica.contains(event_fd)) + else if (timeout_fd_to_replica_location.contains(event_fd)) { - ReplicaStatePtr replica = timeout_fd_to_replica[event_fd]; - processTimeoutEvent(replica, replica->active_timeouts[event_fd]); + ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; + processTimeoutEvent(location, offset_states[location.offset].replicas[location.index].active_timeouts[event_fd]); } else if (event_fd == hedged_connections_factory.getFileDescriptor()) tryGetNewReplica(false); @@ -309,10 +307,11 @@ Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { - for (auto & [fd, replica] : fd_to_replica) + for (auto & [fd, location] : fd_to_replica_location) { - if (replica->connection->hasReadPendingData()) - return replica->connection->getSocket()->impl()->sockfd(); + ReplicaState & replica = offset_states[location.offset].replicas[location.index]; + if (replica.connection->hasReadPendingData()) + return replica.connection->getSocket()->impl()->sockfd(); } epoll_event event; @@ -321,15 +320,16 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) return event.data.fd; } -Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback) +Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback) { + ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); - Packet packet = replica->connection->receivePacket(std::move(async_callback)); + Packet packet = replica.connection->receivePacket(std::move(async_callback)); switch (packet.type) { case Protocol::Server::Data: - if (!offset_states[replica->offset].first_packet_of_data_received) - processReceivedFirstDataPacket(replica); + if (!offset_states[replica_location.offset].first_packet_of_data_received) + processReceivedFirstDataPacket(replica_location); addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); break; case Protocol::Server::PartUUIDs: @@ -354,21 +354,21 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaStatePtr replica, Asyn return packet; } -void HedgedConnections::processReceivedFirstDataPacket(ReplicaStatePtr replica) +void HedgedConnections::processReceivedFirstDataPacket(ReplicaLocation & replica_location) { /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. - OffsetState & offset_state = offset_states[replica->offset]; - removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, replica); + OffsetState & offset_state = offset_states[replica_location.offset]; + removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, offset_state.replicas[replica_location.index]); ++offsets_with_received_first_data_packet; offset_state.first_packet_of_data_received = true; - for (auto & other_replica : offset_state.replicas) + for (size_t i = 0; i != offset_state.replicas.size(); ++i) { - if (replica != other_replica && other_replica->connection) + if (i != replica_location.index && offset_state.replicas[i].connection) { - other_replica->connection->sendCancel(); - finishProcessReplica(other_replica, true); + offset_state.replicas[i].connection->sendCancel(); + finishProcessReplica(offset_state.replicas[i], true); } } @@ -384,23 +384,24 @@ void HedgedConnections::processReceivedFirstDataPacket(ReplicaStatePtr replica) } } -void HedgedConnections::processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) +void HedgedConnections::processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor) { + ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; epoll.remove(timeout_descriptor->timer.getDescriptor()); - replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica.erase(timeout_descriptor->timer.getDescriptor()); + replica.active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica_location.erase(timeout_descriptor->timer.getDescriptor()); if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) { finishProcessReplica(replica, true); /// Check if there is no active connections with the same offset and there is no new replica in process. - if (offset_states[replica->offset].active_connection_count == 0 && !next_replica_in_process) + if (offset_states[replica_location.offset].active_connection_count == 0 && !next_replica_in_process) throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); } else if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT) { - offsets_queue.push(replica->offset); + offsets_queue.push(replica_location.offset); tryGetNewReplica(true); } } @@ -413,24 +414,24 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) /// Skip replicas that doesn't support two-level aggregation if we didn't disable it in sendQuery. while (state == HedgedConnectionsFactory::State::READY && !disable_two_level_aggregation && connection->getServerRevision(hedged_connections_factory.getConnectionTimeouts()) - < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) state = hedged_connections_factory.getNextConnection(true, false, connection); if (state == HedgedConnectionsFactory::State::READY) { size_t offset = offsets_queue.front(); offsets_queue.pop(); + size_t index = offset_states[offset].replicas.size(); - ReplicaStatePtr replica = std::make_shared(); - replica->connection = connection; - replica->offset = offset; - int socket_fd = replica->connection->getSocket()->impl()->sockfd(); + ReplicaState replica; + replica.connection = connection; + int socket_fd = replica.connection->getSocket()->impl()->sockfd(); epoll.add(socket_fd); - fd_to_replica[socket_fd] = replica; - offset_states[offset].replicas.push_back(replica); + fd_to_replica_location[socket_fd] = ReplicaLocation{offset, index}; ++offset_states[offset].active_connection_count; ++active_connection_count; pipeline_for_new_replicas.run(replica); + offset_states[offset].replicas.push_back(replica); } else if (state == HedgedConnectionsFactory::State::NOT_READY && !next_replica_in_process) { @@ -457,50 +458,50 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) } } -void HedgedConnections::finishProcessReplica(ReplicaStatePtr replica, bool disconnect) +void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { removeTimeoutsFromReplica(replica); - int socket_fd = replica->connection->getSocket()->impl()->sockfd(); + int socket_fd = replica.connection->getSocket()->impl()->sockfd(); epoll.remove(socket_fd); - --offset_states[replica->offset].active_connection_count; - fd_to_replica.erase(socket_fd); + --offset_states[fd_to_replica_location[socket_fd].offset].active_connection_count; + fd_to_replica_location.erase(socket_fd); --active_connection_count; if (disconnect) - replica->connection->disconnect(); - replica->connection = nullptr; + replica.connection->disconnect(); + replica.connection = nullptr; } -void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica) +void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica) { ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, hedged_connections_factory.getConnectionTimeouts()); epoll.add(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()] - = fd_to_replica[replica->connection->getSocket()->impl()->sockfd()]; - replica->active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); + timeout_fd_to_replica_location[timeout_descriptor->timer.getDescriptor()] + = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()]; + replica.active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); } -void HedgedConnections::removeTimeoutsFromReplica(ReplicaStatePtr replica) +void HedgedConnections::removeTimeoutsFromReplica(ReplicaState & replica) { - for (auto & [fd, _] : replica->active_timeouts) + for (auto & [fd, _] : replica.active_timeouts) { epoll.remove(fd); - timeout_fd_to_replica.erase(fd); + timeout_fd_to_replica_location.erase(fd); } - replica->active_timeouts.clear(); + replica.active_timeouts.clear(); } -void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr replica) +void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica) { auto it = std::find_if( - replica->active_timeouts.begin(), replica->active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); + replica.active_timeouts.begin(), replica.active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); - if (it != replica->active_timeouts.end()) + if (it != replica.active_timeouts.end()) { epoll.remove(it->first); - timeout_fd_to_replica.erase(it->first); - replica->active_timeouts.erase(it); + timeout_fd_to_replica_location.erase(it->first); + replica.active_timeouts.erase(it); } } diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 00145544096..4e3b6a67169 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -22,14 +22,17 @@ public: { Connection * connection = nullptr; std::unordered_map active_timeouts; - size_t offset = 0; }; - using ReplicaStatePtr = std::shared_ptr; + struct ReplicaLocation + { + size_t offset; + size_t index; + }; struct OffsetState { - std::vector replicas; + std::vector replicas; size_t active_connection_count; bool first_packet_of_data_received; }; @@ -76,32 +79,32 @@ private: class Pipeline { public: - void add(std::function send_function); + void add(std::function send_function); - void run(ReplicaStatePtr replica); + void run(ReplicaState & replica); private: - std::vector> pipeline; + std::vector> pipeline; }; - Packet receivePacketFromReplica(ReplicaStatePtr replica, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback = {}); Packet receivePacketImpl(AsyncCallback async_callback = {}); - void processReceivedFirstDataPacket(ReplicaStatePtr replica); + void processReceivedFirstDataPacket(ReplicaLocation & replica_location); - void processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); + void processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor); void tryGetNewReplica(bool start_new_connection); - void finishProcessReplica(ReplicaStatePtr replica, bool disconnect); + void finishProcessReplica(ReplicaState & replica, bool disconnect); int getReadyFileDescriptor(AsyncCallback async_callback = {}); - void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica); + void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica); - void removeTimeoutsFromReplica(ReplicaStatePtr replica); + void removeTimeoutsFromReplica(ReplicaState & replica); - void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaStatePtr replica); + void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica); HedgedConnectionsFactory hedged_connections_factory; @@ -111,10 +114,10 @@ private: /// replica_states[offset].replicas.size() = 1 (like in MultiplexedConnections). std::vector offset_states; - /// Map socket file descriptor to replica. - std::unordered_map fd_to_replica; - /// Map timeout file descriptor to replica. - std::unordered_map timeout_fd_to_replica; + /// Map socket file descriptor to replica location (it's offset and index in OffsetState.replicas). + std::unordered_map fd_to_replica_location; + /// Map timeout file descriptor to replica location (it's offset and index in OffsetState.replicas). + std::unordered_map timeout_fd_to_replica_location; /// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from /// the replica, we push it's offset to this queue and start trying to get @@ -142,7 +145,6 @@ private: Epoll epoll; const Settings & settings; ThrottlerPtr throttler; - Poco::Logger * log; bool sent_query = false; bool cancelled = false; diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 732e1e4b7d0..12362635904 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -24,6 +24,8 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( for (size_t i = 0; i != shuffled_pools.size(); ++i) connection_establishers.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); + replicas_timeouts.resize(shuffled_pools.size()); + max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); @@ -57,6 +59,7 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode std::vector connections; connections.reserve(max_entries); + Connection * connection = nullptr; /// Try to start establishing connections with max_entries replicas. int index; @@ -66,14 +69,13 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode if (index == -1) break; - ReplicaStatePtr replica = startEstablishingConnection(index); - if (replica->state == State::READY) - connections.push_back(replica->connection); + auto state = startEstablishingConnection(index, connection); + if (state == State::READY) + connections.push_back(connection); } /// Process connections until we get enough READY connections /// (work asynchronously with all connections we started). - Connection * connection = nullptr; while (connections.size() < max_entries) { auto state = getNextConnection(false, true, connection); @@ -102,7 +104,6 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out) { - ReplicaStatePtr replica = nullptr; int index = -1; if (start_new_connection) @@ -112,22 +113,14 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool { if (index != -1) { - replica = startEstablishingConnection(index); - if (replica->state == State::READY) - { - connection_out = replica->connection; - return State::READY; - } + State state = startEstablishingConnection(index, connection_out); + if (state == State::READY) + return state; } - if (!processEpollEvents(replica, blocking)) - return State::NOT_READY; - - if (replica->state == State::READY) - { - connection_out = replica->connection; - return State::READY; - } + State state = processEpollEvents(blocking, connection_out); + if (state != State::EMPTY) + return state; index = getNextIndex(); } @@ -139,22 +132,19 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool if (!fallback_to_stale_replicas || !canGetNewConnection()) return State::CANNOT_CHOOSE; - setBestUsableReplica(replica); - connection_out = replica->connection; - return replica->state; + return setBestUsableReplica(connection_out); } void HedgedConnectionsFactory::stopChoosingReplicas() { - for (auto & [fd, replica] : fd_to_replica) + for (auto & [fd, replica_index] : fd_to_replica_index) { - removeTimeoutsFromReplica(replica); + removeTimeoutsFromReplica(replica_index); epoll.remove(fd); - connection_establishers[replica->index].reset(); - replica->reset(); + connection_establishers[replica_index].reset(); } - fd_to_replica.clear(); + fd_to_replica_index.clear(); } int HedgedConnectionsFactory::getNextIndex() @@ -190,56 +180,54 @@ int HedgedConnectionsFactory::getNextIndex() return next_index; } -HedgedConnectionsFactory::ReplicaStatePtr HedgedConnectionsFactory::startEstablishingConnection(int index) +HedgedConnectionsFactory::State HedgedConnectionsFactory::startEstablishingConnection(int replica_index, Connection *& connection_out) { - ReplicaStatePtr replica = createNewReplica(); - + State state; do { - ConnectionEstablisher & connection_establisher = connection_establishers[index]; + ConnectionEstablisher & connection_establisher = connection_establishers[replica_index]; - replica->state = State::NOT_READY; - replica->index = index; - indexes_in_process.insert(index); + state = State::NOT_READY; + indexes_in_process.insert(replica_index); connection_establisher.reset(); connection_establisher.run(); - if (connection_establisher.stage != ConnectionEstablisher::Stage::FAILED) - replica->connection = &*connection_establisher.result.entry; + state = processConnectionEstablisherStage(replica_index); - processConnectionEstablisherStage(replica); - - if (replica->state == State::NOT_READY) + if (state == State::NOT_READY) { epoll.add(connection_establisher.socket_fd); - fd_to_replica[connection_establisher.socket_fd] = replica; + fd_to_replica_index[connection_establisher.socket_fd] = replica_index; connection_establisher.setActionBeforeDisconnect([&](int fd) { epoll.remove(fd); - fd_to_replica.erase(fd); + fd_to_replica_index.erase(fd); }); - addTimeouts(replica); + addTimeouts(replica_index); } } - while (replica->state == State::EMPTY && (index = getNextIndex()) != -1); + while (state == State::EMPTY && (replica_index = getNextIndex()) != -1); - return replica; + if (state == State::READY) + connection_out = &*connection_establishers[replica_index].result.entry; + + return state; } -void HedgedConnectionsFactory::processConnectionEstablisherStage(ReplicaStatePtr replica, bool remove_from_epoll) +HedgedConnectionsFactory::State HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_index, bool remove_from_epoll) { - ConnectionEstablisher & connection_establisher = connection_establishers[replica->index]; + ConnectionEstablisher & connection_establisher = connection_establishers[replica_index]; if (connection_establisher.stage == ConnectionEstablisher::Stage::FINISHED) { - indexes_in_process.erase(replica->index); + indexes_in_process.erase(replica_index); ++entries_count; if (remove_from_epoll) { epoll.remove(connection_establisher.socket_fd); - fd_to_replica.erase(connection_establisher.socket_fd); + fd_to_replica_index.erase(connection_establisher.socket_fd); } if (connection_establisher.result.is_usable) @@ -247,24 +235,28 @@ void HedgedConnectionsFactory::processConnectionEstablisherStage(ReplicaStatePtr ++usable_count; if (connection_establisher.result.is_up_to_date) { - replica->state = State::READY; - ready_indexes.insert(replica->index); - return; + ready_indexes.insert(replica_index); + return State::READY; } } /// This replica is not up to date, we will try to find up to date. - replica->reset(); + return State::EMPTY; } else if (connection_establisher.stage == ConnectionEstablisher::Stage::FAILED) - processFailedConnection(replica); + { + processFailedConnection(replica_index); + return State::EMPTY; + } + + return State::NOT_READY; } -void HedgedConnectionsFactory::processFailedConnection(ReplicaStatePtr replica) +void HedgedConnectionsFactory::processFailedConnection(int replica_index) { - ShuffledPool & shuffled_pool = shuffled_pools[replica->index]; + ShuffledPool & shuffled_pool = shuffled_pools[replica_index]; LOG_WARNING( - log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), connection_establishers[replica->index].fail_message); + log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), connection_establishers[replica_index].fail_message); ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); @@ -275,83 +267,78 @@ void HedgedConnectionsFactory::processFailedConnection(ReplicaStatePtr replica) ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); } - std::string & fail_message = connection_establishers[replica->index].fail_message; + std::string & fail_message = connection_establishers[replica_index].fail_message; if (!fail_message.empty()) fail_messages += fail_message + "\n"; - indexes_in_process.erase(replica->index); - replica->reset(); + indexes_in_process.erase(replica_index); } -void HedgedConnectionsFactory::addTimeouts(ReplicaStatePtr replica) +void HedgedConnectionsFactory::addTimeouts(int replica_index) { - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica_index); - auto stage = connection_establishers[replica->index].stage; + auto stage = connection_establishers[replica_index].stage; if (stage == ConnectionEstablisher::Stage::RECEIVE_HELLO) - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT, replica); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT, replica_index); else if (stage == ConnectionEstablisher::Stage::RECEIVE_TABLES_STATUS) - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT, replica); + addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT, replica_index); } -void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica) +void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, int replica_index) { ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, timeouts); epoll.add(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()] = replica; - replica->active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); + timeout_fd_to_replica_index[timeout_descriptor->timer.getDescriptor()] = replica_index; + replicas_timeouts[replica_index][timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); } -void HedgedConnectionsFactory::removeTimeoutsFromReplica(ReplicaStatePtr replica) +void HedgedConnectionsFactory::removeTimeoutsFromReplica(int replica_index) { - for (auto & [fd, _] : replica->active_timeouts) + for (auto & [fd, _] : replicas_timeouts[replica_index]) { epoll.remove(fd); - timeout_fd_to_replica.erase(fd); + timeout_fd_to_replica_index.erase(fd); } - replica->active_timeouts.clear(); + replicas_timeouts[replica_index].clear(); } -bool HedgedConnectionsFactory::processEpollEvents(ReplicaStatePtr & replica, bool blocking) +HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out) { int event_fd; - bool finish = false; - while (!finish) + while (true) { event_fd = getReadyFileDescriptor(blocking); /// Check if there is no events. if (event_fd == -1) - return false; + return State::NOT_READY; - if (fd_to_replica.find(event_fd) != fd_to_replica.end()) + if (fd_to_replica_index.find(event_fd) != fd_to_replica_index.end()) { - replica = fd_to_replica[event_fd]; - processReplicaEvent(replica); - /// Check if replica is ready or we need to try next replica. - if (replica->state == State::READY || replica->state == State::EMPTY) - finish = true; + int replica_index = fd_to_replica_index[event_fd]; + State state = processReplicaEvent(replica_index, connection_out); + /// Return only if replica is ready or we need to try next replica. + if (state != State::NOT_READY) + return state; } - else if (timeout_fd_to_replica.find(event_fd) != timeout_fd_to_replica.end()) + else if (timeout_fd_to_replica_index.find(event_fd) != timeout_fd_to_replica_index.end()) { - replica = timeout_fd_to_replica[event_fd]; - processTimeoutEvent(replica, replica->active_timeouts[event_fd]); - /// Check if we need to try next replica. - if (replica->state == State::EMPTY) - finish = true; + int replica_index = timeout_fd_to_replica_index[event_fd]; + /// Process received timeout. If retured values is true, we need to try new replica. + if (processTimeoutEvent(replica_index, replicas_timeouts[replica_index][event_fd])) + return State::EMPTY; } else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } - - return true; } int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) { - for (auto & [fd, replica] : fd_to_replica) - if (replica->connection->hasReadPendingData()) - return replica->connection->getSocket()->impl()->sockfd(); + for (auto & [fd, replica_index] : fd_to_replica_index) + if (connection_establishers[replica_index].result.entry->hasReadPendingData()) + return connection_establishers[replica_index].socket_fd; epoll_event event; event.data.fd = -1; @@ -359,41 +346,44 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) return event.data.fd; } -void HedgedConnectionsFactory::processReplicaEvent(ReplicaStatePtr replica) +HedgedConnectionsFactory::State HedgedConnectionsFactory::processReplicaEvent(int replica_index, Connection *& connection_out) { - removeTimeoutsFromReplica(replica); - connection_establishers[replica->index].run(); - processConnectionEstablisherStage(replica, true); - if (replica->state == State::NOT_READY) - addTimeouts(replica); + removeTimeoutsFromReplica(replica_index); + connection_establishers[replica_index].run(); + State state = processConnectionEstablisherStage(replica_index, true); + if (state == State::NOT_READY) + addTimeouts(replica_index); + if (state == State::READY) + connection_out = &*connection_establishers[replica_index].result.entry; + return state; } -void HedgedConnectionsFactory::processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor) +bool HedgedConnectionsFactory::processTimeoutEvent(int replica_index, ConnectionTimeoutDescriptorPtr timeout_descriptor) { epoll.remove(timeout_descriptor->timer.getDescriptor()); - replica->active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica[timeout_descriptor->timer.getDescriptor()]; + replicas_timeouts[replica_index].erase(timeout_descriptor->timer.getDescriptor()); + timeout_fd_to_replica_index[timeout_descriptor->timer.getDescriptor()]; if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) { - removeTimeoutsFromReplica(replica); - int fd = replica->connection->getSocket()->impl()->sockfd(); + removeTimeoutsFromReplica(replica_index); + int fd = connection_establishers[replica_index].socket_fd; epoll.remove(fd); - fd_to_replica.erase(fd); + fd_to_replica_index.erase(fd); - ConnectionEstablisher & connection_establisher = connection_establishers[replica->index]; + ConnectionEstablisher & connection_establisher = connection_establishers[replica_index]; connection_establisher.fail_message = "Receive timeout expired (" + connection_establisher.result.entry->getDescription() + ")"; connection_establisher.resetResult(); connection_establisher.stage = ConnectionEstablisher::Stage::FAILED; - processFailedConnection(replica); + processFailedConnection(replica_index); + return true; } - else if ((timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT - || timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT) - && entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size()) - replica = createNewReplica(); + + /// Return true if we can try to start one more connection. + return entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size(); } -void HedgedConnectionsFactory::setBestUsableReplica(ReplicaStatePtr replica) +HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(Connection *& connection_out) { std::vector indexes(connection_establishers.size()); for (size_t i = 0; i != indexes.size(); ++i) @@ -412,10 +402,7 @@ void HedgedConnectionsFactory::setBestUsableReplica(ReplicaStatePtr replica) indexes.end()); if (indexes.empty()) - { - replica->state = State::CANNOT_CHOOSE; - return; - } + return State::CANNOT_CHOOSE; /// Sort replicas by staleness. std::stable_sort( @@ -426,10 +413,9 @@ void HedgedConnectionsFactory::setBestUsableReplica(ReplicaStatePtr replica) return connection_establishers[lhs].result.staleness < connection_establishers[rhs].result.staleness; }); - replica->index = indexes[0]; - replica->connection = &*connection_establishers[indexes[0]].result.entry; - replica->state = State::READY; - ready_indexes.insert(replica->index); + ready_indexes.insert(indexes[0]); + connection_out = &*connection_establishers[indexes[0]].result.entry; + return State::READY; } ConnectionTimeoutDescriptorPtr createConnectionTimeoutDescriptor(ConnectionTimeoutType type, const ConnectionTimeouts & timeouts) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 398629cf13c..048d90e1de6 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -48,24 +48,6 @@ public: CANNOT_CHOOSE = 3, }; - struct ReplicaState - { - Connection * connection = nullptr; - size_t index = -1; - State state = State::EMPTY; - std::unordered_map active_timeouts; - - void reset() - { - connection = nullptr; - index = -1; - state = State::EMPTY; - active_timeouts.clear(); - } - }; - - using ReplicaStatePtr = std::shared_ptr; - HedgedConnectionsFactory(const ConnectionPoolWithFailoverPtr & pool_, const Settings * settings_, const ConnectionTimeouts & timeouts_, @@ -96,9 +78,9 @@ public: ~HedgedConnectionsFactory(); private: - ReplicaStatePtr startEstablishingConnection(int index); + State startEstablishingConnection(int index, Connection *& connection_out); - void processConnectionEstablisherStage(ReplicaStatePtr replica, bool remove_from_epoll = false); + State processConnectionEstablisherStage(int replica_index, bool remove_from_epoll = false); /// Find an index of the next free replica to start connection. /// Return -1 if there is no free replica. @@ -106,27 +88,23 @@ private: int getReadyFileDescriptor(bool blocking); - void addTimeouts(ReplicaStatePtr replica); + void addTimeouts(int replica_index); - void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaStatePtr replica); + void addTimeoutToReplica(ConnectionTimeoutType type, int replica_index); - void removeTimeoutsFromReplica(ReplicaStatePtr replica); + void removeTimeoutsFromReplica(int replica_index); - void processFailedConnection(ReplicaStatePtr replica); + void processFailedConnection(int replica_index); - void processReceiveTimeout(ReplicaStatePtr replica); + State processReplicaEvent(int replica_index, Connection *& connection_out); - void processReplicaEvent(ReplicaStatePtr replica); - - void processTimeoutEvent(ReplicaStatePtr replica, ConnectionTimeoutDescriptorPtr timeout_descriptor); + bool processTimeoutEvent(int replica_index, ConnectionTimeoutDescriptorPtr timeout_descriptor); /// Return false if there is no ready events, return true if replica is ready /// or we need to try next replica. - bool processEpollEvents(ReplicaStatePtr & replica, bool blocking); + State processEpollEvents(bool blocking, Connection *& connection_out); - void setBestUsableReplica(ReplicaStatePtr replica); - - ReplicaStatePtr createNewReplica() { return std::make_shared(); } + State setBestUsableReplica(Connection *& connection_out); const ConnectionPoolWithFailoverPtr pool; const Settings * settings; @@ -136,10 +114,12 @@ private: std::vector connection_establishers; std::vector shuffled_pools; - /// Map socket file descriptor to replica. - std::unordered_map fd_to_replica; - /// Map timeout file descriptor to replica. - std::unordered_map timeout_fd_to_replica; + std::vector> replicas_timeouts; + + /// Map socket file descriptor to replica index. + std::unordered_map fd_to_replica_index; + /// Map timeout file descriptor to replica index. + std::unordered_map timeout_fd_to_replica_index; /// Indexes of replicas, that are in process of connection. std::unordered_set indexes_in_process; From 4cc7e2c5c68026af391af1b61ccae93ff87ff291 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 9 Feb 2021 05:13:47 +0300 Subject: [PATCH 088/510] Update comment --- src/Client/HedgedConnectionsFactory.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 048d90e1de6..45a03e212c0 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -100,8 +100,8 @@ private: bool processTimeoutEvent(int replica_index, ConnectionTimeoutDescriptorPtr timeout_descriptor); - /// Return false if there is no ready events, return true if replica is ready - /// or we need to try next replica. + /// Return NOT_READY state if there is no ready events, READY if replica is ready + /// and EMPTY if we need to try next replica. State processEpollEvents(bool blocking, Connection *& connection_out); State setBestUsableReplica(Connection *& connection_out); From be3be85fa2167beb909ec75a6180ae0a63421186 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 13:57:41 +0300 Subject: [PATCH 089/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/type-conversion-functions.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 1742f6b8888..cab71f46bf5 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -492,8 +492,9 @@ Result: ## accurateCast(x, T) {#type_conversion_function-accurate-cast} -Converts `x` to the `T` data type. The differente from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` -does not allow overflow of numeric types during cast if type value `x` does not fit bounds of type `T`. +Converts `x` to the `T` data type. + +The difference from [cast(x, T)](#type_conversion_function-cast) is that `accurateCast` does not allow overflow of numeric types during cast if type value `x` does not fit the bounds of type `T`. For example, `accurateCast(-1, 'UInt8')` throws an exception. **Example** From b676f63f1dec7b606f4f5559f910f02098f9c135 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 13:58:22 +0300 Subject: [PATCH 090/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index aa55e015c61..d95a5279716 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -497,7 +497,7 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; accurateCastOrNull(x, T) ``` -**Parameters** +**Параметры** - `x` — входное значение. - `T` — имя возвращаемого типа данных. From c22412b775b36009f3ceba36fb82a595a5d49075 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 13:58:47 +0300 Subject: [PATCH 091/510] Update docs/en/sql-reference/operators/in.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/in.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index a0dd0455c4d..4796c0f6bc0 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -17,7 +17,7 @@ Don’t list too many values explicitly (i.e. millions). If a data set is large The right side of the operator can be a set of constant expressions, a set of tuples with constant expressions (shown in the examples above), or the name of a database table or SELECT subquery in brackets. -ClickHouse allows different types inside `IN` subquery. For left hand side it applies conversion to the type of right hand side with [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). +ClickHouse allows different types in the left and right parts of `IN` subquery. In this case it converts the left hand side to the type of the right hand side as if the [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null) function is applied. **Example** From df123e91e650c9f4dd11d12dff78753df58bbe6d Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 13:59:58 +0300 Subject: [PATCH 092/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../en/sql-reference/functions/type-conversion-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index cab71f46bf5..83cbad6f53b 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -559,9 +559,9 @@ Query: ``` sql SELECT - cast(-1, 'UInt8') as uint8, - cast(128, 'Int8') as int8, - cast('Test', 'FixedString(2)') as fixed_string; + accurateCastOrNull(-1, 'UInt8') as uint8, + accurateCastOrNull(128, 'Int8') as int8, + accurateCastOrNull('Test', 'FixedString(2)') as fixed_string; ``` Result: From 94a489ce97eef31f4036759b04d9651f4cd5512e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 17:25:04 +0300 Subject: [PATCH 093/510] Update docs/ru/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 0acb9e3cd39..d019c18a688 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -686,7 +686,7 @@ SELECT formatDateTime(toDate('2010-01-04'), '%g'); ## FROM\_UNIXTIME {#fromunixtime} -Функция преобразует метку времени Unix в дату. +Функция преобразует Unix timestamp в календарную дату и время. **Примеры** From 79a1a5741f723374b41325953c78f927fc4a92a4 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 17:25:38 +0300 Subject: [PATCH 094/510] Update docs/en/sql-reference/data-types/simpleaggregatefunction.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/data-types/simpleaggregatefunction.md b/docs/en/sql-reference/data-types/simpleaggregatefunction.md index e25d4803613..244779c5ca8 100644 --- a/docs/en/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/en/sql-reference/data-types/simpleaggregatefunction.md @@ -35,7 +35,7 @@ The following aggregate functions are supported: **Example** ``` sql -CREATE TABLE simple (id UInt64,val SimpleAggregateFunction(sum,Double)) ENGINE=AggregatingMergeTree ORDER BY id; +CREATE TABLE simple (id UInt64, val SimpleAggregateFunction(sum, Double)) ENGINE=AggregatingMergeTree ORDER BY id; ``` [Original article](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From 55727f511df2baa19584f32a7289d4e2ae298add Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 17:27:39 +0300 Subject: [PATCH 095/510] Update docs/en/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index ce2092a7818..ca62d2a61e5 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -663,7 +663,7 @@ Result: ## FROM\_UNIXTIME {#fromunixfime} -Function converts Unix timestamp to date. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. +Function converts Unix timestamp to a calendar date and a time of a day. When there is only a single argument of [Integer](../../sql-reference/data-types/int-uint.md) type, it acts in the same way as [toDateTime](../../sql-reference/functions/type-conversion-functions.md#todatetime) and return [DateTime](../../sql-reference/data-types/datetime.md) type. **Example:** From 44e857b5ea3ca2bbf49d3746af1c1941ac3a2f33 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Tue, 9 Feb 2021 17:30:16 +0300 Subject: [PATCH 096/510] Update simpleaggregatefunction.md --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index b80826803de..7677b64e924 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -33,7 +33,7 @@ **Пример** ``` sql -CREATE TABLE simple (id UInt64,val SimpleAggregateFunction(sum,Double)) ENGINE=AggregatingMergeTree ORDER BY id; +CREATE TABLE simple (id UInt64, val SimpleAggregateFunction(sum, Double)) ENGINE=AggregatingMergeTree ORDER BY id; ``` [Оригинальная статья](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) From 178ada23f811354e47683677ab0c787c6170750e Mon Sep 17 00:00:00 2001 From: George Date: Wed, 10 Feb 2021 15:55:18 +0300 Subject: [PATCH 097/510] early draft --- .../functions/tuple-map-functions.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index a46c36395b8..50015cd996e 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -112,4 +112,34 @@ Result: └──────────────────────────────┴───────────────────────────────────┘ ``` +## mapContains {#mapcontains} + +Determines whether `map.keys` contains the `key` parameter. + +**Syntax** + +``` sql +mapContains(map, key) +``` + +**Parameters** + +- `map` — Map. [Type name](relative/path/to/type/dscr.md#type). +- `key` — Key. Type matches the type of `map.keys`. + +**Returned value** + +- `1` if `map.keys` contains `key`, `0` if not. + +Type: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Example** + +Query: + + +## mapKeys {#mapKeys} + +## mapValues {#mapvalues} + [Original article](https://clickhouse.tech/docs/en/query_language/functions/tuple-map-functions/) From 48f6f7e490754880ad179c3568d2c118454d0db9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 10 Feb 2021 19:26:49 +0300 Subject: [PATCH 098/510] Split filter for predicate push down. --- src/Interpreters/ActionsDAG.cpp | 194 +++++++++++++++++++++++++++++++- src/Interpreters/ActionsDAG.h | 9 +- 2 files changed, 201 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 176745c707d..223b4341f46 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -338,7 +339,7 @@ void ActionsDAG::removeUnusedActions(const std::vector & required_nodes) removeUnusedActions(); } -void ActionsDAG::removeUnusedActions() +void ActionsDAG::removeUnusedActions(bool allow_remove_inputs) { std::unordered_set visited_nodes; std::stack stack; @@ -357,6 +358,9 @@ void ActionsDAG::removeUnusedActions() visited_nodes.insert(&node); stack.push(&node); } + + if (node.type == ActionType::INPUT && !allow_remove_inputs) + visited_nodes.insert(&node); } while (!stack.empty()) @@ -1153,4 +1157,192 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co return split(split_nodes); } +ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, const Names & available_inputs) +{ + std::unordered_map> inputs_map; + for (const auto & input : inputs) + inputs_map[input->result_name].emplace_back(input); + + std::unordered_set allowed_nodes; + for (const auto & name : available_inputs) + { + auto & inputs_list = inputs_map[name]; + if (inputs_list.empty()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find input {} in ActionsDAG. DAG:\n{}", name, dumpDAG()); + + allowed_nodes.emplace(inputs_list.front()); + inputs_list.pop_front(); + } + + auto it = index.begin(); + for (; it != index.end(); ++it) + if ((*it)->result_name == filter_name) + break; + + if (it == index.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Index for ActionsDAG does not contain filter column name {}. DAG:\n{}", + filter_name, dumpDAG()); + + std::unordered_set selected_predicates; + + { + struct Frame + { + const Node * node; + bool is_predicate = false; + size_t next_child_to_visit = 0; + size_t num_allowed_children = 0; + }; + + std::stack stack; + std::unordered_set visited_nodes; + + stack.push(Frame{.node = *it, .is_predicate = true}); + visited_nodes.insert(*it); + while (!stack.empty()) + { + auto & cur = stack.top(); + bool is_conjunction = cur.is_predicate + && cur.node->type == ActionType::FUNCTION + && cur.node->function_base->getName() == "and"; + + /// At first, visit all children. + while (cur.next_child_to_visit < cur.node->children.size()) + { + auto * child = cur.node->children[cur.next_child_to_visit]; + + if (visited_nodes.count(child) == 0) + { + visited_nodes.insert(child); + stack.push({.node = child, .is_predicate = is_conjunction}); + break; + } + + if (allowed_nodes.contains(child)) + ++cur.num_allowed_children; + ++cur.next_child_to_visit; + } + + if (cur.next_child_to_visit == cur.node->children.size()) + { + if (cur.num_allowed_children == cur.node->children.size()) + { + if (cur.node->type != ActionType::ARRAY_JOIN && cur.node->type != ActionType::INPUT) + allowed_nodes.emplace(cur.node); + } + else if (is_conjunction) + { + for (auto * child : cur.node->children) + if (allowed_nodes.count(child)) + selected_predicates.insert(child); + } + + stack.pop(); + } + } + } + + if (selected_predicates.empty()) + { + if (allowed_nodes.count(*it)) + selected_predicates.insert(*it); + else + return nullptr; + } + + auto actions = cloneEmpty(); + actions->settings.project_input = false; + + std::unordered_map nodes_mapping; + + { + struct Frame + { + const Node * node; + size_t next_child_to_visit = 0; + }; + + std::stack stack; + + for (const auto * predicate : selected_predicates) + { + if (nodes_mapping.count(predicate)) + continue; + + stack.push({.node = predicate}); + while (!stack.empty()) + { + auto & cur = stack.top(); + /// At first, visit all children. + while (cur.next_child_to_visit < cur.node->children.size()) + { + auto * child = cur.node->children[cur.next_child_to_visit]; + + if (nodes_mapping.count(child) == 0) + { + stack.push({.node = child}); + break; + } + + ++cur.next_child_to_visit; + } + + if (cur.next_child_to_visit == cur.node->children.size()) + { + auto & node = actions->nodes.emplace_back(*cur.node); + nodes_mapping[cur.node] = &node; + + for (auto & child : node.children) + child = nodes_mapping[child]; + + if (node.type == ActionType::INPUT) + { + actions->inputs.emplace_back(&node); + actions->index.insert(&node); + } + } + } + } + + Node * result_predicate = nodes_mapping[*selected_predicates.begin()]; + + if (selected_predicates.size() > 1) + { + FunctionOverloadResolverPtr func_builder_and = + std::make_shared( + std::make_unique( + std::make_shared())); + + std::vector args; + args.reserve(selected_predicates.size()); + for (const auto * predicate : selected_predicates) + args.emplace_back(nodes_mapping[predicate]); + + result_predicate = &actions->addFunction(func_builder_and, args, {}, true); + } + + actions->index.insert(result_predicate); + } + + + + /// Replace all predicates which are copied to constants. + /// Note: This also keeps valid const propagation. AND is constant only if all elements are. + /// But if all elements are constant, AND should is moved to split actions and replaced itself. + for (const auto & predicate : selected_predicates) + { + Node node; + node.type = ActionType::COLUMN; + node.result_name = std::move(predicate->result_name); + node.result_type = std::move(predicate->result_type); + node.column = node.result_type->createColumnConst(0, 1); + *predicate = std::move(node); + } + + removeUnusedActions(false); + + return actions; +} + } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index e13a9bd62b3..6fd4e14568a 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -274,6 +274,13 @@ public: /// Index of initial actions must contain column_name. SplitResult splitActionsForFilter(const std::string & column_name) const; + /// Create actions which may calculate part of filter using only available_inputs. + /// If nothing may be calculated, returns nullptr. + /// Otherwise, return actions which inputs are from available_inputs. + /// Returned actions add single column which may be used for filter. + /// Also, replace some nodes of current inputs to constant 1 in case they are filtered. + ActionsDAGPtr splitActionsForFilter(const std::string & filter_name, const Names & available_inputs); + private: Node & addNode(Node node, bool can_replace = false); Node & getNode(const std::string & name); @@ -297,7 +304,7 @@ private: } void removeUnusedActions(const std::vector & required_nodes); - void removeUnusedActions(); + void removeUnusedActions(bool allow_remove_inputs = true); void addAliases(const NamesWithAliases & aliases, std::vector & result_nodes); void compileFunctions(); From a83885392e8233a9b9faa462eea371c71df2c745 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 10 Feb 2021 20:47:48 +0300 Subject: [PATCH 099/510] Split filter for predicate push down. --- src/Interpreters/ActionsDAG.cpp | 117 ++++++++++++++++++++++++++------ src/Interpreters/ActionsDAG.h | 2 +- 2 files changed, 98 insertions(+), 21 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 223b4341f46..eb1ff9ad998 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1157,7 +1157,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co return split(split_nodes); } -ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, const Names & available_inputs) +ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs) { std::unordered_map> inputs_map; for (const auto & input : inputs) @@ -1185,6 +1185,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, filter_name, dumpDAG()); std::unordered_set selected_predicates; + std::unordered_set other_predicates; { struct Frame @@ -1234,8 +1235,12 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, else if (is_conjunction) { for (auto * child : cur.node->children) + { if (allowed_nodes.count(child)) selected_predicates.insert(child); + else + other_predicates.insert(child); + } } stack.pop(); @@ -1254,6 +1259,11 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, auto actions = cloneEmpty(); actions->settings.project_input = false; + FunctionOverloadResolverPtr func_builder_and = + std::make_shared( + std::make_unique( + std::make_shared())); + std::unordered_map nodes_mapping; { @@ -1309,11 +1319,6 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, if (selected_predicates.size() > 1) { - FunctionOverloadResolverPtr func_builder_and = - std::make_shared( - std::make_unique( - std::make_shared())); - std::vector args; args.reserve(selected_predicates.size()); for (const auto * predicate : selected_predicates) @@ -1325,22 +1330,94 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, actions->index.insert(result_predicate); } - - - /// Replace all predicates which are copied to constants. - /// Note: This also keeps valid const propagation. AND is constant only if all elements are. - /// But if all elements are constant, AND should is moved to split actions and replaced itself. - for (const auto & predicate : selected_predicates) + if (selected_predicates.count(*it)) { - Node node; - node.type = ActionType::COLUMN; - node.result_name = std::move(predicate->result_name); - node.result_type = std::move(predicate->result_type); - node.column = node.result_type->createColumnConst(0, 1); - *predicate = std::move(node); - } + /// The whole predicate was split. + if (can_remove_filter) + { + for (auto i = index.begin(); i != index.end(); ++i) + { + if (*i == *it) + { + index.remove(i); + break; + } + } + } + else + { + Node node; + node.type = ActionType::COLUMN; + node.result_name = std::move((*it)->result_name); + node.result_type = std::move((*it)->result_type); + node.column = node.result_type->createColumnConst(0, 1); + *(*it) = std::move(node); + } - removeUnusedActions(false); + removeUnusedActions(false); + } + else if ((*it)->type == ActionType::FUNCTION && (*it)->function_base->getName() == "and") + { + std::vector new_children(other_predicates.begin(), other_predicates.end()); + + if (new_children.size() == 1) + { + if (new_children.front()->result_type->equals(*((*it)->result_type))) + { + Node node; + node.type = ActionType::ALIAS; + node.result_name = (*it)->result_name; + node.result_type = (*it)->result_type; + node.children.swap(new_children); + *(*it) = std::move(node); + } + else + { + (*it)->children.swap(new_children); + ColumnsWithTypeAndName arguments; + arguments.reserve((*it)->children.size()); + + for (const auto * child : (*it)->children) + { + ColumnWithTypeAndName argument; + argument.column = child->column; + argument.type = child->result_type; + argument.name = child->result_name; + + arguments.emplace_back(std::move(argument)); + } + + FunctionOverloadResolverPtr func_builder_cast = + std::make_shared( + CastOverloadResolver::createImpl(false)); + + (*it)->function_builder = func_builder_cast; + (*it)->function_base = (*it)->function_builder->build(arguments); + (*it)->function = (*it)->function_base->prepare(arguments); + } + } + else + { + (*it)->children.swap(new_children); + ColumnsWithTypeAndName arguments; + arguments.reserve((*it)->children.size()); + + for (const auto * child : (*it)->children) + { + ColumnWithTypeAndName argument; + argument.column = child->column; + argument.type = child->result_type; + argument.name = child->result_name; + + arguments.emplace_back(std::move(argument)); + } + + (*it)->function_base = (*it)->function_builder->build(arguments); + (*it)->function = (*it)->function_base->prepare(arguments); + } + + removeUnusedActions(false); + } return actions; } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 6fd4e14568a..112c507e79f 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -279,7 +279,7 @@ public: /// Otherwise, return actions which inputs are from available_inputs. /// Returned actions add single column which may be used for filter. /// Also, replace some nodes of current inputs to constant 1 in case they are filtered. - ActionsDAGPtr splitActionsForFilter(const std::string & filter_name, const Names & available_inputs); + ActionsDAGPtr splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs); private: Node & addNode(Node node, bool can_replace = false); From e87e71ee43550f0f3a59abf227d20ce661a3bf4f Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 10 Feb 2021 21:59:28 +0300 Subject: [PATCH 100/510] Document two functions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал две функции. --- .../functions/type-conversion-functions.md | 180 ++++++++++++++++++ 1 file changed, 180 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 3ca36f41c78..2116e55e3ef 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -689,6 +689,186 @@ Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it r Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it returns zero date or zero date time when it encounters a date format that cannot be processed. +## parseDateTimeBestEffortUSOrNull {#parsedatetimebesteffortusornull} + +Same as for [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) except that it returns null when it encounters a date format that cannot be processed. + +**Syntax** + +``` sql +parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); +``` + +**Parameters** + +- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md). +- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md). + +**Supported non-standard formats** + +- A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). +- A string with a date and a time component: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. +- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. +- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted as `2000-01`. +- A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. + +**Returned values** + +- `time_string` converted to the `DateTime` data type. +- `NULL`. + +**Examples** + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02/10/2021 21:12:57') +AS parseDateTimeBestEffortUSOrNull; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02-10-2021 21:12:57') +AS parseDateTimeBestEffortUSOrNull; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02.10.2021 21:12:57') +AS parseDateTimeBestEffortUSOrNull; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02.2021 21:12:57') +AS parseDateTimeBestEffortUSOrNull; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ ᴺᵁᴸᴸ │ +└─────────────────────────────────┘ +``` + +## parseDateTimeBestEffortUSOrZero {#parsedatetimebesteffortusorzero} + +Same as for [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) except that it returns zero date or zero date time when it encounters a date format that cannot be processed. + +**Syntax** + +``` sql +parseDateTimeBestEffortUSOrZero(time_string [, time_zone]); +``` + +**Parameters** + +- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md). +- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md). + +**Supported non-standard formats** + +- A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). +- A string with a date and a time component: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. +- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. +- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted as `2000-01`. +- A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. + +**Returned value** + +- `time_string` converted to the `DateTime` data type. +- `zero date time`. + +**Examples** + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02/10/2021 21:12:57') +AS parseDateTimeBestEffortUSOrZero; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02-10-2021 21:12:57') +AS parseDateTimeBestEffortUSOrZero; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02.10.2021 21:12:57') +AS parseDateTimeBestEffortUS; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02.2021 21:12:57') +AS parseDateTimeBestEffortUSOrZero; +``` + +Result: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 1970-01-01 00:00:00 │ +└─────────────────────────────────┘ +``` + ## toLowCardinality {#tolowcardinality} Converts input parameter to the [LowCardianlity](../../sql-reference/data-types/lowcardinality.md) version of same data type. From 537b372c32732ddecc9a5f7414c23ea1722ec2fc Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 11 Feb 2021 00:16:23 +0300 Subject: [PATCH 101/510] Update type-conversion-functions.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Исправил null на NULL. --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 2116e55e3ef..f752bb9f6cb 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -691,7 +691,7 @@ Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it r ## parseDateTimeBestEffortUSOrNull {#parsedatetimebesteffortusornull} -Same as for [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) except that it returns null when it encounters a date format that cannot be processed. +Same as for [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) except that it returns `NULL` when it encounters a date format that cannot be processed. **Syntax** From 59752cbf27104d76fa7a0c9b669f5dbe3b423c3e Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 11 Feb 2021 01:50:13 +0300 Subject: [PATCH 102/510] Update type-conversion-functions.md Fix changes from EN review. --- .../functions/type-conversion-functions.md | 50 ++++++++++++++++--- 1 file changed, 44 insertions(+), 6 deletions(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index d95a5279716..3a6d2bd9ca0 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -423,8 +423,11 @@ SELECT uuid = uuid2; ## CAST(x, T) {#type_conversion_function-cast} -Преобразует x в тип данных t. -Поддерживается также синтаксис CAST(x AS t). +Преобразует вхожное значение `x` в указананный тип данных `T`. + +Поддерживается также синтаксис `CAST(x AS t)`. + +Обратите внимание, что если значение `x` не соответствует границам типа `T`, функция переполняется. Например, `CAST(-1, 'UInt8')` возвращает 255. **Пример** @@ -487,9 +490,44 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; - Настройка [cast_keep_nullable](../../operations/settings/settings.md#cast_keep_nullable) +## accurateCast(x, T) {#type_conversion_function-accurate-cast} + +Преобразует входное значение `x` в указанный тип данных `T`. + +Отличие от [cast(x, T)](#type_conversion_function-cast) в том, что `accurateCast` не допускает переполнения числовых типов, если значение типа `x` не соответствует границам типа `T`. Например, `accurateCast(-1, 'UInt8')` вернет ошибку. + +**Примеры** + +Запрос: + +``` sql +SELECT cast(-1, 'UInt8') as uint8; +``` + +Результат: + +``` text +┌─uint8─┐ +│ 255 │ +└───── + +Запрос: + +```sql +SELECT accurateCast(-1, 'UInt8') as uint8; +``` + +Результат: + +``` text +Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in column Int8 cannot be safely converted into type UInt8: While processing accurateCast(-1, 'UInt8') AS uint8. +``` + ## accurateCastOrNull(x, T) {#type_conversion_function-accurate-cast_or_null} -Преобразует входное значение `x` в указанный тип данных `T`. Всегда возвращает тип [Nullable](../../sql-reference/data-types/nullable.md), и возвращает [NULL](../../sql-reference/syntax.md#null-literal), если приведенное значение не может быть представлено в целевом типе. +Преобразует входное значение `x` в указанный тип данных `T`. + +Всегда возвращает тип [Nullable](../../sql-reference/data-types/nullable.md), и возвращает [NULL](../../sql-reference/syntax.md#null-literal), если приведенное значение не может быть представлено в целевом типе. **Синтаксис** @@ -522,9 +560,9 @@ SELECT toTypeName(accurateCastOrNull(5, 'UInt8')); ``` sql SELECT - cast(-1, 'UInt8') as uint8, - cast(128, 'Int8') as int8, - cast('Test', 'FixedString(2)') as fixed_string; + accurateCastOrNull(-1, 'UInt8') as uint8, + accurateCastOrNull(128, 'Int8') as int8, + accurateCastOrNull('Test', 'FixedString(2)') as fixed_string; ``` Результат: From d4580f9fb4b18d4bb9ec1e2870a8d35db06fa6ef Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 11 Feb 2021 01:51:19 +0300 Subject: [PATCH 103/510] Update type-conversion-functions.md --- .../sql-reference/functions/type-conversion-functions.md | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 83cbad6f53b..b452adbde60 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -427,7 +427,12 @@ Result: ## CAST(x, T) {#type_conversion_function-cast} -Converts input value `x` to the `T` data type. The syntax `CAST(x AS t)` is also supported. +Converts input value `x` to the `T` data type. + +The syntax `CAST(x AS t)` is also supported. + +Note, that if value `x` does not fit the bounds of type T, the function overflows. For example, CAST(-1, 'UInt8') returns 255. + **Example** From d4bd82c6c98eb2c4942ce80a42a8f543fd3865e9 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 11 Feb 2021 01:56:12 +0300 Subject: [PATCH 104/510] Update in.md Updates in IN from EN comments. --- docs/ru/sql-reference/operators/in.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/operators/in.md b/docs/ru/sql-reference/operators/in.md index d86d6f9ec57..c2d88a729be 100644 --- a/docs/ru/sql-reference/operators/in.md +++ b/docs/ru/sql-reference/operators/in.md @@ -17,7 +17,8 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... В качестве правой части оператора может быть множество константных выражений, множество кортежей с константными выражениями (показано в примерах выше), а также имя таблицы или подзапрос SELECT в скобках. -ClickHouse допускает различные типы внутри подзапроса `IN`. Для левой стороны он применяет преобразование к типу правой стороны с помощью [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). +ClickHouse допускает различные типы в левой и правой частях подзапроса `IN`. +В этом случае он преобразует левую сторону в тип правой стороны, применяя функцию [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). **Пример** From 60f9f2e913fed325c4747fecbe0e1291265bc666 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 11 Feb 2021 02:03:23 +0300 Subject: [PATCH 105/510] Update type-conversion-functions.md Add Returned values --- docs/en/sql-reference/functions/type-conversion-functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index b452adbde60..268a7565b81 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -544,6 +544,10 @@ accurateCastOrNull(x, T) - `x` — Input value. - `T` — The name of the returned data type. +**Returned value** + +- The value in specified data type `T`. + **Example** Query: From 37979c8b87d4747816446b1939248911a40ea081 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 11 Feb 2021 02:03:36 +0300 Subject: [PATCH 106/510] Update type-conversion-functions.md Add Returned values --- docs/ru/sql-reference/functions/type-conversion-functions.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 3a6d2bd9ca0..e16fa438aed 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -540,6 +540,10 @@ accurateCastOrNull(x, T) - `x` — входное значение. - `T` — имя возвращаемого типа данных. +**Возвращаемое значение** + +- Значение, преобразованное в указанный тип `T`. + **Примеры** Запрос: From 3feded8d0cb562b7d0ed7a8c4bd4939f2524301c Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Thu, 11 Feb 2021 02:03:53 +0300 Subject: [PATCH 107/510] Create type-conversion-functions.md Add Returned values From b574d8331b2cd6c2cd8dfe7d36ad8257b392db83 Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 11:46:31 +0300 Subject: [PATCH 108/510] Updated description --- .../functions/tuple-map-functions.md | 97 ++++++++++++++++++- 1 file changed, 96 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 50015cd996e..d3503937af2 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -137,9 +137,104 @@ Type: [UInt8](../../sql-reference/data-types/int-uint.md). Query: +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; -## mapKeys {#mapKeys} +INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); + +SELECT mapContains(a, 'name') FROM test; + +``` + +Result: + +```text +┌─mapContains(a, 'name')─┐ +│ 1 │ +│ 0 │ +└────────────────────────┘ +``` + +## mapKeys {#mapkeys} + +Returns all the keys from `map` parameter. + +**Syntax** + +```sql +mapKeys(map) +``` + +**Parameters** + +- `map`- Map. + +**Returned value** + +- Array containing all the keys from `map`. + +Type: [Array](../../sql-reference/data-types/array.md). + +**Example** + +Query: + +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; + +INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); + +SELECT mapKeys(a) FROM test; +``` + +Result: + +```text +┌─mapKeys(a)────────────┐ +│ ['name','age'] │ +│ ['number','position'] │ +└───────────────────────┘ +``` ## mapValues {#mapvalues} +Returns all the values from `map` parameter. + +**Syntax** + +```sql +mapKeys(map) +``` + +**Parameters** + +- `map`- Map. + +**Returned value** + +- Array containing all the values from `map`. + +Type: [Array](../../sql-reference/data-types/array.md). + +**Example** + +Query: + +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; + +INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); + +SELECT mapValues(a) FROM test; +``` + +Result: + +```text +┌─mapValues(a)─────┐ +│ ['eleven','11'] │ +│ ['twelve','6.0'] │ +└──────────────────┘ +``` + [Original article](https://clickhouse.tech/docs/en/query_language/functions/tuple-map-functions/) From 3a020d2dd5c4ffda10fb4dd79509f5e04f45e692 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Feb 2021 11:49:12 +0300 Subject: [PATCH 109/510] filter push down for Aggregating --- src/Processors/QueryPlan/AggregatingStep.h | 2 + .../QueryPlan/Optimizations/Optimizations.h | 7 +- .../Optimizations/filterPushDown.cpp | 77 +++++++++++++++++++ 3 files changed, 85 insertions(+), 1 deletion(-) create mode 100644 src/Processors/QueryPlan/Optimizations/filterPushDown.cpp diff --git a/src/Processors/QueryPlan/AggregatingStep.h b/src/Processors/QueryPlan/AggregatingStep.h index 853173895b3..6be92394fab 100644 --- a/src/Processors/QueryPlan/AggregatingStep.h +++ b/src/Processors/QueryPlan/AggregatingStep.h @@ -32,6 +32,8 @@ public: void describeActions(FormatSettings &) const override; void describePipeline(FormatSettings & settings) const override; + const Aggregator::Params & getParams() const { return params; } + private: Aggregator::Params params; bool final; diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 454eab9649a..be7f81e5db0 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -38,14 +38,19 @@ size_t trySplitFilter(QueryPlan::Node * node, QueryPlan::Nodes & nodes); /// Replace chain `FilterStep -> ExpressionStep` to single FilterStep size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &); +/// Move FilterStep down if possible. +/// May split FilterStep and push down only part of it. +size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); + inline const auto & getOptimizations() { - static const std::array optimizations = + static const std::array optimizations = {{ {tryLiftUpArrayJoin, "liftUpArrayJoin"}, {tryPushDownLimit, "pushDownLimit"}, {trySplitFilter, "splitFilter"}, {tryMergeExpressions, "mergeExpressions"}, + {tryPushDownLimit, "pushDownFilter"}, }}; return optimizations; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp new file mode 100644 index 00000000000..82704bcbce9 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -0,0 +1,77 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB::ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace DB::QueryPlanOptimizations +{ + +size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) +{ + if (parent_node->children.size() != 1) + return 0; + + QueryPlan::Node * child_node = parent_node->children.front(); + + auto & parent = parent_node->step; + auto & child = child_node->step; + auto * filter = typeid_cast(parent.get()); + + if (!filter) + return 0; + + const auto & expression = filter->getExpression(); + const auto & filter_column_name = filter->getFilterColumnName(); + bool removes_filter = filter->removesFilterColumn(); + + if (auto * aggregating = typeid_cast(child.get())) + { + const auto & params = aggregating->getParams(); + + Names keys; + keys.reserve(params.keys.size()); + for (auto pos : params.keys) + keys.push_back(params.src_header.getByPosition(pos).name); + + if (auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, keys)) + { + auto it = expression->getIndex().find(filter_column_name); + if (it == expression->getIndex().end()) + { + if (!removes_filter) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", + filter_column_name, expression->dumpDAG()); + + parent = std::make_unique(child->getOutputStream(), expression); + } + + /// Add new Filter step before Aggregating. + /// Expression/Filter -> Aggregating -> Something + auto & node = nodes.emplace_back(); + node.children.swap(child_node->children); + child_node->children.emplace_back(&node); + /// Expression/Filter -> Aggregating -> Filter -> Something + + /// New filter column is added to the end. + auto split_filter_column_name = (*split_filter->getIndex().rbegin())->result_name; + node.step = std::make_unique( + node.children.at(0)->step->getOutputStream(), + std::move(split_filter), std::move(split_filter_column_name), true); + + return 3; + } + } + + return 0; +} + +} From 8b4d9e421a1037f132f8c6511b92ee1a3a21580b Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 12:21:59 +0300 Subject: [PATCH 110/510] Added translation --- .../functions/tuple-map-functions.md | 4 +- .../functions/tuple-map-functions.md | 127 +++++++++++++++++- 2 files changed, 128 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index d3503937af2..a08ca70e851 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -124,7 +124,7 @@ mapContains(map, key) **Parameters** -- `map` — Map. [Type name](relative/path/to/type/dscr.md#type). +- `map` — Map. - `key` — Key. Type matches the type of `map.keys`. **Returned value** @@ -237,4 +237,4 @@ Result: └──────────────────┘ ``` -[Original article](https://clickhouse.tech/docs/en/query_language/functions/tuple-map-functions/) +[Original article](https://clickhouse.tech/docs/en/sql-reference/functions/tuple-map-functions/) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index a2b25e68fe5..6461412aec5 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -116,4 +116,129 @@ select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type └──────────────────────────────┴───────────────────────────────────┘ ``` -[Оригинальная статья](https://clickhouse.tech/docs/en/query_language/functions/tuple-map-functions/) +## mapContains {#mapcontains} + +Определяет, включает ли в себя `map.keys` параметр `key`. + +**Синтаксис** + +``` sql +mapContains(map, key) +``` + +**Параметры** + +- `map` — Map. +- `key` — ключ. Тип соответстует типу `map.keys`. + +**Возвращаемое значение** + +- `1` если `map.keys` включает `key`, иначе `0`. + +Тип: [UInt8](../../sql-reference/data-types/int-uint.md). + +**Пример** + +Запрос: + +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; + +INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); + +SELECT mapContains(a, 'name') FROM test; + +``` + +Результат: + +```text +┌─mapContains(a, 'name')─┐ +│ 1 │ +│ 0 │ +└────────────────────────┘ +``` + +## mapKeys {#mapkeys} + +Возвращает все ключи контейнера `map`. + +**Синтаксис** + +```sql +mapKeys(map) +``` + +**Параметры** + +- `map`- map. + +**Возвращаемое значение** + +- Массив со всеми ключами контейнера `map`. + +Тип: [Array](../../sql-reference/data-types/array.md). + +**Пример** + +Запрос: + +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; + +INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); + +SELECT mapKeys(a) FROM test; +``` + +Результат: + +```text +┌─mapKeys(a)────────────┐ +│ ['name','age'] │ +│ ['number','position'] │ +└───────────────────────┘ +``` + +## mapValues {#mapvalues} + +Возвращает все значения контейнера `map`. + +**Синтаксис** + +```sql +mapKeys(map) +``` + +**Параметры** + +- `map`- map. + +**Возвращаемое значение** + +- Массив со всеми значениями `map`. + +Тип: [Array](../../sql-reference/data-types/array.md). + +**Примеры** + +Запрос: + +```sql +CREATE TABLE test (a Map(String,String)) ENGINE = Memory; + +INSERT INTO test VALUES ({'name':'eleven','age':'11'}), ({'number':'twelve','position':'6.0'}); + +SELECT mapValues(a) FROM test; +``` + +Результат: + +```text +┌─mapValues(a)─────┐ +│ ['eleven','11'] │ +│ ['twelve','6.0'] │ +└──────────────────┘ +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/functions/tuple-map-functions/) From 48b8685d6ef0e690ee7055f0ba1812fa8dfa50e1 Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 12:46:14 +0300 Subject: [PATCH 111/510] minor fixes --- docs/en/sql-reference/functions/tuple-map-functions.md | 4 ++-- docs/ru/sql-reference/functions/tuple-map-functions.md | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index a08ca70e851..f8755f1e2a9 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -167,7 +167,7 @@ mapKeys(map) **Parameters** -- `map`- Map. +- `map` — Map. **Returned value** @@ -208,7 +208,7 @@ mapKeys(map) **Parameters** -- `map`- Map. +- `map` — Map. **Returned value** diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 6461412aec5..22bf1e98369 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -128,7 +128,7 @@ mapContains(map, key) **Параметры** -- `map` — Map. +- `map` — контейнер map. - `key` — ключ. Тип соответстует типу `map.keys`. **Возвращаемое значение** @@ -171,7 +171,7 @@ mapKeys(map) **Параметры** -- `map`- map. +- `map` — контейнер map. **Возвращаемое значение** @@ -212,7 +212,7 @@ mapKeys(map) **Параметры** -- `map`- map. +- `map` — контейнер map. **Возвращаемое значение** From e24b8e8a13ecea65e9d35e53cbe1a7fa44917680 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Feb 2021 15:06:28 +0300 Subject: [PATCH 112/510] Fix ActionsDAG::splitActionsForFilter --- src/Interpreters/ActionsDAG.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index eb1ff9ad998..cd3a2853687 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1311,6 +1311,8 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, actions->inputs.emplace_back(&node); actions->index.insert(&node); } + + stack.pop(); } } } From 2deff0d9d09bab61a149b62acfc49a34e6d4011f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Feb 2021 18:44:10 +0300 Subject: [PATCH 113/510] Fix ActionsDAG::splitActionsForFilter --- src/Interpreters/ActionsDAG.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index cd3a2853687..78254e5139a 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1375,7 +1375,17 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, } else { - (*it)->children.swap(new_children); + Node node; + node.type = ActionType::COLUMN; + node.result_name = (*it)->result_type->getName(); + node.column = DataTypeString().createColumnConst(0, node.result_name); + node.result_type = std::make_shared(); + + auto * right_arg = &nodes.emplace_back(std::move(node)); + auto * left_arg = new_children.front(); + + + (*it)->children = {left_arg, right_arg}; ColumnsWithTypeAndName arguments; arguments.reserve((*it)->children.size()); From a26c8d9eee365d72d151e55416137377e2ea56bb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Feb 2021 19:08:54 +0300 Subject: [PATCH 114/510] Fix const filter resilt for filter push down. --- .../QueryPlan/Optimizations/filterPushDown.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 82704bcbce9..2a42b08af73 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB::ErrorCodes { @@ -41,8 +42,15 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) for (auto pos : params.keys) keys.push_back(params.src_header.getByPosition(pos).name); + std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; if (auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, keys)) { + std::cerr << "===============\n" << expression->dumpDAG() << std::endl; + std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; + + //if (split_filter) + // throw Exception("!!!!", 0); + auto it = expression->getIndex().find(filter_column_name); if (it == expression->getIndex().end()) { @@ -53,6 +61,10 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) parent = std::make_unique(child->getOutputStream(), expression); } + else if ((*it)->column && isColumnConst(*(*it)->column)) + { + parent = std::make_unique(child->getOutputStream(), expression); + } /// Add new Filter step before Aggregating. /// Expression/Filter -> Aggregating -> Something From fc6587319c97c6cccb8b5dc1b108a7b56afaed9e Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 19:29:46 +0300 Subject: [PATCH 115/510] Edit and translated encription-functions --- .../functions/encryption-functions.md | 14 +- .../functions/encryption-functions.md | 305 ++++++++---------- 2 files changed, 150 insertions(+), 169 deletions(-) diff --git a/docs/en/sql-reference/functions/encryption-functions.md b/docs/en/sql-reference/functions/encryption-functions.md index 9e360abfe26..f3e851db29b 100644 --- a/docs/en/sql-reference/functions/encryption-functions.md +++ b/docs/en/sql-reference/functions/encryption-functions.md @@ -55,7 +55,7 @@ CREATE TABLE encryption_test `comment` String, `secret` String ) -ENGINE = Memory +ENGINE = Memory; ``` Insert some data (please avoid storing the keys/ivs in the database as this undermines the whole concept of encryption), also storing 'hints' is unsafe too and used only for illustrative purposes: @@ -110,7 +110,7 @@ Result: Compatible with mysql encryption and resulting ciphertext can be decrypted with [AES_DECRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-decrypt) function. -Will produce same ciphertext as `encrypt` on equal inputs. But when `key` or `iv` are longer than they should normally be, `aes_encrypt_mysql` will stick to what MySQL's `aes_encrypt` does: 'fold' `key` and ignore excess bits of `IV`. +Will produce the same ciphertext as `encrypt` on equal inputs. But when `key` or `iv` are longer than they should normally be, `aes_encrypt_mysql` will stick to what MySQL's `aes_encrypt` does: 'fold' `key` and ignore excess bits of `iv`. Supported encryption modes: @@ -138,7 +138,6 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv]) - Ciphertext binary string. [String](../../sql-reference/data-types/string.md#string). - **Examples** Given equal input `encrypt` and `aes_encrypt_mysql` produce the same ciphertext: @@ -157,7 +156,6 @@ Result: └───────────────────┘ ``` - But `encrypt` fails when `key` or `iv` is longer than expected: Query: @@ -252,7 +250,7 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad]) **Examples** -Re-using table from [encrypt](./encryption-functions.md#encrypt). +Re-using table from [encrypt](#encrypt). Query: @@ -284,6 +282,7 @@ SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920 ``` Result: + ``` text ┌─comment─────────────────────────────┬─plaintext─┐ │ aes-256-cfb128 no IV │ Secret │ @@ -294,7 +293,7 @@ Result: └─────────────────────────────────────┴───────────┘ ``` -Notice how only portion of the data was properly decrypted, and the rest is gibberish since either `mode`, `key`, or `iv` were different upon encryption. +Notice how only a portion of the data was properly decrypted, and the rest is gibberish since either `mode`, `key`, or `iv` were different upon encryption. ## aes_decrypt_mysql {#aes_decrypt_mysql} @@ -331,6 +330,7 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) **Examples** Let's decrypt data we've previously encrypted with MySQL: + ``` sql mysql> SET block_encryption_mode='aes-256-cfb128'; Query OK, 0 rows affected (0.00 sec) @@ -345,11 +345,13 @@ mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviv ``` Query: + ``` sql SELECT aes_decrypt_mysql('aes-256-cfb128', unhex('24E9E4966469'), '123456789101213141516171819202122', 'iviviviviviviviv123456') AS plaintext ``` Result: + ``` text ┌─plaintext─┐ │ Secret │ diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index f1f6516d453..14ce97f5513 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -11,7 +11,7 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438 \u0434\u043b\u044f \u0448 Длина инициализирующего вектора всегда 16 байт (лишнии байты игнорируются). -Обратите внимание, что эти функции работают медленно. +Обратите внимание, что до версии Clickhouse 21.1 эти функции работают медленно. ## encrypt {#encrypt} @@ -41,7 +41,7 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad]) **Возвращаемое значение** -- Зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). +- Двоичная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). **Примеры** @@ -52,57 +52,38 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad]) ``` sql CREATE TABLE encryption_test ( - input String, - key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'), - iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'), - key32 String DEFAULT substring(key, 1, 32), - key24 String DEFAULT substring(key, 1, 24), - key16 String DEFAULT substring(key, 1, 16) -) Engine = Memory; + `comment` String, + `secret` String +) +ENGINE = Memory; ``` -Вставим эти данные: +Вставим некоторые данные (замечание: не храните ключи или инициализирующие векторы в базе данных, так как это компрометирует всю концепцию шифрования), также хранение "подсказок" небезопасно и используется только для наглядности: Запрос: ``` sql -INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?'); +INSERT INTO encryption_test VALUES('aes-256-cfb128 no IV', encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212')),\ +('aes-256-cfb128 no IV, different key', encrypt('aes-256-cfb128', 'Secret', 'keykeykeykeykeykeykeykeykeykeyke')),\ +('aes-256-cfb128 with IV', encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv')),\ +('aes-256-cbc no IV', encrypt('aes-256-cbc', 'Secret', '12345678910121314151617181920212')); ``` -Пример без `iv`: - Запрос: ``` sql -SELECT 'aes-128-ecb' AS mode, hex(encrypt(mode, input, key16)) FROM encryption_test; +SELECT comment, hex(secret) FROM encryption_test; ``` Результат: ``` text -┌─mode────────┬─hex(encrypt('aes-128-ecb', input, key16))────────────────────────┐ -│ aes-128-ecb │ 4603E6862B0D94BBEC68E0B0DF51D60F │ -│ aes-128-ecb │ 3004851B86D3F3950672DE7085D27C03 │ -│ aes-128-ecb │ E807F8C8D40A11F65076361AFC7D8B68D8658C5FAA6457985CAA380F16B3F7E4 │ -└─────────────┴──────────────────────────────────────────────────────────────────┘ -``` - -Пример с `iv`: - -Запрос: - -``` sql -SELECT 'aes-256-ctr' AS mode, hex(encrypt(mode, input, key32, iv)) FROM encryption_test; -``` - -Результат: - -``` text -┌─mode────────┬─hex(encrypt('aes-256-ctr', input, key32, iv))─┐ -│ aes-256-ctr │ │ -│ aes-256-ctr │ 7FB039F7 │ -│ aes-256-ctr │ 5CBD20F7ABD3AC41FCAA1A5C0E119E2B325949 │ -└─────────────┴───────────────────────────────────────────────┘ +┌─comment─────────────────────────────┬─hex(secret)──────────────────────┐ +│ aes-256-cfb128 no IV │ B4972BDC4459 │ +│ aes-256-cfb128 no IV, different key │ 2FF57C092DC9 │ +│ aes-256-cfb128 with IV │ 5E6CB398F653 │ +│ aes-256-cbc no IV │ 1BC0629A92450D9E73A00E7D02CF4142 │ +└─────────────────────────────────────┴──────────────────────────────────┘ ``` Пример в режиме `-gcm`: @@ -110,41 +91,27 @@ SELECT 'aes-256-ctr' AS mode, hex(encrypt(mode, input, key32, iv)) FROM encrypti Запрос: ``` sql -SELECT 'aes-256-gcm' AS mode, hex(encrypt(mode, input, key32, iv)) FROM encryption_test; +INSERT INTO encryption_test VALUES('aes-256-gcm', encrypt('aes-256-gcm', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv')), \ +('aes-256-gcm with AAD', encrypt('aes-256-gcm', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv', 'aad')); + +SELECT comment, hex(secret) FROM encryption_test WHERE comment LIKE '%gcm%'; ``` Результат: ``` text -┌─mode────────┬─hex(encrypt('aes-256-gcm', input, key32, iv))──────────────────────────┐ -│ aes-256-gcm │ E99DBEBC01F021758352D7FBD9039EFA │ -│ aes-256-gcm │ 8742CE3A7B0595B281C712600D274CA881F47414 │ -│ aes-256-gcm │ A44FD73ACEB1A64BDE2D03808A2576EDBB60764CC6982DB9AF2C33C893D91B00C60DC5 │ -└─────────────┴────────────────────────────────────────────────────────────────────────┘ -``` - -Пример в режиме `-gcm` и с `aad`: - -Запрос: - -``` sql -SELECT 'aes-192-gcm' AS mode, hex(encrypt(mode, input, key24, iv, 'AAD')) FROM encryption_test; -``` - -Результат: - -``` text -┌─mode────────┬─hex(encrypt('aes-192-gcm', input, key24, iv, 'AAD'))───────────────────┐ -│ aes-192-gcm │ 04C13E4B1D62481ED22B3644595CB5DB │ -│ aes-192-gcm │ 9A6CF0FD2B329B04EAD18301818F016DF8F77447 │ -│ aes-192-gcm │ B961E9FD9B940EBAD7ADDA75C9F198A40797A5EA1722D542890CC976E21113BBB8A7AA │ -└─────────────┴────────────────────────────────────────────────────────────────────────┘ +┌─comment──────────────┬─hex(secret)──────────────────────────────────┐ +│ aes-256-gcm │ A8A3CCBC6426CFEEB60E4EAE03D3E94204C1B09E0254 │ +│ aes-256-gcm with AAD │ A8A3CCBC6426D9A1017A0A932322F1852260A4AD6837 │ +└──────────────────────┴──────────────────────────────────────────────┘ ``` ## aes_encrypt_mysql {#aes_encrypt_mysql} Совместима с шифрованием myqsl, результат может быть расшифрован функцией [AES_DECRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-decrypt). +При одинаковых вводных зашифрованный текст будет совпадать с результатом `encrypt`. Однако, когда `key` или `iv` длиннее, чем должны быть, `aes_encrypt_mysql` будет работать аналогично MySQL `aes_encrypt`: свернет ключ и проигнорирует лишнюю часть `iv`. + Функция поддерживает шифрофание данных следующими режимами: - aes-128-ecb, aes-192-ecb, aes-256-ecb @@ -156,7 +123,7 @@ SELECT 'aes-192-gcm' AS mode, hex(encrypt(mode, input, key24, iv, 'AAD')) FROM e **Синтаксис** -```sql +``` sql aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv]) ``` @@ -164,78 +131,96 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv]) - `mode` — режим шифрования. [String](../../sql-reference/data-types/string.md#string). - `plaintext` — текст, который будет зашифрован. [String](../../sql-reference/data-types/string.md#string). -- `key` — ключ шифрования. [String](../../sql-reference/data-types/string.md#string). -- `iv` — инициализирующий вектор. Необязателен. [String](../../sql-reference/data-types/string.md#string). +- `key` — ключ шифрования. Если ключ длиннее, чем требует режим шифрования, производится специфичная для MySQL свертка ключа. [String](../../sql-reference/data-types/string.md#string). +- `iv` — инициализирующий вектор. Необязателен, учитываются только первые 16 байтов. [String](../../sql-reference/data-types/string.md#string). **Возвращаемое значение** -- Зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). +- Двоичная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). **Примеры** -Создадим такую таблицу: +При одинаковых вводных результаты шифрования `encrypt` и `aes_encrypt_mysql` будут совпадать. Запрос: ``` sql -CREATE TABLE encryption_test -( - input String, - key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'), - iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'), - key32 String DEFAULT substring(key, 1, 32), - key24 String DEFAULT substring(key, 1, 24), - key16 String DEFAULT substring(key, 1, 16) -) Engine = Memory; -``` - -Вставим эти данные: - -Запрос: - -``` sql -INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?'); -``` - -Пример без `iv`: - -Запрос: - -``` sql -SELECT 'aes-128-cbc' AS mode, hex(aes_encrypt_mysql(mode, input, key32)) FROM encryption_test; +SELECT encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv') = aes_encrypt_mysql('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', 'iviviviviviviviv') AS ciphertexts_equal; ``` Результат: ``` text -┌─mode────────┬─hex(aes_encrypt_mysql('aes-128-cbc', input, key32))──────────────┐ -│ aes-128-cbc │ FEA8CFDE6EE2C6E7A2CC6ADDC9F62C83 │ -│ aes-128-cbc │ 78B16CD4BE107660156124C5FEE6454A │ -│ aes-128-cbc │ 67C0B119D96F18E2823968D42871B3D179221B1E7EE642D628341C2B29BA2E18 │ -└─────────────┴──────────────────────────────────────────────────────────────────┘ +┌─ciphertexts_equal─┐ +│ 1 │ +└───────────────────┘ ``` -Пример с `iv`: +Но `encrypt` генерирует исключение, когда `key` или `iv` длиннее, чем нужно: Запрос: ``` sql -SELECT 'aes-256-cfb128' AS mode, hex(aes_encrypt_mysql(mode, input, key32, iv)) FROM encryption_test; +SELECT encrypt('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123'); ``` Результат: ``` text -┌─mode───────────┬─hex(aes_encrypt_mysql('aes-256-cfb128', input, key32, iv))─┐ -│ aes-256-cfb128 │ │ -│ aes-256-cfb128 │ 7FB039F7 │ -│ aes-256-cfb128 │ 5CBD20F7ABD3AC41FCAA1A5C0E119E2BB5174F │ -└────────────────┴────────────────────────────────────────────────────────────┘ +Received exception from server (version 21.1.2): +Code: 36. DB::Exception: Received from localhost:9000. DB::Exception: Invalid key size: 33 expected 32: While processing encrypt('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123'). +``` + +Тогда как `aes_encrypt_mysql` возвращает совместимый с MySQL вывод: + +Запрос: + +``` sql +SELECT hex(aes_encrypt_mysql('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123')) AS ciphertext; +``` + +Результат: + +```text +┌─ciphertext───┐ +│ 24E9E4966469 │ +└──────────────┘ +``` + +Если передать `iv` еще длиннее, результат останется таким же: + +Запрос: + +``` sql +SELECT hex(aes_encrypt_mysql('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123456')) AS ciphertext +``` + +Результат: + +``` text +┌─ciphertext───┐ +│ 24E9E4966469 │ +└──────────────┘ +``` + +Это совпадает с тем, что выводит MySQL с такими же вводными: + +``` sql +mysql> SET block_encryption_mode='aes-256-cfb128'; +Query OK, 0 rows affected (0.00 sec) + +mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviviviviviviv123456') as ciphertext; ++------------------------+ +| ciphertext | ++------------------------+ +| 0x24E9E4966469 | ++------------------------+ +1 row in set (0.00 sec) ``` ## decrypt {#decrypt} -Функция поддерживает расшифровку данных следующими режимами: +Функция расшифровывает зашифрованный текст в обычный следующими режимами: - aes-128-ecb, aes-192-ecb, aes-256-ecb - aes-128-cbc, aes-192-cbc, aes-256-cbc @@ -247,7 +232,7 @@ SELECT 'aes-256-cfb128' AS mode, hex(aes_encrypt_mysql(mode, input, key32, iv)) **Синтаксис** -```sql +``` sql decrypt('mode', 'ciphertext', 'key' [, iv, aad]) ``` @@ -265,51 +250,57 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad]) **Примеры** -Создадим такую таблицу: +Используется таблица из [encrypt](#encrypt). Запрос: ``` sql -CREATE TABLE encryption_test -( - input String, - key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'), - iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'), - key32 String DEFAULT substring(key, 1, 32), - key24 String DEFAULT substring(key, 1, 24), - key16 String DEFAULT substring(key, 1, 16) -) Engine = Memory; -``` - -Вставим эти данные: - -Запрос: - -``` sql -INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?'); -``` - -Запрос: - -``` sql - -SELECT 'aes-128-ecb' AS mode, decrypt(mode, encrypt(mode, input, key16), key16) FROM encryption_test; +SELECT comment, hex(secret) FROM encryption_test; ``` Результат: -```text -┌─mode────────┬─decrypt('aes-128-ecb', encrypt('aes-128-ecb', input, key16), key16)─┐ -│ aes-128-ecb │ │ -│ aes-128-ecb │ text │ -│ aes-128-ecb │ What Is ClickHouse? │ -└─────────────┴─────────────────────────────────────────────────────────────────────┘ +``` text +┌─comment──────────────┬─hex(secret)──────────────────────────────────┐ +│ aes-256-gcm │ A8A3CCBC6426CFEEB60E4EAE03D3E94204C1B09E0254 │ +│ aes-256-gcm with AAD │ A8A3CCBC6426D9A1017A0A932322F1852260A4AD6837 │ +└──────────────────────┴──────────────────────────────────────────────┘ +┌─comment─────────────────────────────┬─hex(secret)──────────────────────┐ +│ aes-256-cfb128 no IV │ B4972BDC4459 │ +│ aes-256-cfb128 no IV, different key │ 2FF57C092DC9 │ +│ aes-256-cfb128 with IV │ 5E6CB398F653 │ +│ aes-256-cbc no IV │ 1BC0629A92450D9E73A00E7D02CF4142 │ +└─────────────────────────────────────┴──────────────────────────────────┘ ``` +Теперь попытаемся расшифровать эти данные: + +Запрос: + +``` sql +SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920212') as plaintext FROM encryption_test +``` + +Результат: + +``` text +┌─comment─────────────────────────────┬─plaintext─┐ +│ aes-256-cfb128 no IV │ Secret │ +│ aes-256-cfb128 no IV, different key │ �4� + � │ +│ aes-256-cfb128 with IV │ ���6�~ │ + │aes-256-cbc no IV │ �2*4�h3c�4w��@ +└─────────────────────────────────────┴───────────┘ +``` + +Обратите внимание, что только часть данных была расшифрована, а остальное является бессмыслицей, как как `mode`, `key`, или `iv`были другими во время шифрования. + ## aes_decrypt_mysql {#aes_decrypt_mysql} Совместима с шифрованием myqsl и может расшифровать данные, зашифрованные функцией [AES_ENCRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-encrypt). +При одинаковых вводных расшифрованный текст будет совпадать с результатом `decrypt`. Однако, когда `key` или `iv` длиннее, чем должны быть, `aes_decrypt_mysql` будет работать аналогично MySQL `aes_decrypt`: свернет ключ и проигнорирует лишнюю часть `iv`. + Функция поддерживает расшифровку данных следующими режимами: - aes-128-ecb, aes-192-ecb, aes-256-ecb @@ -321,7 +312,7 @@ SELECT 'aes-128-ecb' AS mode, decrypt(mode, encrypt(mode, input, key16), key16) **Синтаксис** -```sql +``` sql aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) ``` @@ -332,51 +323,39 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) - `key` — ключ шифрования. [String](../../sql-reference/data-types/string.md#string). - `iv` — инициализирующий вектор. Необязателен. [String](../../sql-reference/data-types/string.md#string). - **Возвращаемое значение** - Расшифрованная строка. [String](../../sql-reference/data-types/string.md#string). **Примеры** -Создадим такую таблицу: +Расшифруем данные, которые до этого зашифровали с помощью MySQL: -Запрос: ``` sql -CREATE TABLE encryption_test -( - input String, - key String DEFAULT unhex('fb9958e2e897ef3fdb49067b51a24af645b3626eed2f9ea1dc7fd4dd71b7e38f9a68db2a3184f952382c783785f9d77bf923577108a88adaacae5c141b1576b0'), - iv String DEFAULT unhex('8CA3554377DFF8A369BC50A89780DD85'), - key32 String DEFAULT substring(key, 1, 32), - key24 String DEFAULT substring(key, 1, 24), - key16 String DEFAULT substring(key, 1, 16) -) Engine = Memory; -``` +mysql> SET block_encryption_mode='aes-256-cfb128'; +Query OK, 0 rows affected (0.00 sec) -Вставим эти данные: - -Запрос: - -``` sql -INSERT INTO encryption_test (input) VALUES (''), ('text'), ('What Is ClickHouse?'); +mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviviviviviviv123456') as ciphertext; ++------------------------+ +| ciphertext | ++------------------------+ +| 0x24E9E4966469 | ++------------------------+ +1 row in set (0.00 sec) ``` Запрос: ``` sql -SELECT 'aes-128-cbc' AS mode, aes_decrypt_mysql(mode, aes_encrypt_mysql(mode, input, key), key) FROM encryption_test; +SELECT aes_decrypt_mysql('aes-256-cfb128', unhex('24E9E4966469'), '123456789101213141516171819202122', 'iviviviviviviviv123456') AS plaintext ``` Результат: ``` text -┌─mode────────┬─aes_decrypt_mysql('aes-128-cbc', aes_encrypt_mysql('aes-128-cbc', input, key), key)─┐ -│ aes-128-cbc │ │ -│ aes-128-cbc │ text │ -│ aes-128-cbc │ What Is ClickHouse? │ -└─────────────┴─────────────────────────────────────────────────────────────────────────────────────┘ +┌─plaintext─┐ +│ Secret │ +└───────────┘ ``` - [Original article](https://clickhouse.tech/docs/ru/sql-reference/functions/encryption_functions/) From 2a9a6cf4048969d1fa670fb7afac18d57b86649a Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 19:46:23 +0300 Subject: [PATCH 116/510] Edited and translated parametric-functions --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 4b3bf12aa8c..2d2df3bd6cb 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -241,7 +241,7 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN) **Parameters** -- `window` — Length of the sliding window. The unit of `window` depends on the timestamp itself and varies. Determined using the expression `timestamp of cond2 <= timestamp of cond1 + window`. +- `window` — Length of the sliding window. The unit of `window` depends on the `timestamp` itself and varies. Determined using the expression `timestamp of cond2 <= timestamp of cond1 + window`. - `mode` - It is an optional argument. - `'strict'` - When the `'strict'` is set, the windowFunnel() applies conditions only for the unique values. - `timestamp` — Name of the column containing the timestamp. Data types supported: [Date](../../sql-reference/data-types/date.md), [DateTime](../../sql-reference/data-types/datetime.md#data_type-datetime) and other unsigned integer types (note that even though timestamp supports the `UInt64` type, it’s value can’t exceed the Int64 maximum, which is 2^63 - 1). diff --git a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md index f20acaa45c3..2c367882714 100644 --- a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md @@ -239,7 +239,7 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN) **Параметры** -- `window` — ширина скользящего окна по времени в секундах. [UInt](../../sql-reference/aggregate-functions/parametric-functions.md). +- `window` — ширина скользящего окна по времени. Единица измерения зависит от `timestamp` и может варьироваться. Определяется выражением `timestamp от cond2 <= timestamp от cond1 + window`. - `mode` - необязательный параметр. Если установлено значение `'strict'`, то функция `windowFunnel()` применяет условия только для уникальных значений. - `timestamp` — имя столбца, содержащего временные отметки. [Date](../../sql-reference/aggregate-functions/parametric-functions.md), [DateTime](../../sql-reference/aggregate-functions/parametric-functions.md#data_type-datetime) и другие параметры с типом `Integer`. В случае хранения меток времени в столбцах с типом `UInt64`, максимально допустимое значение соответствует ограничению для типа `Int64`, т.е. равно `2^63-1`. - `cond` — условия или данные, описывающие цепочку событий. [UInt8](../../sql-reference/aggregate-functions/parametric-functions.md). From cd11212bba784958174fdfbd334622a533686756 Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 19:57:41 +0300 Subject: [PATCH 117/510] Edited and translated settings --- docs/en/operations/settings/settings.md | 4 ++-- docs/ru/operations/settings/settings.md | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index c7ee48c11bf..70809885a99 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1956,8 +1956,8 @@ Default value: 16. **See Also** -- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine -- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine +- [Kafka](../../engines/table-engines/integrations/kafka.md#kafka) engine. +- [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine) engine. ## validate_polygons {#validate_polygons} diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 1352fe850df..fed10d21920 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1939,6 +1939,21 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; Значение по умолчанию: 16. +## background_message_broker_schedule_pool_size {#background_message_broker_schedule_pool_size} + +Задает количество потоков для вывода потокового вывода сообщений. Настройка применяется при запуске сервера ClickHouse и не может быть изменена в пользовательском сеансе. + +Допустимые значения: + +- Положительное целое число. + +Значение по умолчанию: 16. + +**Смотрите также** + +- Движок [Kafka](../../engines/table-engines/integrations/kafka.md#kafka). +- Движок [RabbitMQ](../../engines/table-engines/integrations/rabbitmq.md#rabbitmq-engine). + ## format_avro_schema_registry_url {#format_avro_schema_registry_url} Задает URL реестра схем [Confluent](https://docs.confluent.io/current/schema-registry/index.html) для использования с форматом [AvroConfluent](../../interfaces/formats.md#data-format-avro-confluent). From 93ea1e5e82da3a3eb07dbe9daa355d3ab31accf5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 11 Feb 2021 20:13:59 +0300 Subject: [PATCH 118/510] Comment output --- .../QueryPlan/Optimizations/filterPushDown.cpp | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 2a42b08af73..a5f1d37e2f2 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -42,14 +42,11 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) for (auto pos : params.keys) keys.push_back(params.src_header.getByPosition(pos).name); - std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; + // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; if (auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, keys)) { - std::cerr << "===============\n" << expression->dumpDAG() << std::endl; - std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; - - //if (split_filter) - // throw Exception("!!!!", 0); + // std::cerr << "===============\n" << expression->dumpDAG() << std::endl; + // std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; auto it = expression->getIndex().find(filter_column_name); if (it == expression->getIndex().end()) From 838dab756491d5bdcd6151fb5075756d0807b807 Mon Sep 17 00:00:00 2001 From: George Date: Thu, 11 Feb 2021 21:07:38 +0300 Subject: [PATCH 119/510] Edit and translated Kafka --- .../table-engines/integrations/kafka.md | 22 +++++++++---------- .../table-engines/integrations/kafka.md | 19 +++++++++------- 2 files changed, 22 insertions(+), 19 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/kafka.md b/docs/en/engines/table-engines/integrations/kafka.md index c519d6bb136..fb1df62bb15 100644 --- a/docs/en/engines/table-engines/integrations/kafka.md +++ b/docs/en/engines/table-engines/integrations/kafka.md @@ -38,20 +38,20 @@ SETTINGS Required parameters: -- `kafka_broker_list` – A comma-separated list of brokers (for example, `localhost:9092`). -- `kafka_topic_list` – A list of Kafka topics. -- `kafka_group_name` – A group of Kafka consumers. Reading margins are tracked for each group separately. If you don’t want messages to be duplicated in the cluster, use the same group name everywhere. -- `kafka_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. +- `kafka_broker_list` — A comma-separated list of brokers (for example, `localhost:9092`). +- `kafka_topic_list` — A list of Kafka topics. +- `kafka_group_name` — A group of Kafka consumers. Reading margins are tracked for each group separately. If you don’t want messages to be duplicated in the cluster, use the same group name everywhere. +- `kafka_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: -- `kafka_row_delimiter` – Delimiter character, which ends the message. -- `kafka_schema` – Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. -- `kafka_num_consumers` – The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition. -- `kafka_max_block_size` - The maximum batch size (in messages) for poll (default: `max_block_size`). -- `kafka_skip_broken_messages` – Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data). -- `kafka_commit_every_batch` - Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`). -- `kafka_thread_per_consumer` - Provide independent thread for each consumer (default: `0`). When enabled, every consumer flush the data independently, in parallel (otherwise - rows from several consumers squashed to form one block). +- `kafka_row_delimiter` — Delimiter character, which ends the message. +- `kafka_schema` — Parameter that must be used if the format requires a schema definition. For example, [Cap’n Proto](https://capnproto.org/) requires the path to the schema file and the name of the root `schema.capnp:Message` object. +- `kafka_num_consumers` — The number of consumers per table. Default: `1`. Specify more consumers if the throughput of one consumer is insufficient. The total number of consumers should not exceed the number of partitions in the topic, since only one consumer can be assigned per partition. +- `kafka_max_block_size` — The maximum batch size (in messages) for poll (default: `max_block_size`). +- `kafka_skip_broken_messages` — Kafka message parser tolerance to schema-incompatible messages per block. Default: `0`. If `kafka_skip_broken_messages = N` then the engine skips *N* Kafka messages that cannot be parsed (a message equals a row of data). +- `kafka_commit_every_batch` — Commit every consumed and handled batch instead of a single commit after writing a whole block (default: `0`). +- `kafka_thread_per_consumer` — Provide independent thread for each consumer (default: `0`). When enabled, every consumer flush the data independently, in parallel (otherwise — rows from several consumers squashed to form one block). Examples: diff --git a/docs/ru/engines/table-engines/integrations/kafka.md b/docs/ru/engines/table-engines/integrations/kafka.md index 940fee2452b..2b9dfcd49da 100644 --- a/docs/ru/engines/table-engines/integrations/kafka.md +++ b/docs/ru/engines/table-engines/integrations/kafka.md @@ -31,21 +31,24 @@ SETTINGS [kafka_schema = '',] [kafka_num_consumers = N,] [kafka_skip_broken_messages = N] + [kafka_commit_every_batch = 0,] + [kafka_thread_per_consumer = 0] ``` Обязательные параметры: -- `kafka_broker_list` – перечень брокеров, разделенный запятыми (`localhost:9092`). -- `kafka_topic_list` – перечень необходимых топиков Kafka. -- `kafka_group_name` – группа потребителя Kafka. Отступы для чтения отслеживаются для каждой группы отдельно. Если необходимо, чтобы сообщения не повторялись на кластере, используйте везде одно имя группы. -- `kafka_format` – формат сообщений. Названия форматов должны быть теми же, что можно использовать в секции `FORMAT`, например, `JSONEachRow`. Подробнее читайте в разделе [Форматы](../../../interfaces/formats.md). +- `kafka_broker_list` — перечень брокеров, разделенный запятыми (`localhost:9092`). +- `kafka_topic_list` — перечень необходимых топиков Kafka. +- `kafka_group_name` — группа потребителя Kafka. Отступы для чтения отслеживаются для каждой группы отдельно. Если необходимо, чтобы сообщения не повторялись на кластере, используйте везде одно имя группы. +- `kafka_format` — формат сообщений. Названия форматов должны быть теми же, что можно использовать в секции `FORMAT`, например, `JSONEachRow`. Подробнее читайте в разделе [Форматы](../../../interfaces/formats.md). Опциональные параметры: -- `kafka_row_delimiter` – символ-разделитель записей (строк), которым завершается сообщение. -- `kafka_schema` – опциональный параметр, необходимый, если используется формат, требующий определения схемы. Например, [Cap’n Proto](https://capnproto.org/) требует путь к файлу со схемой и название корневого объекта `schema.capnp:Message`. -- `kafka_num_consumers` – количество потребителей (consumer) на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. Общее число потребителей не должно превышать количество партиций в топике, так как на одну партицию может быть назначено не более одного потребителя. -- `kafka_skip_broken_messages` – максимальное количество некорректных сообщений в блоке. Если `kafka_skip_broken_messages = N`, то движок отбрасывает `N` сообщений Кафки, которые не получилось обработать. Одно сообщение в точности соответствует одной записи (строке). Значение по умолчанию – 0. +- `kafka_row_delimiter` — символ-разделитель записей (строк), которым завершается сообщение. +- `kafka_schema` — опциональный параметр, необходимый, если используется формат, требующий определения схемы. Например, [Cap’n Proto](https://capnproto.org/) требует путь к файлу со схемой и название корневого объекта `schema.capnp:Message`. +- `kafka_num_consumers` — количество потребителей (consumer) на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. Общее число потребителей не должно превышать количество партиций в топике, так как на одну партицию может быть назначено не более одного потребителя. +- `kafka_skip_broken_messages` — максимальное количество некорректных сообщений в блоке. Если `kafka_skip_broken_messages = N`, то движок отбрасывает `N` сообщений Кафки, которые не получилось обработать. Одно сообщение в точности соответствует одной записи (строке). Значение по умолчанию – 0. +- `kafka_thread_per_consumer` — снабжает каждого потребителя независимым потоком (по умолчанию `0`). При включенном состоянии каждый потребитель сбрасывает данные независимо и параллельно (иначе — строки от нескольких потребителей склеиваются в один блок). Примеры From 4c8632bd9ab32322af29abb04cf70c39c6cd3c79 Mon Sep 17 00:00:00 2001 From: George Date: Fri, 12 Feb 2021 00:22:55 +0300 Subject: [PATCH 120/510] Minor fixes --- docs/ru/operations/settings/settings.md | 2 +- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index fed10d21920..a7754cfc421 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -1941,7 +1941,7 @@ SELECT idx, i FROM null_in WHERE i IN (1, NULL) SETTINGS transform_null_in = 1; ## background_message_broker_schedule_pool_size {#background_message_broker_schedule_pool_size} -Задает количество потоков для вывода потокового вывода сообщений. Настройка применяется при запуске сервера ClickHouse и не может быть изменена в пользовательском сеансе. +Задает количество потоков для фонового потокового вывода сообщений. Настройка применяется при запуске сервера ClickHouse и не может быть изменена в пользовательском сеансе. Допустимые значения: diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 14ce97f5513..91b26a2415d 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -293,7 +293,7 @@ SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920 └─────────────────────────────────────┴───────────┘ ``` -Обратите внимание, что только часть данных была расшифрована, а остальное является бессмыслицей, как как `mode`, `key`, или `iv`были другими во время шифрования. +Обратите внимание, что только часть данных была расшифрована, а остальное является бессмыслицей, как как `mode`, `key`, или `iv` были другими во время шифрования. ## aes_decrypt_mysql {#aes_decrypt_mysql} From 7e75965af887d7a7d68699b7bac5e0401cbf02c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 12:35:26 +0300 Subject: [PATCH 121/510] Fix ActionsDAG::splitActionsForFilter --- src/Interpreters/ActionsDAG.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 78254e5139a..6a7dbc47230 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1168,7 +1168,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, { auto & inputs_list = inputs_map[name]; if (inputs_list.empty()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find input {} in ActionsDAG. DAG:\n{}", name, dumpDAG()); + continue; allowed_nodes.emplace(inputs_list.front()); inputs_list.pop_front(); From 443a3e7e6fd2452bf3efa8e4ab2a349feaf3b29f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 13:12:31 +0300 Subject: [PATCH 122/510] Fix limit push down. --- src/Processors/QueryPlan/Optimizations/Optimizations.h | 4 ++-- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index be7f81e5db0..a5c3af488a9 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -40,7 +40,7 @@ size_t tryMergeExpressions(QueryPlan::Node * parent_node, QueryPlan::Nodes &); /// Move FilterStep down if possible. /// May split FilterStep and push down only part of it. -size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); +size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes); inline const auto & getOptimizations() { @@ -50,7 +50,7 @@ inline const auto & getOptimizations() {tryPushDownLimit, "pushDownLimit"}, {trySplitFilter, "splitFilter"}, {tryMergeExpressions, "mergeExpressions"}, - {tryPushDownLimit, "pushDownFilter"}, + {tryPushDownFilter, "pushDownFilter"}, }}; return optimizations; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index a5f1d37e2f2..ac95d69d237 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -42,11 +42,11 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) for (auto pos : params.keys) keys.push_back(params.src_header.getByPosition(pos).name); - // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; + std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; if (auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, keys)) { - // std::cerr << "===============\n" << expression->dumpDAG() << std::endl; - // std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; + std::cerr << "===============\n" << expression->dumpDAG() << std::endl; + std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; auto it = expression->getIndex().find(filter_column_name); if (it == expression->getIndex().end()) From 93e1428f2119ecc5b3979ff5bff0d0304327579c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 13:51:16 +0300 Subject: [PATCH 123/510] Fix limit push down. --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index ac95d69d237..ec005e59729 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -15,7 +15,7 @@ namespace DB::ErrorCodes namespace DB::QueryPlanOptimizations { -size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) +size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) { if (parent_node->children.size() != 1) return 0; @@ -42,11 +42,11 @@ size_t tryPushDownLimit(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) for (auto pos : params.keys) keys.push_back(params.src_header.getByPosition(pos).name); - std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; + // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; if (auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, keys)) { - std::cerr << "===============\n" << expression->dumpDAG() << std::endl; - std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; + // std::cerr << "===============\n" << expression->dumpDAG() << std::endl; + // std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; auto it = expression->getIndex().find(filter_column_name); if (it == expression->getIndex().end()) From 683d793cc289ec12b8885efe1405b79a22350a36 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 14:31:14 +0300 Subject: [PATCH 124/510] Update test. --- .../01655_plan_optimizations.reference | 33 +++++++++++- .../0_stateless/01655_plan_optimizations.sh | 51 ++++++++++++++++++- 2 files changed, 80 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index fda40305f9d..510224146ed 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -1,7 +1,7 @@ -sipHash should be calculated after filtration +> sipHash should be calculated after filtration FUNCTION sipHash64 Filter column: equals -sorting steps should know about limit +> sorting steps should know about limit Limit 10 MergingSorted Limit 10 @@ -9,3 +9,32 @@ MergeSorting Limit 10 PartialSorting Limit 10 +-- filter push down -- +> filter should be pushed down after aggregating +Aggregating +Filter +> filter should be pushed down after aggregating, column after aggregation is const +COLUMN Const(UInt8) -> notEquals(y, 0) +Aggregating +Filter +Filter +> one condition of filter should be pushed down after aggregating, other condition is aliased +Filter column +ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4)) +Aggregating +Filter column: notEquals(y, 0) +> one condition of filter should be pushed down after aggregating, other condition is casted +Filter column +FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) +Aggregating +Filter column: notEquals(y, 0) +> one condition of filter should be pushed down after aggregating, other two conditions are ANDed +Filter column +FUNCTION and(minus(s, 4) :: 2, minus(s, 8) :: 1) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) +Aggregating +Filter column: notEquals(y, 0) +> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased +Filter column +ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) +Aggregating +Filter column: and(minus(y, 4), notEquals(y, 0)) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 4f3541f9dde..ea76d15c648 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -4,7 +4,54 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -echo "sipHash should be calculated after filtration" +echo "> sipHash should be calculated after filtration" $CLICKHOUSE_CLIENT -q "explain actions = 1 select sum(x), sum(y) from (select sipHash64(number) as x, bitAnd(number, 1024) as y from numbers_mt(1000000000) limit 1000000000) where y = 0" | grep -o "FUNCTION sipHash64\|Filter column: equals" -echo "sorting steps should know about limit" +echo "> sorting steps should know about limit" $CLICKHOUSE_CLIENT -q "explain actions = 1 select number from (select number from numbers(500000000) order by -number) limit 10" | grep -o "MergingSorted\|MergeSorting\|PartialSorting\|Limit 10" + +echo "-- filter push down --" +echo "> filter should be pushed down after aggregating" +$CLICKHOUSE_CLIENT -q " + explain select * from (select sum(x), y from ( + select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 + settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter" + +echo "> filter should be pushed down after aggregating, column after aggregation is const" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select *, y != 0 from (select sum(x), y from ( + select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 + settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y, 0)" + +echo "> one condition of filter should be pushed down after aggregating, other condition is aliased" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select * from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s != 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))" + +echo "> one condition of filter should be pushed down after aggregating, other condition is casted" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select * from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s - 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" + +echo "> one condition of filter should be pushed down after aggregating, other two conditions are ANDed" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select * from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s - 8 and s - 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 2, minus(s, 8) :: 1) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" + +echo "> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" +$CLICKHOUSE_CLIENT -q " + explain optimize = 1, actions = 1 select * from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s != 8 and y - 4 + settings enable_optimize_predicate_expression=0" | + grep -o "Aggregating\|Filter column\|Filter column: and(minus(y, 4), notEquals(y, 0))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" From bbed905461d9e08adaa1303f71c228d2f62fff8c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 18:20:54 +0300 Subject: [PATCH 125/510] Fix ActionsDAG::removeUnusedResult --- src/Interpreters/ActionsDAG.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 6a7dbc47230..255c774bbf9 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -490,6 +490,11 @@ bool ActionsDAG::removeUnusedResult(const std::string & column_name) if (col == child) return false; + /// Do not remove input if it was mentioned in index several times. + for (const auto * node : index) + if (col == node) + return false; + /// Remove from nodes and inputs. for (auto jt = nodes.begin(); jt != nodes.end(); ++jt) { From 90c7cf5a5293a32654e97cc8b4f8cb1d2090d3be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 18:24:31 +0300 Subject: [PATCH 126/510] Push down for ArrayJoin --- .../Optimizations/filterPushDown.cpp | 116 ++++++++++++------ 1 file changed, 80 insertions(+), 36 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index ec005e59729..98e923249f3 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -3,7 +3,9 @@ #include #include #include +#include #include +#include #include #include @@ -15,6 +17,68 @@ namespace DB::ErrorCodes namespace DB::QueryPlanOptimizations { +static size_t tryAddNewFilterStep( + QueryPlan::Node * parent_node, + QueryPlan::Nodes & nodes, + const Names & allowed_inputs) +{ + QueryPlan::Node * child_node = parent_node->children.front(); + + auto & parent = parent_node->step; + auto & child = child_node->step; + + auto * filter = static_cast(parent.get()); + const auto & expression = filter->getExpression(); + const auto & filter_column_name = filter->getFilterColumnName(); + bool removes_filter = filter->removesFilterColumn(); + + // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; + + auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, allowed_inputs); + if (!split_filter) + return 0; + + // std::cerr << "===============\n" << expression->dumpDAG() << std::endl; + // std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; + + const auto & index = expression->getIndex(); + auto it = index.begin(); + for (; it != index.end(); ++it) + if ((*it)->result_name == filter_column_name) + break; + + if (it == expression->getIndex().end()) + { + if (!removes_filter) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", + filter_column_name, expression->dumpDAG()); + + std::cerr << "replacing to expr because filter " << filter_column_name << " was removed\n"; + parent = std::make_unique(child->getOutputStream(), expression); + } + else if ((*it)->column && isColumnConst(*(*it)->column)) + { + std::cerr << "replacing to expr because filter is const\n"; + parent = std::make_unique(child->getOutputStream(), expression); + } + + /// Add new Filter step before Aggregating. + /// Expression/Filter -> Aggregating -> Something + auto & node = nodes.emplace_back(); + node.children.swap(child_node->children); + child_node->children.emplace_back(&node); + /// Expression/Filter -> Aggregating -> Filter -> Something + + /// New filter column is added to the end. + auto split_filter_column_name = (*split_filter->getIndex().rbegin())->result_name; + node.step = std::make_unique( + node.children.at(0)->step->getOutputStream(), + std::move(split_filter), std::move(split_filter_column_name), true); + + return 3; +} + size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) { if (parent_node->children.size() != 1) @@ -29,10 +93,6 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (!filter) return 0; - const auto & expression = filter->getExpression(); - const auto & filter_column_name = filter->getFilterColumnName(); - bool removes_filter = filter->removesFilterColumn(); - if (auto * aggregating = typeid_cast(child.get())) { const auto & params = aggregating->getParams(); @@ -42,42 +102,26 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes for (auto pos : params.keys) keys.push_back(params.src_header.getByPosition(pos).name); - // std::cerr << "Filter: \n" << expression->dumpDAG() << std::endl; - if (auto split_filter = expression->splitActionsForFilter(filter_column_name, removes_filter, keys)) - { - // std::cerr << "===============\n" << expression->dumpDAG() << std::endl; - // std::cerr << "---------------\n" << split_filter->dumpDAG() << std::endl; + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, keys)) + return updated_steps; + } - auto it = expression->getIndex().find(filter_column_name); - if (it == expression->getIndex().end()) - { - if (!removes_filter) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", - filter_column_name, expression->dumpDAG()); + if (auto * array_join = typeid_cast(child.get())) + { + const auto & array_join_actions = array_join->arrayJoin(); + const auto & keys = array_join_actions->columns; + const auto & array_join_header = array_join->getInputStreams().front().header; - parent = std::make_unique(child->getOutputStream(), expression); - } - else if ((*it)->column && isColumnConst(*(*it)->column)) - { - parent = std::make_unique(child->getOutputStream(), expression); - } + Names allowed_inputs; + for (const auto & column : array_join_header) + if (keys.count(column.name) == 0) + allowed_inputs.push_back(column.name); - /// Add new Filter step before Aggregating. - /// Expression/Filter -> Aggregating -> Something - auto & node = nodes.emplace_back(); - node.children.swap(child_node->children); - child_node->children.emplace_back(&node); - /// Expression/Filter -> Aggregating -> Filter -> Something + for (const auto & name : allowed_inputs) + std::cerr << name << std::endl; - /// New filter column is added to the end. - auto split_filter_column_name = (*split_filter->getIndex().rbegin())->result_name; - node.step = std::make_unique( - node.children.at(0)->step->getOutputStream(), - std::move(split_filter), std::move(split_filter_column_name), true); - - return 3; - } + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) + return updated_steps; } return 0; From 5fd80555aa6241e01737c9a9083f663a8d7ed0eb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 12 Feb 2021 19:06:18 +0300 Subject: [PATCH 127/510] Update test. --- .../queries/0_stateless/01655_plan_optimizations.reference | 4 ++++ tests/queries/0_stateless/01655_plan_optimizations.sh | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 510224146ed..1e638829c74 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -38,3 +38,7 @@ Filter column ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(minus(y, 4), notEquals(y, 0)) +> filter is split, one part is filtered before ARRAY JOIN +Filter column: and(notEquals(y, 2), notEquals(x, 0)) +ARRAY JOIN x +Filter column: notEquals(y, 2) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index ea76d15c648..ccd331df45e 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -55,3 +55,10 @@ $CLICKHOUSE_CLIENT -q " ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: and(minus(y, 4), notEquals(y, 0))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" + +echo "> filter is split, one part is filtered before ARRAY JOIN" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select x, y from ( + select range(number) as x, number + 1 as y from numbers(3) + ) array join x where y != 2 and x != 0" | + grep -o "Filter column: and(notEquals(y, 2), notEquals(x, 0))\|ARRAY JOIN x\|Filter column: notEquals(y, 2)" \ No newline at end of file From 10d773d67154d67c2fa975f5c8d46c8f9ccfb5a6 Mon Sep 17 00:00:00 2001 From: lehasm Date: Sat, 13 Feb 2021 22:35:53 +0300 Subject: [PATCH 128/510] HTTP compression info updated xz compression method added. Text rearranged and edited. Examples improved. --- docs/en/interfaces/http.md | 38 +++++++++++++++++++++++++++----------- 1 file changed, 27 insertions(+), 11 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 310286e3d44..84c1e268e07 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -148,25 +148,41 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- For successful requests that don’t return a data table, an empty response body is returned. -You can use the internal ClickHouse compression format when transmitting data. The compressed data has a non-standard format, and you will need to use the special `clickhouse-compressor` program to work with it (it is installed with the `clickhouse-client` package). To increase the efficiency of data insertion, you can disable server-side checksum verification by using the [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) setting. -If you specified `compress=1` in the URL, the server compresses the data it sends you. -If you specified `decompress=1` in the URL, the server decompresses the same data that you pass in the `POST` method. +## Compression {#compression} -You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. In order for ClickHouse to compress the response, you must append `Accept-Encoding: compression_method`. ClickHouse supports `gzip`, `br`, and `deflate` [compression methods](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens). To enable HTTP compression, you must use the ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting. You can configure the data compression level in the [http_zlib_compression_level](#settings-http_zlib_compression_level) setting for all the compression methods. +You can use compression to reduce network traffic when transmitting a large amount of data or for creating dumps that are immediately compressed. -You can use this to reduce network traffic when transmitting a large amount of data, or for creating dumps that are immediately compressed. +You can use the internal ClickHouse compression format when transmitting data. The compressed data has a non-standard format, and you need `clickhouse-compressor` program to work with it. It is installed with the `clickhouse-client` package. To increase the efficiency of data insertion, you can disable server-side checksum verification by using the [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) setting. -Examples of sending data with compression: +If you specify `compress=1` in the URL, the server will compress the data it sends to you. +If you specify `decompress=1` in the URL, the server will decompress the data which you pass in the `POST` method. +You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). ClickHouse supports the following [compression methods](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens): + +- `gzip` +- `br` +- `deflate` +- `xz` + +To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. Example: ``` bash -#Sending data to the server: -$ curl -vsS "http://localhost:8123/?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10' -H 'Accept-Encoding: gzip' - -#Sending data to the client: -$ echo "SELECT 1" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' +$ echo "SELECT 1" | gzip -c | \ + curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' ``` +In order for ClickHouse to compress the response, enable compression with [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting and append `Accept-Encoding: compression_method` header to the request. You can configure the data compression level in the [http_zlib_compression_level](../operations/settings/settings.md#settings-http_zlib_compression_level) setting for all compression methods. +``` bash +$ curl -vsS "http://localhost:8123/?enable_http_compression=1" \ + -H 'Accept-Encoding: gzip' --output result.gz -d 'SELECT number FROM system.numbers LIMIT 3' +$ zcat result.gz +0 +1 +2 +``` + +## Default Database {#default-database} + !!! note "Note" Some HTTP clients might decompress data from the server by default (with `gzip` and `deflate`) and you might get decompressed data even if you use the compression settings correctly. From d4ba07c5c6737f2c978331969d6b7c4ce535613c Mon Sep 17 00:00:00 2001 From: lehasm Date: Sat, 13 Feb 2021 23:26:56 +0300 Subject: [PATCH 129/510] Fix missplaced header --- docs/en/interfaces/http.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 84c1e268e07..d82d8baeb75 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -181,11 +181,12 @@ $ zcat result.gz 2 ``` -## Default Database {#default-database} - !!! note "Note" Some HTTP clients might decompress data from the server by default (with `gzip` and `deflate`) and you might get decompressed data even if you use the compression settings correctly. + +## Default Database {#default-database} + You can use the ‘database’ URL parameter or the ‘X-ClickHouse-Database’ header to specify the default database. ``` bash From 79592b73f840179faa1efc4ea447f6c2107921fe Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 14 Feb 2021 02:07:13 +0400 Subject: [PATCH 130/510] Store filter info in prewhere info instead of multiple prewheres Some cleanups --- src/Interpreters/ExpressionAnalyzer.cpp | 6 +- src/Interpreters/ExpressionAnalyzer.h | 4 +- src/Interpreters/InterpreterSelectQuery.cpp | 140 +++++++++--------- src/Interpreters/InterpreterSelectQuery.h | 4 +- .../getHeaderForProcessingStage.cpp | 22 ++- src/Storages/IStorage.cpp | 8 +- .../MergeTreeBaseSelectProcessor.cpp | 63 +++++--- .../MergeTree/MergeTreeBaseSelectProcessor.h | 8 +- .../MergeTree/MergeTreeBlockReadUtils.cpp | 25 +++- .../MergeTree/MergeTreeBlockReadUtils.h | 7 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 30 ++-- .../MergeTree/MergeTreeRangeReader.cpp | 76 ++++++---- src/Storages/MergeTree/MergeTreeRangeReader.h | 7 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 8 +- src/Storages/MergeTree/MergeTreeReadPool.h | 9 +- .../MergeTreeReverseSelectProcessor.cpp | 11 +- .../MergeTreeReverseSelectProcessor.h | 2 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 11 +- .../MergeTree/MergeTreeSelectProcessor.h | 2 +- ...rgeTreeThreadSelectBlockInputProcessor.cpp | 8 +- ...MergeTreeThreadSelectBlockInputProcessor.h | 2 +- src/Storages/SelectQueryInfo.h | 60 +++++--- src/Storages/StorageBuffer.cpp | 39 +++-- 23 files changed, 320 insertions(+), 232 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 5fdd75d19eb..3b9e317934b 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1320,7 +1320,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( bool first_stage_, bool second_stage_, bool only_types, - const FilterInfoPtr & filter_info_, + const FilterDAGInfoPtr & filter_info_, const Block & source_header) : first_stage(first_stage_) , second_stage(second_stage_) @@ -1383,7 +1383,7 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && filter_info_) { filter_info = filter_info_; - query_analyzer.appendPreliminaryFilter(chain, filter_info->actions_dag, filter_info->column_name); + query_analyzer.appendPreliminaryFilter(chain, filter_info->actions, filter_info->column_name); } if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) @@ -1583,7 +1583,7 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si void ExpressionAnalysisResult::removeExtraColumns() const { if (hasFilter()) - filter_info->actions_dag->projectInput(); + filter_info->actions->projectInput(); if (hasWhere()) before_where->projectInput(); if (hasHaving()) diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 71301ad64a2..3ba9da534bc 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -215,7 +215,7 @@ struct ExpressionAnalysisResult NameSet columns_to_remove_after_prewhere; PrewhereDAGInfoPtr prewhere_info; - FilterInfoPtr filter_info; + FilterDAGInfoPtr filter_info; ConstantFilterDescription prewhere_constant_filter_description; ConstantFilterDescription where_constant_filter_description; /// Actions by every element of ORDER BY @@ -230,7 +230,7 @@ struct ExpressionAnalysisResult bool first_stage, bool second_stage, bool only_types, - const FilterInfoPtr & filter_info, + const FilterDAGInfoPtr & filter_info, const Block & source_header); /// Filter for row-level security. diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 6a0e2515801..3d91b271b82 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -447,10 +447,10 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// Fix source_header for filter actions. if (row_policy_filter) { - filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions_dag, row_policy_filter, required_columns); + filter_info = std::make_shared(); + filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns); source_header = metadata_snapshot->getSampleBlockForColumns( - filter_info->actions_dag->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); + filter_info->actions->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); } } @@ -807,9 +807,12 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu bool intermediate_stage = false; bool to_aggregation_stage = false; bool from_aggregation_stage = false; - const bool filter_in_prewhere = ( - (settings.optimize_move_to_prewhere || expressions.prewhere_info) && - !input && !input_pipe && storage && storage->supportsPrewhere() + const bool execute_row_level_filter_in_prewhere = ( + ( + settings.optimize_move_to_prewhere || // ...when it is allowed to move things to prewhere, so we do it for row-level filter actions too. + expressions.prewhere_info // ...or when we already have prewhere and must execute row-level filter before it. + ) && + !input && !input_pipe && storage && storage->supportsPrewhere() // Check that prewhere can be used at all. ); if (options.only_analyze) @@ -817,11 +820,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu auto read_nothing = std::make_unique(source_header); query_plan.addStep(std::move(read_nothing)); - if (expressions.filter_info && filter_in_prewhere) + if (expressions.filter_info && execute_row_level_filter_in_prewhere) { auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), - expressions.filter_info->actions_dag, + expressions.filter_info->actions, expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -880,7 +883,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu to_aggregation_stage = true; /// Read the data from Storage. from_stage - to what stage the request was completed in Storage. - executeFetchColumns(from_stage, query_plan, filter_in_prewhere); + executeFetchColumns(from_stage, query_plan, execute_row_level_filter_in_prewhere); LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } @@ -945,11 +948,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu if (expressions.first_stage) { - if (expressions.filter_info && !filter_in_prewhere) + if (expressions.filter_info && !execute_row_level_filter_in_prewhere) { auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), - expressions.filter_info->actions_dag, + expressions.filter_info->actions, expressions.filter_info->column_name, expressions.filter_info->do_remove_column); @@ -1200,40 +1203,55 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { Pipe pipe(std::make_shared(source_header)); - if (query_info.prewhere_info_list) + if (query_info.prewhere_info) { - for (const auto & prewhere_info : *query_info.prewhere_info_list) + auto & prewhere_info = *query_info.prewhere_info; + + if (prewhere_info.filter_info) { - if (prewhere_info.alias_actions) - { - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, prewhere_info.alias_actions); - }); - } + auto & filter_info = *prewhere_info.filter_info; pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( header, - prewhere_info.prewhere_actions, - prewhere_info.prewhere_column_name, - prewhere_info.remove_prewhere_column); + filter_info.actions, + filter_info.column_name, + filter_info.do_remove_column); }); + } - // To remove additional columns - // In some cases, we did not read any marks so that the pipeline.streams is empty - // Thus, some columns in prewhere are not removed as expected - // This leads to mismatched header in distributed table - if (prewhere_info.remove_columns_actions) + if (prewhere_info.alias_actions) + { + pipe.addSimpleTransform([&](const Block & header) { - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, prewhere_info.remove_columns_actions); - }); - } + return std::make_shared( + header, + prewhere_info.alias_actions); + }); + } + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + prewhere_info.prewhere_actions, + prewhere_info.prewhere_column_name, + prewhere_info.remove_prewhere_column); + }); + + // To remove additional columns + // In some cases, we did not read any marks so that the pipeline.streams is empty + // Thus, some columns in prewhere are not removed as expected + // This leads to mismatched header in distributed table + if (prewhere_info.remove_columns_actions) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + prewhere_info.remove_columns_actions); + }); } } @@ -1242,7 +1260,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c query_plan.addStep(std::move(read_from_pipe)); } -void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool filter_in_prewhere) +void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool execute_row_level_filter_in_prewhere) { auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -1569,47 +1587,33 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc query_info.syntax_analyzer_result = syntax_analyzer_result; query_info.sets = query_analyzer->getPreparedSets(); - if (expressions.filter_info && filter_in_prewhere) - { - if (!query_info.prewhere_info_list) - query_info.prewhere_info_list = std::make_shared(); - - query_info.prewhere_info_list->emplace( - query_info.prewhere_info_list->begin(), - std::make_shared(expressions.filter_info->actions_dag), - expressions.filter_info->column_name); - - if (alias_actions) - { - query_info.prewhere_info_list->back().alias_actions = std::make_shared(alias_actions); - alias_actions = nullptr; - } - - auto & new_filter_info = query_info.prewhere_info_list->front(); - - new_filter_info.remove_prewhere_column = expressions.filter_info->do_remove_column; - new_filter_info.need_filter = true; - } - if (prewhere_info) { - if (!query_info.prewhere_info_list) - query_info.prewhere_info_list = std::make_shared(); + query_info.prewhere_info = std::make_shared(); - query_info.prewhere_info_list->emplace_back( - std::make_shared(prewhere_info->prewhere_actions), - prewhere_info->prewhere_column_name); + if (expressions.filter_info && execute_row_level_filter_in_prewhere) + { + query_info.prewhere_info->filter_info = std::make_shared(); - auto & new_prewhere_info = query_info.prewhere_info_list->back(); + if (expressions.filter_info->actions) + query_info.prewhere_info->filter_info->actions = std::make_shared(expressions.filter_info->actions); + + query_info.prewhere_info->filter_info->column_name = expressions.filter_info->column_name; + query_info.prewhere_info->filter_info->do_remove_column = expressions.filter_info->do_remove_column; + } if (prewhere_info->alias_actions) - new_prewhere_info.alias_actions = std::make_shared(prewhere_info->alias_actions); + query_info.prewhere_info->alias_actions = std::make_shared(prewhere_info->alias_actions); + + if (prewhere_info->prewhere_actions) + query_info.prewhere_info->prewhere_actions = std::make_shared(prewhere_info->prewhere_actions); if (prewhere_info->remove_columns_actions) - new_prewhere_info.remove_columns_actions = std::make_shared(prewhere_info->remove_columns_actions); + query_info.prewhere_info->remove_columns_actions = std::make_shared(prewhere_info->remove_columns_actions); - new_prewhere_info.remove_prewhere_column = prewhere_info->remove_prewhere_column; - new_prewhere_info.need_filter = prewhere_info->need_filter; + query_info.prewhere_info->prewhere_column_name = prewhere_info->prewhere_column_name; + query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column; + query_info.prewhere_info->need_filter = prewhere_info->need_filter; } /// Create optimizer with prepared actions. diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 6fcbf102b05..793df612103 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -108,7 +108,7 @@ private: /// Different stages of query execution. - void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool filter_in_prewhere); + void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool execute_row_level_filter_in_prewhere); void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); void executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final); @@ -157,7 +157,7 @@ private: /// Is calculated in getSampleBlock. Is used later in readImpl. ExpressionAnalysisResult analysis_result; /// For row-level security. - FilterInfoPtr filter_info; + FilterDAGInfoPtr filter_info; QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 761f04e81ee..2aef3c25c3c 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -42,14 +42,26 @@ Block getHeaderForProcessingStage( case QueryProcessingStage::FetchColumns: { Block header = metadata_snapshot->getSampleBlockForColumns(column_names, storage.getVirtuals(), storage.getStorageID()); - if (query_info.prewhere_info_list) + if (query_info.prewhere_info) { - for (const auto & prewhere_info : *query_info.prewhere_info_list) + auto & prewhere_info = *query_info.prewhere_info; + + if (prewhere_info.filter_info) { - prewhere_info.prewhere_actions->execute(header); - if (prewhere_info.remove_prewhere_column) - header.erase(prewhere_info.prewhere_column_name); + auto & filter_info = *prewhere_info.filter_info; + + if (filter_info.actions) + filter_info.actions->execute(header); + + if (filter_info.do_remove_column) + header.erase(filter_info.column_name); } + + if (prewhere_info.prewhere_actions) + prewhere_info.prewhere_actions->execute(header); + + if (prewhere_info.remove_prewhere_column) + header.erase(prewhere_info.prewhere_column_name); } return header; } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 5f500518516..b83c33be2aa 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -203,14 +203,14 @@ std::string PrewhereDAGInfo::dump() const return ss.str(); } -std::string FilterInfo::dump() const +std::string FilterDAGInfo::dump() const { WriteBufferFromOwnString ss; - ss << "FilterInfo for column '" << column_name <<"', do_remove_column " + ss << "FilterDAGInfo for column '" << column_name <<"', do_remove_column " << do_remove_column << "\n"; - if (actions_dag) + if (actions) { - ss << "actions_dag " << actions_dag->dumpDAG() << "\n"; + ss << "actions " << actions->dumpDAG() << "\n"; } return ss.str(); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 3405a211c98..54b343519fa 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -22,17 +22,17 @@ MergeTreeBaseSelectProcessor::MergeTreeBaseSelectProcessor( Block header, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, UInt64 preferred_max_column_in_block_size_bytes_, const MergeTreeReaderSettings & reader_settings_, bool use_uncompressed_cache_, const Names & virt_column_names_) - : SourceWithProgress(getHeader(std::move(header), prewhere_info_list_, virt_column_names_)) + : SourceWithProgress(getHeader(std::move(header), prewhere_info_, virt_column_names_)) , storage(storage_) , metadata_snapshot(metadata_snapshot_) - , prewhere_info_list(prewhere_info_list_) + , prewhere_info(prewhere_info_) , max_block_size_rows(max_block_size_rows_) , preferred_block_size_bytes(preferred_block_size_bytes_) , preferred_max_column_in_block_size_bytes(preferred_max_column_in_block_size_bytes_) @@ -70,18 +70,18 @@ Chunk MergeTreeBaseSelectProcessor::generate() void MergeTreeBaseSelectProcessor::initializeRangeReaders(MergeTreeReadTask & current_task) { - if (prewhere_info_list) + if (prewhere_info) { if (reader->getColumns().empty()) { - current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info_list, true); + current_task.range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, true); } else { MergeTreeRangeReader * pre_reader_ptr = nullptr; if (pre_reader != nullptr) { - current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info_list, false); + current_task.pre_range_reader = MergeTreeRangeReader(pre_reader.get(), nullptr, prewhere_info, false); pre_reader_ptr = ¤t_task.pre_range_reader; } @@ -309,37 +309,60 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns(Chunk & chunk, MergeTree chunk.setColumns(columns, num_rows); } -void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const PrewhereInfoListPtr & prewhere_info_list) +void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info) { - if (!prewhere_info_list) - return; - - for (const auto & prewhere_info : *prewhere_info_list) + if (prewhere_info) { - if (prewhere_info.alias_actions) - prewhere_info.alias_actions->execute(block); + if (prewhere_info->filter_info) + { + auto & filter_info = *prewhere_info->filter_info; - prewhere_info.prewhere_actions->execute(block); - auto & prewhere_column = block.getByName(prewhere_info.prewhere_column_name); + if (filter_info.actions) + filter_info.actions->execute(block); + auto & filter_column = block.getByName(filter_info.column_name); + if (!filter_column.type->canBeUsedInBooleanContext()) + { + throw Exception("Invalid type for row-level security filter: " + filter_column.type->getName(), + ErrorCodes::LOGICAL_ERROR); + } + + if (filter_info.do_remove_column) + block.erase(filter_info.column_name); + else + { + auto & ctn = block.getByName(filter_info.column_name); + ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); + } + } + + if (prewhere_info->alias_actions) + prewhere_info->alias_actions->execute(block); + + if (prewhere_info->prewhere_actions) + prewhere_info->prewhere_actions->execute(block); + + auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) + { throw Exception("Invalid type for filter in PREWHERE: " + prewhere_column.type->getName(), ErrorCodes::LOGICAL_ERROR); + } - if (prewhere_info.remove_prewhere_column) - block.erase(prewhere_info.prewhere_column_name); + if (prewhere_info->remove_prewhere_column) + block.erase(prewhere_info->prewhere_column_name); else { - auto & ctn = block.getByName(prewhere_info.prewhere_column_name); + auto & ctn = block.getByName(prewhere_info->prewhere_column_name); ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); } } } Block MergeTreeBaseSelectProcessor::getHeader( - Block block, const PrewhereInfoListPtr & prewhere_info_list, const Names & virtual_columns) + Block block, const PrewhereInfoPtr & prewhere_info, const Names & virtual_columns) { - executePrewhereActions(block, prewhere_info_list); + executePrewhereActions(block, prewhere_info); injectVirtualColumns(block, nullptr, virtual_columns); return block; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index a3d7520b89a..00ef131ae45 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -23,7 +23,7 @@ public: Block header, const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, UInt64 max_block_size_rows_, UInt64 preferred_block_size_bytes_, UInt64 preferred_max_column_in_block_size_bytes_, @@ -33,7 +33,7 @@ public: ~MergeTreeBaseSelectProcessor() override; - static void executePrewhereActions(Block & block, const PrewhereInfoListPtr & prewhere_info_list); + static void executePrewhereActions(Block & block, const PrewhereInfoPtr & prewhere_info); protected: Chunk generate() final; @@ -49,7 +49,7 @@ protected: static void injectVirtualColumns(Block & block, MergeTreeReadTask * task, const Names & virtual_columns); static void injectVirtualColumns(Chunk & chunk, MergeTreeReadTask * task, const Names & virtual_columns); - static Block getHeader(Block block, const PrewhereInfoListPtr & prewhere_info_list, const Names & virtual_columns); + static Block getHeader(Block block, const PrewhereInfoPtr & prewhere_info, const Names & virtual_columns); void initializeRangeReaders(MergeTreeReadTask & task); @@ -57,7 +57,7 @@ protected: const MergeTreeData & storage; StorageMetadataPtr metadata_snapshot; - PrewhereInfoListPtr prewhere_info_list; + PrewhereInfoPtr prewhere_info; UInt64 max_block_size_rows; UInt64 preferred_block_size_bytes; diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f3191a76120..d5fb2f3300c 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -118,10 +118,11 @@ NameSet injectRequiredColumns(const MergeTreeData & storage, const StorageMetada MergeTreeReadTask::MergeTreeReadTask( const MergeTreeData::DataPartPtr & data_part_, const MarkRanges & mark_ranges_, const size_t part_index_in_query_, const Names & ordered_names_, const NameSet & column_name_set_, const NamesAndTypesList & columns_, - const NamesAndTypesList & pre_columns_, const bool should_reorder_, MergeTreeBlockSizePredictorPtr && size_predictor_) + const NamesAndTypesList & pre_columns_, const bool remove_prewhere_column_, const bool should_reorder_, + MergeTreeBlockSizePredictorPtr && size_predictor_) : data_part{data_part_}, mark_ranges{mark_ranges_}, part_index_in_query{part_index_in_query_}, ordered_names{ordered_names_}, column_name_set{column_name_set_}, columns{columns_}, pre_columns{pre_columns_}, - should_reorder{should_reorder_}, size_predictor{std::move(size_predictor_)} + remove_prewhere_column{remove_prewhere_column_}, should_reorder{should_reorder_}, size_predictor{std::move(size_predictor_)} { } @@ -257,7 +258,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, - const PrewhereInfoListPtr & prewhere_info_list, + const PrewhereInfoPtr & prewhere_info, bool check_columns) { Names column_names = required_columns; @@ -266,12 +267,22 @@ MergeTreeReadTaskColumns getReadTaskColumns( /// inject columns required for defaults evaluation bool should_reorder = !injectRequiredColumns(storage, metadata_snapshot, data_part, column_names).empty(); - if (prewhere_info_list) + if (prewhere_info) { - for (const auto & prewhere_info : *prewhere_info_list) + if (prewhere_info->filter_info && prewhere_info->filter_info->actions) { - const auto required_column_names = (prewhere_info.alias_actions ? - prewhere_info.alias_actions->getRequiredColumns() : prewhere_info.prewhere_actions->getRequiredColumns()); + const auto required_column_names = prewhere_info->filter_info->actions->getRequiredColumns(); + pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); + } + + if (prewhere_info->alias_actions) + { + const auto required_column_names = prewhere_info->alias_actions->getRequiredColumns(); + pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); + } + else + { + const auto required_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index f2537c554c3..31d609e4242 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -42,6 +42,8 @@ struct MergeTreeReadTask const NamesAndTypesList & columns; /// column names to read during PREWHERE const NamesAndTypesList & pre_columns; + /// should PREWHERE column be returned to requesting side? + const bool remove_prewhere_column; /// resulting block may require reordering in accordance with `ordered_names` const bool should_reorder; /// Used to satistfy preferred_block_size_bytes limitation @@ -55,7 +57,8 @@ struct MergeTreeReadTask MergeTreeReadTask( const MergeTreeData::DataPartPtr & data_part_, const MarkRanges & mark_ranges_, const size_t part_index_in_query_, const Names & ordered_names_, const NameSet & column_name_set_, const NamesAndTypesList & columns_, - const NamesAndTypesList & pre_columns_, const bool should_reorder_, MergeTreeBlockSizePredictorPtr && size_predictor_); + const NamesAndTypesList & pre_columns_, const bool remove_prewhere_column_, const bool should_reorder_, + MergeTreeBlockSizePredictorPtr && size_predictor_); virtual ~MergeTreeReadTask(); }; @@ -75,7 +78,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const StorageMetadataPtr & metadata_snapshot, const MergeTreeData::DataPartPtr & data_part, const Names & required_columns, - const PrewhereInfoListPtr & prewhere_info_list, + const PrewhereInfoPtr & prewhere_info, bool check_columns); struct MergeTreeBlockSizePredictor diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 9d331f19a56..b44e7197c12 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -834,20 +834,14 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( plan->addStep(std::move(adding_column)); } - if (query_info.prewhere_info_list) + if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions) { - for (const auto & prewhere_info : *query_info.prewhere_info_list) - { - if (prewhere_info.remove_columns_actions) - { - auto expression_step = std::make_unique( - plan->getCurrentDataStream(), - prewhere_info.remove_columns_actions->getActionsDAG().clone()); + auto expression_step = std::make_unique( + plan->getCurrentDataStream(), + query_info.prewhere_info->remove_columns_actions->getActionsDAG().clone()); - expression_step->setStepDescription("Remove unused columns after PREWHERE"); - plan->addStep(std::move(expression_step)); - } - } + expression_step->setStepDescription("Remove unused columns after PREWHERE"); + plan->addStep(std::move(expression_step)); } return plan; @@ -983,7 +977,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( std::move(parts), data, metadata_snapshot, - query_info.prewhere_info_list, + query_info.prewhere_info, true, column_names, MergeTreeReadPool::BackoffSettings(settings), @@ -999,7 +993,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, data, metadata_snapshot, use_uncompressed_cache, - query_info.prewhere_info_list, reader_settings, virt_columns); + query_info.prewhere_info, reader_settings, virt_columns); if (i == 0) { @@ -1022,7 +1016,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( auto source = std::make_shared( data, metadata_snapshot, part.data_part, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, column_names, part.ranges, use_uncompressed_cache, - query_info.prewhere_info_list, true, reader_settings, virt_columns, part.part_index_in_query); + query_info.prewhere_info, true, reader_settings, virt_columns, part.part_index_in_query); res.emplace_back(std::move(source)); } @@ -1223,7 +1217,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( column_names, ranges_to_get_from_part, use_uncompressed_cache, - query_info.prewhere_info_list, + query_info.prewhere_info, true, reader_settings, virt_columns, @@ -1241,7 +1235,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( column_names, ranges_to_get_from_part, use_uncompressed_cache, - query_info.prewhere_info_list, + query_info.prewhere_info, true, reader_settings, virt_columns, @@ -1395,7 +1389,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( column_names, part_it->ranges, use_uncompressed_cache, - query_info.prewhere_info_list, + query_info.prewhere_info, true, reader_settings, virt_columns, diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 7c9b1b36b33..8d149d9473f 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -543,12 +543,12 @@ size_t MergeTreeRangeReader::ReadResult::countBytesInResultFilter(const IColumn: MergeTreeRangeReader::MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, bool last_reader_in_chain_) : merge_tree_reader(merge_tree_reader_) , index_granularity(&(merge_tree_reader->data_part->index_granularity)) , prev_reader(prev_reader_) - , prewhere_info_list(prewhere_info_list_) + , prewhere_info(prewhere_info_) , last_reader_in_chain(last_reader_in_chain_) , is_initialized(true) { @@ -558,19 +558,25 @@ MergeTreeRangeReader::MergeTreeRangeReader( for (const auto & name_and_type : merge_tree_reader->getColumns()) sample_block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name}); - if (prewhere_info_list) + if (prewhere_info) { - for (const auto & prewhere_info : *prewhere_info_list) + if (prewhere_info->filter_info) { - if (prewhere_info.alias_actions) - prewhere_info.alias_actions->execute(sample_block, true); + if (prewhere_info->filter_info->actions) + prewhere_info->filter_info->actions->execute(sample_block, true); - if (prewhere_info.prewhere_actions) - prewhere_info.prewhere_actions->execute(sample_block, true); - - if (prewhere_info.remove_prewhere_column) - sample_block.erase(prewhere_info.prewhere_column_name); + if (prewhere_info->filter_info->do_remove_column) + sample_block.erase(prewhere_info->filter_info->column_name); } + + if (prewhere_info->alias_actions) + prewhere_info->alias_actions->execute(sample_block, true); + + if (prewhere_info->prewhere_actions) + prewhere_info->prewhere_actions->execute(sample_block, true); + + if (prewhere_info->remove_prewhere_column) + sample_block.erase(prewhere_info->prewhere_column_name); } } @@ -860,7 +866,7 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - if (!prewhere_info_list || prewhere_info_list->empty()) + if (!prewhere_info) return; const auto & header = merge_tree_reader->getColumns(); @@ -890,29 +896,37 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type) block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - for (size_t i = 0; i < prewhere_info_list->size(); ++i) + if (prewhere_info->filter_info) { - const auto & prewhere_info = (*prewhere_info_list)[i]; + if (prewhere_info->filter_info->actions) + prewhere_info->filter_info->actions->execute(block); - if (prewhere_info.alias_actions) - prewhere_info.alias_actions->execute(block); + const auto filter_column_pos = block.getPositionByName(prewhere_info->filter_info->column_name); + result.addFilter(block.getByPosition(filter_column_pos).column); - /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. - result.block_before_prewhere = block; - prewhere_info.prewhere_actions->execute(block); - - prewhere_column_pos = block.getPositionByName(prewhere_info.prewhere_column_name); - result.addFilter(block.getByPosition(prewhere_column_pos).column); - - if (i + 1 != prewhere_info_list->size() && prewhere_info.remove_prewhere_column) - block.erase(prewhere_column_pos); + if (prewhere_info->filter_info->do_remove_column) + block.erase(prewhere_info->filter_info->column_name); else - block.getByPosition(prewhere_column_pos).column = block.getByPosition(prewhere_column_pos).type->createColumnConst(result.num_rows, 1); + block.getByPosition(filter_column_pos).column = block.getByPosition(filter_column_pos).type->createColumnConst(result.num_rows, 1); } - block.getByPosition(prewhere_column_pos).column = nullptr; + if (prewhere_info->alias_actions) + prewhere_info->alias_actions->execute(block); + + /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. + result.block_before_prewhere = block; + + if (prewhere_info->prewhere_actions) + prewhere_info->prewhere_actions->execute(block); + + prewhere_column_pos = block.getPositionByName(prewhere_info->prewhere_column_name); + result.addFilter(block.getByPosition(prewhere_column_pos).column); + + block.getByPosition(prewhere_column_pos).column.reset(); + result.columns.clear(); result.columns.reserve(block.columns()); + for (auto & col : block) result.columns.emplace_back(std::move(col.column)); } @@ -925,7 +939,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.totalRowsPerGranule() == 0) result.setFilterConstFalse(); /// If we need to filter in PREWHERE - else if (prewhere_info_list->back().need_filter || result.need_filter) + else if (prewhere_info->need_filter || result.need_filter) { /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) @@ -966,11 +980,11 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Check if the PREWHERE column is needed if (!result.columns.empty()) { - if (prewhere_info_list->back().remove_prewhere_column) + if (prewhere_info->remove_prewhere_column) result.columns.erase(result.columns.begin() + prewhere_column_pos); else result.columns[prewhere_column_pos] = - getSampleBlock().getByName(prewhere_info_list->back().prewhere_column_name).type-> + getSampleBlock().getByName(prewhere_info->prewhere_column_name).type-> createColumnConst(result.num_rows, 1u)->convertToFullColumnIfConst(); } } @@ -978,7 +992,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r else { result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); - if (getSampleBlock().getByName(prewhere_info_list->back().prewhere_column_name).type->isNullable()) + if (getSampleBlock().getByName(prewhere_info->prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); result.clearFilter(); // Acting as a flag to not filter in PREWHERE } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 6ee7c9f3e29..884d2dbafd1 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -13,8 +13,7 @@ using ColumnUInt8 = ColumnVector; class IMergeTreeReader; class MergeTreeIndexGranularity; struct PrewhereInfo; -using PrewhereInfoList = std::vector; -using PrewhereInfoListPtr = std::shared_ptr; +using PrewhereInfoPtr = std::shared_ptr; /// MergeTreeReader iterator which allows sequential reading for arbitrary number of rows between pairs of marks in the same part. /// Stores reading state, which can be inside granule. Can skip rows in current granule and start reading from next mark. @@ -25,7 +24,7 @@ public: MergeTreeRangeReader( IMergeTreeReader * merge_tree_reader_, MergeTreeRangeReader * prev_reader_, - const PrewhereInfoListPtr & prewhere_info_list, + const PrewhereInfoPtr & prewhere_info_, bool last_reader_in_chain_); MergeTreeRangeReader() = default; @@ -218,7 +217,7 @@ private: IMergeTreeReader * merge_tree_reader = nullptr; const MergeTreeIndexGranularity * index_granularity = nullptr; MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly. - PrewhereInfoListPtr prewhere_info_list; + PrewhereInfoPtr prewhere_info; Stream stream; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index a3a580fa7f2..d9a250e3f7a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -24,7 +24,7 @@ MergeTreeReadPool::MergeTreeReadPool( RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, const bool check_columns_, const Names & column_names_, const BackoffSettings & backoff_settings_, @@ -37,7 +37,7 @@ MergeTreeReadPool::MergeTreeReadPool( , column_names{column_names_} , do_not_steal_tasks{do_not_steal_tasks_} , predict_block_size_bytes{preferred_block_size_bytes_ > 0} - , prewhere_info_list{prewhere_info_list_} + , prewhere_info{prewhere_info_} , parts_ranges{std::move(parts_)} { /// parts don't contain duplicate MergeTreeDataPart's. @@ -139,7 +139,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read, return std::make_unique( part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names, per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx], - per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); + prewhere_info && prewhere_info->remove_prewhere_column, per_part_should_reorder[part_idx], std::move(curr_task_size_predictor)); } MarkRanges MergeTreeReadPool::getRestMarks(const IMergeTreeDataPart & part, const MarkRange & from) const @@ -229,7 +229,7 @@ std::vector MergeTreeReadPool::fillPerPartInfo( per_part_sum_marks.push_back(sum_marks); auto [required_columns, required_pre_columns, should_reorder] = - getReadTaskColumns(data, metadata_snapshot, part.data_part, column_names, prewhere_info_list, check_columns); + getReadTaskColumns(data, metadata_snapshot, part.data_part, column_names, prewhere_info, check_columns); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & required_column_names = required_columns.getNames(); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index ec9523ccbe3..aa6811661e6 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -71,9 +71,10 @@ private: public: MergeTreeReadPool( const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_, - RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, - const PrewhereInfoListPtr & prewhere_info_list, const bool check_columns_, const Names & column_names_, - const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, const bool do_not_steal_tasks_ = false); + RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_, + const bool check_columns_, const Names & column_names_, + const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, + const bool do_not_steal_tasks_ = false); MergeTreeReadTaskPtr getTask(const size_t min_marks_to_read, const size_t thread, const Names & ordered_names); @@ -106,7 +107,7 @@ private: std::vector per_part_pre_columns; std::vector per_part_should_reorder; std::vector per_part_size_predictor; - PrewhereInfoListPtr prewhere_info_list; + PrewhereInfoPtr prewhere_info; struct Part { diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp index 35df1106339..ee0a77ba3cf 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.cpp @@ -22,7 +22,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( Names required_columns_, MarkRanges mark_ranges_, bool use_uncompressed_cache_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, bool check_columns, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_, @@ -31,7 +31,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( : MergeTreeBaseSelectProcessor{ metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), - storage_, metadata_snapshot_, prewhere_info_list_, max_block_size_rows_, + storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, @@ -56,7 +56,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( ordered_names = header_without_virtual_columns.getNames(); - task_columns = getReadTaskColumns(storage, metadata_snapshot, data_part, required_columns, prewhere_info_list, check_columns); + task_columns = getReadTaskColumns(storage, metadata_snapshot, data_part, required_columns, prewhere_info, check_columns); /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const auto & column_names = task_columns.columns.getNames(); @@ -71,7 +71,7 @@ MergeTreeReverseSelectProcessor::MergeTreeReverseSelectProcessor( all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); - if (prewhere_info_list) + if (prewhere_info) pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); } @@ -100,7 +100,8 @@ try task = std::make_unique( data_part, mark_ranges_for_task, part_index_in_query, ordered_names, column_name_set, - task_columns.columns, task_columns.pre_columns, task_columns.should_reorder, std::move(size_predictor)); + task_columns.columns, task_columns.pre_columns, prewhere_info && prewhere_info->remove_prewhere_column, + task_columns.should_reorder, std::move(size_predictor)); return true; } diff --git a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h index b6da7166457..c9fd06c5534 100644 --- a/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeReverseSelectProcessor.h @@ -26,7 +26,7 @@ public: Names required_columns_, MarkRanges mark_ranges, bool use_uncompressed_cache, - const PrewhereInfoListPtr & prewhere_info_list, + const PrewhereInfoPtr & prewhere_info, bool check_columns, const MergeTreeReaderSettings & reader_settings, const Names & virt_column_names = {}, diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index cdb97f47a47..65f9b1eba3b 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -22,7 +22,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( Names required_columns_, MarkRanges mark_ranges_, bool use_uncompressed_cache_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, bool check_columns_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_, @@ -31,7 +31,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( : MergeTreeBaseSelectProcessor{ metadata_snapshot_->getSampleBlockForColumns(required_columns_, storage_.getVirtuals(), storage_.getStorageID()), - storage_, metadata_snapshot_, prewhere_info_list_, max_block_size_rows_, + storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, required_columns{std::move(required_columns_)}, @@ -69,7 +69,7 @@ try task_columns = getReadTaskColumns( storage, metadata_snapshot, data_part, - required_columns, prewhere_info_list, check_columns); + required_columns, prewhere_info, check_columns); auto size_predictor = (preferred_block_size_bytes == 0) ? nullptr @@ -81,7 +81,8 @@ try task = std::make_unique( data_part, all_mark_ranges, part_index_in_query, ordered_names, column_name_set, task_columns.columns, - task_columns.pre_columns, task_columns.should_reorder, std::move(size_predictor)); + task_columns.pre_columns, prewhere_info && prewhere_info->remove_prewhere_column, + task_columns.should_reorder, std::move(size_predictor)); if (!reader) { @@ -93,7 +94,7 @@ try reader = data_part->getReader(task_columns.columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); - if (prewhere_info_list) + if (prewhere_info) pre_reader = data_part->getReader(task_columns.pre_columns, metadata_snapshot, all_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings); } diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 521bbbfdba4..925c437f1ce 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -26,7 +26,7 @@ public: Names required_columns_, MarkRanges mark_ranges, bool use_uncompressed_cache, - const PrewhereInfoListPtr & prewhere_info_list, + const PrewhereInfoPtr & prewhere_info, bool check_columns, const MergeTreeReaderSettings & reader_settings, const Names & virt_column_names = {}, diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp index eb1a80acb49..f57247e39ab 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.cpp @@ -18,12 +18,12 @@ MergeTreeThreadSelectBlockInputProcessor::MergeTreeThreadSelectBlockInputProcess const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const bool use_uncompressed_cache_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_) : MergeTreeBaseSelectProcessor{ - pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_list_, + pool_->getHeader(), storage_, metadata_snapshot_, prewhere_info_, max_block_size_rows_, preferred_block_size_bytes_, preferred_max_column_in_block_size_bytes_, reader_settings_, use_uncompressed_cache_, virt_column_names_}, @@ -78,7 +78,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); - if (prewhere_info_list) + if (prewhere_info) pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, IMergeTreeReader::ValueSizeMap{}, profile_callback); @@ -94,7 +94,7 @@ bool MergeTreeThreadSelectBlockInputProcessor::getNewTask() owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); - if (prewhere_info_list) + if (prewhere_info) pre_reader = task->data_part->getReader(task->pre_columns, metadata_snapshot, rest_mark_ranges, owned_uncompressed_cache.get(), owned_mark_cache.get(), reader_settings, reader->getAvgValueSizeHints(), profile_callback); diff --git a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h index dd3ba8c973c..2b2ed36fc18 100644 --- a/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h +++ b/src/Storages/MergeTree/MergeTreeThreadSelectBlockInputProcessor.h @@ -24,7 +24,7 @@ public: const MergeTreeData & storage_, const StorageMetadataPtr & metadata_snapshot_, const bool use_uncompressed_cache_, - const PrewhereInfoListPtr & prewhere_info_list_, + const PrewhereInfoPtr & prewhere_info_, const MergeTreeReaderSettings & reader_settings_, const Names & virt_column_names_); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 68f2f8f1361..325f54435ed 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -15,8 +15,34 @@ using ExpressionActionsPtr = std::shared_ptr; class ActionsDAG; using ActionsDAGPtr = std::shared_ptr; +struct PrewhereInfo; +using PrewhereInfoPtr = std::shared_ptr; + +struct PrewhereDAGInfo; +using PrewhereDAGInfoPtr = std::shared_ptr; + +struct FilterInfo; +using FilterInfoPtr = std::shared_ptr; + +struct FilterDAGInfo; +using FilterDAGInfoPtr = std::shared_ptr; + +struct InputOrderInfo; +using InputOrderInfoPtr = std::shared_ptr; + +struct TreeRewriterResult; +using TreeRewriterResultPtr = std::shared_ptr; + +class ReadInOrderOptimizer; +using ReadInOrderOptimizerPtr = std::shared_ptr; + +class Cluster; +using ClusterPtr = std::shared_ptr; + struct PrewhereInfo { + /// Information about the preliminary filter expression, if any. + FilterInfoPtr filter_info; /// Actions which are executed in order to alias columns are used for prewhere actions. ExpressionActionsPtr alias_actions; /// Actions which are executed on block in order to get filter column for prewhere step. @@ -26,15 +52,9 @@ struct PrewhereInfo String prewhere_column_name; bool remove_prewhere_column = false; bool need_filter = false; - - PrewhereInfo() = default; - explicit PrewhereInfo(ExpressionActionsPtr prewhere_actions_, String prewhere_column_name_) - : prewhere_actions(std::move(prewhere_actions_)), prewhere_column_name(std::move(prewhere_column_name_)) {} }; -using PrewhereInfoList = std::vector; - -/// Same as PrewhereInfo, but with ActionsDAG +/// Same as PrewhereInfo, but with ActionsDAG. struct PrewhereDAGInfo { ActionsDAGPtr alias_actions; @@ -54,7 +74,15 @@ struct PrewhereDAGInfo /// Helper struct to store all the information about the filter expression. struct FilterInfo { - ActionsDAGPtr actions_dag; + ExpressionActionsPtr actions; + String column_name; + bool do_remove_column = false; +}; + +/// Same as FilterInfo, but with ActionsDAG. +struct FilterDAGInfo +{ + ActionsDAGPtr actions; String column_name; bool do_remove_column = false; @@ -77,20 +105,6 @@ struct InputOrderInfo bool operator !=(const InputOrderInfo & other) const { return !(*this == other); } }; -using PrewhereInfoListPtr = std::shared_ptr; -using PrewhereDAGInfoPtr = std::shared_ptr; -using FilterInfoPtr = std::shared_ptr; -using InputOrderInfoPtr = std::shared_ptr; - -struct TreeRewriterResult; -using TreeRewriterResultPtr = std::shared_ptr; - -class ReadInOrderOptimizer; -using ReadInOrderOptimizerPtr = std::shared_ptr; - -class Cluster; -using ClusterPtr = std::shared_ptr; - /** Query along with some additional data, * that can be used during query processing * inside storage engines. @@ -106,7 +120,7 @@ struct SelectQueryInfo TreeRewriterResultPtr syntax_analyzer_result; - PrewhereInfoListPtr prewhere_info_list; + PrewhereInfoPtr prewhere_info; ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 53fee054f4b..6a2acb74192 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -314,27 +314,38 @@ void StorageBuffer::read( } else { - if (query_info.prewhere_info_list) + if (query_info.prewhere_info) { - for (const auto & prewhere_info : *query_info.prewhere_info_list) + if (query_info.prewhere_info->filter_info) { pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( - header, prewhere_info.prewhere_actions, - prewhere_info.prewhere_column_name, - prewhere_info.remove_prewhere_column); + header, + query_info.prewhere_info->filter_info->actions, + query_info.prewhere_info->filter_info->column_name, + query_info.prewhere_info->filter_info->do_remove_column); }); - - if (prewhere_info.alias_actions) - { - pipe_from_buffers.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, prewhere_info.alias_actions); - }); - } } + + if (query_info.prewhere_info->alias_actions) + { + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + query_info.prewhere_info->alias_actions); + }); + } + + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + query_info.prewhere_info->prewhere_actions, + query_info.prewhere_info->prewhere_column_name, + query_info.prewhere_info->remove_prewhere_column); + }); } auto read_from_buffers = std::make_unique(std::move(pipe_from_buffers)); From 0d9578efc98af871604b11ee79abf0938c339a58 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 14 Feb 2021 14:14:39 +0300 Subject: [PATCH 131/510] Edit and translate MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил английскую версию и выполнил перевод. --- docs/en/operations/caches.md | 15 ++-- .../system-tables/distributed_ddl_queue.md | 2 +- docs/en/sql-reference/table-functions/file.md | 20 ++--- .../sql-reference/table-functions/remote.md | 26 +++--- docs/en/sql-reference/table-functions/url.md | 14 +-- docs/ru/operations/caches.md | 29 ++++++ .../system-tables/distributed_ddl_queue.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 89 +++++++++++-------- .../sql-reference/table-functions/remote.md | 78 ++++++++++------ docs/ru/sql-reference/table-functions/url.md | 43 ++++++--- 10 files changed, 206 insertions(+), 112 deletions(-) create mode 100644 docs/ru/operations/caches.md diff --git a/docs/en/operations/caches.md b/docs/en/operations/caches.md index 7b096b76f75..ec7e4239a9d 100644 --- a/docs/en/operations/caches.md +++ b/docs/en/operations/caches.md @@ -8,18 +8,21 @@ toc_title: Caches When performing queries, ClichHouse uses different caches. Main cache types: + - `mark_cache` — Cache of marks used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family. - `uncompressed_cache` — Cache of uncompressed data used by table engines of the [MergeTree](../engines/table-engines/mergetree-family/mergetree.md) family. Additional cache types: -- DNS cache -- [regexp](../interfaces/formats.md#data-format-regexp) cache -- compiled expressions cache -- [Avro format](../interfaces/formats.md#data-format-avro) schemas cache -- [dictionaries data cache](../sql-reference/dictionaries/index.md) + +- DNS cache. +- [Regexp](../interfaces/formats.md#data-format-regexp) cache. +- Compiled expressions cache. +- [Avro format](../interfaces/formats.md#data-format-avro) schemas cache. +- [Dictionaries](../sql-reference/dictionaries/index.md) data cache. Indirectly used: -- OS page cache + +- OS page cache. To drop cache, use [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md) statements. diff --git a/docs/en/operations/system-tables/distributed_ddl_queue.md b/docs/en/operations/system-tables/distributed_ddl_queue.md index c252458af8a..fa871d215b5 100644 --- a/docs/en/operations/system-tables/distributed_ddl_queue.md +++ b/docs/en/operations/system-tables/distributed_ddl_queue.md @@ -14,7 +14,7 @@ Columns: - `initiator` ([String](../../sql-reference/data-types/string.md)) — Node that executed the query. - `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query start time. - `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — Query finish time. -- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — Duration of query execution (in milliseconds). +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — Duration of query execution (in milliseconds). - `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — Exception code from [ZooKeeper](../../operations/tips.md#zookeeper). **Example** diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index d1eb81e52c6..e4ea59aface 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -5,7 +5,7 @@ toc_title: file # file {#file} -Creates a table from a file. This table function is similar to [url](../../sql-reference/table-functions/url.md) and [hdfs](../../sql-reference/table-functions/hdfs.md) ones. +Creates a table from a file. This table function is similar to [url](../../sql-reference/table-functions/url.md) and [hdfs](../../sql-reference/table-functions/hdfs.md) ones. `file` function can be used in `SELECT` and `INSERT` queries on data in [File](../../engines/table-engines/special/file.md) tables. @@ -15,9 +15,9 @@ Creates a table from a file. This table function is similar to [url](../../sql-r file(path, format, structure) ``` -**Input parameters** +**Parameters** -- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format: `'column1_name column1_type, column2_name column2_type, ...'`. @@ -39,7 +39,7 @@ $ cat /var/lib/clickhouse/user_files/test.csv 78,43,45 ``` -Getting data from a table in `test.csv` and selecting first two rows from it: +Getting data from a table in `test.csv` and selecting the first two rows from it: ``` sql SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 2; @@ -51,7 +51,8 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U │ 3 │ 2 │ 1 │ └─────────┴─────────┴─────────┘ ``` -Getting the first 10 lines of a table that contains 3 columns of UInt32 type from a CSV file: + +Getting the first 10 lines of a table that contains 3 columns of [UInt32](../../sql-reference/data-types/int-uint.md) type from a CSV file: ``` sql SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10; @@ -71,7 +72,6 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U └─────────┴─────────┴─────────┘ ``` - ## Globs in Path {#globs-in-path} Multiple path components can have globs. For being processed file should exists and matches to the whole path pattern (not only suffix or prefix). @@ -81,7 +81,7 @@ Multiple path components can have globs. For being processed file should exists - `{some_string,another_string,yet_another_one}` — Substitutes any of strings `'some_string', 'another_string', 'yet_another_one'`. - `{N..M}` — Substitutes any number in range from N to M including both borders. -Constructions with `{}` are similar to the [remote table function](../../sql-reference/table-functions/remote.md)). +Constructions with `{}` are similar to the [remote](remote.md) table function. **Example** @@ -94,13 +94,13 @@ Suppose we have several files with the following relative paths: - 'another_dir/some_file_2' - 'another_dir/some_file_3' -Query the amount of rows in these files: +Query the number of rows in these files: ``` sql SELECT count(*) FROM file('{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32'); ``` -Query the amount of rows in all files of these two directories: +Query the number of rows in all files of these two directories: ``` sql SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); @@ -124,6 +124,6 @@ SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, **See Also** -- [Virtual columns](https://clickhouse.tech/docs/en/operations/table_engines/#table_engines-virtual_columns) +- [Virtual columns](index.md#table_engines-virtual_columns) [Original article](https://clickhouse.tech/docs/en/sql-reference/table-functions/file/) diff --git a/docs/en/sql-reference/table-functions/remote.md b/docs/en/sql-reference/table-functions/remote.md index 8af5b588412..e80e58a76aa 100644 --- a/docs/en/sql-reference/table-functions/remote.md +++ b/docs/en/sql-reference/table-functions/remote.md @@ -5,7 +5,7 @@ toc_title: remote # remote, remoteSecure {#remote-remotesecure} -Allows to access remote servers without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. `remoteSecure` - same as `remote` but with secured connection. +Allows to access remote servers without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. `remoteSecure` - same as `remote` but with a secured connection. Both functions can be used in `SELECT` and `INSERT` queries. @@ -18,31 +18,31 @@ remoteSecure('addresses_expr', db, table[, 'user'[, 'password'], sharding_key]) remoteSecure('addresses_expr', db.table[, 'user'[, 'password'], sharding_key]) ``` -**Input parameters** +**Parameters** -- `addresses_expr` – An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`, or just `host`. +- `addresses_expr` — An expression that generates addresses of remote servers. This may be just one server address. The server address is `host:port`, or just `host`. The host can be specified as the server name, or as the IPv4 or IPv6 address. An IPv6 address is specified in square brackets. - The port is the TCP port on the remote server. If the port is omitted, it uses [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) from the server’s config file in `remote` (by default, 9000) and [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) in `remoteSecure` (by default, 9440). + The port is the TCP port on the remote server. If the port is omitted, it uses [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) from the server’s config file in `remote` (by default, 9000) and [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure) in `remoteSecure` (by default, 9440). The port is required for an IPv6 address. Type: [String](../../sql-reference/data-types/string.md). -- `db` - Database name. Type: [String](../../sql-reference/data-types/string.md). -- `table` - Table name. Type: [String](../../sql-reference/data-types/string.md). -- `user` - User name. If the user is not specified, `default` is used. Type: [String](../../sql-reference/data-types/string.md). -- `password` - User password. If the password is not specified, an empty password is used. Type: [String](../../sql-reference/data-types/string.md). -- `sharding_key` - Sharding key to support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. Type: [UInt32](../../sql-reference/data-types/int-uint.md). +- `db` — Database name. Type: [String](../../sql-reference/data-types/string.md). +- `table` — Table name. Type: [String](../../sql-reference/data-types/string.md). +- `user` — User name. If the user is not specified, `default` is used. Type: [String](../../sql-reference/data-types/string.md). +- `password` — User password. If the password is not specified, an empty password is used. Type: [String](../../sql-reference/data-types/string.md). +- `sharding_key` — Sharding key to support distributing data across nodes. For example: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. Type: [UInt32](../../sql-reference/data-types/int-uint.md). **Returned value** -Dataset from remote servers. +The dataset from remote servers. **Usage** -Using the `remote` table function is less optimal than creating a `Distributed` table, because in this case the server connection is re-established for every request. In addition, if host names are set, the names are resolved, and errors are not counted when working with various replicas. When processing a large number of queries, always create the `Distributed` table ahead of time, and don’t use the `remote` table function. +Using the `remote` table function is less optimal than creating a `Distributed` table because in this case the server connection is re-established for every request. Also, if hostnames are set, the names are resolved, and errors are not counted when working with various replicas. When processing a large number of queries, always create the `Distributed` table ahead of time, and don’t use the `remote` table function. The `remote` table function can be useful in the following cases: @@ -62,7 +62,7 @@ localhost [2a02:6b8:0:1111::11]:9000 ``` -Multiple addresses can be comma-separated. In this case, ClickHouse will use distributed processing, so it will send the query to all specified addresses (like to shards with different data). Example: +Multiple addresses can be comma-separated. In this case, ClickHouse will use distributed processing, so it will send the query to all specified addresses (like shards with different data). Example: ``` text example01-01-1,example01-02-1 @@ -82,7 +82,7 @@ example01-{01..02}-1 If you have multiple pairs of curly brackets, it generates the direct product of the corresponding sets. -Addresses and parts of addresses in curly brackets can be separated by the pipe symbol (\|). In this case, the corresponding sets of addresses are interpreted as replicas, and the query will be sent to the first healthy replica. However, the replicas are iterated in the order currently set in the [load_balancing](../../operations/settings/settings.md) setting. This example specifies two shards that each have two replicas: +Addresses and parts of addresses in curly brackets can be separated by the pipe symbol (\|). In this case, the corresponding sets of addresses are interpreted as replicas, and the query will be sent to the first healthy replica. However, the replicas are iterated in the order currently set in the [load_balancing](../../operations/settings/settings.md#settings-load_balancing) setting. This example specifies two shards that each have two replicas: ``` text example01-{01..02}-{1|2} diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index d70774b7588..0d004f9601a 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -15,25 +15,25 @@ toc_title: url url(URL, format, structure) ``` -**Input parameters** +**Parameters** -- `URL` - HTTP or HTTPS server address, which can accept `GET` (for `SELECT`) or `POST` (for `INSERT`) requests. Type: [String](../../sql-reference/data-types/string.md). -- `format` - [Format](../../interfaces/formats.md#formats) of the data. Type: [String](../../sql-reference/data-types/string.md). -- `structure` - Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). +- `URL` — HTTP or HTTPS server address, which can accept `GET` (for `SELECT`) or `POST` (for `INSERT`) queries. Type: [String](../../sql-reference/data-types/string.md). +- `format` — [Format](../../interfaces/formats.md#formats) of the data. Type: [String](../../sql-reference/data-types/string.md). +- `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). **Returned value** -A table with the specified format and structure and with data from the defined URL. +A table with the specified format and structure and with data from the defined `URL`. **Examples** -Getting the first 3 lines of a table that contains columns of `String` and `UInt32` type from HTTP-server which answers in `CSV` format. +Getting the first 3 lines of a table that contains columns of `String` and [UInt32](../../sql-reference/data-types/int-uint.md) type from HTTP-server which answers in [CSV](../../interfaces/formats.md/#csv) format. ``` sql SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; ``` -Inserting data from a URL into a table: +Inserting data from a `URL` into a table: ``` sql CREATE TABLE test_table (column1 String, column2 UInt32) ENGINE=Memory; diff --git a/docs/ru/operations/caches.md b/docs/ru/operations/caches.md new file mode 100644 index 00000000000..cf7118eb1f3 --- /dev/null +++ b/docs/ru/operations/caches.md @@ -0,0 +1,29 @@ +--- +toc_priority: 65 +toc_title: Кеши +--- + +# Типы кеша {#cache-types} + +При выполнении запросов ClickHouse использует различные типы кеша. + +Основные типы кеша: + +- `mark_cache` — кеш меток, используемых движками таблиц семейства [MergeTree](../engines/table-engines/mergetree-family/mergetree.md). +- `uncompressed_cache` — кеш несжатых данных, используемых движками таблиц семейства [MergeTree](../engines/table-engines/mergetree-family/mergetree.md). + +Дополнительные типы кеша: + +- DNS-кеш. +- Кеш данных формата [regexp](../interfaces/formats.md#data-format-regexp). +- Кеш скомпилированных выражений. +- Кеш схем формата [Avro](../interfaces/formats.md#data-format-avro). +- Кеш данных в [словарях](../sql-reference/dictionaries/index.md). + +Непрямое использование: + +- Кеш страницы ОС. + +Чтобы удалить кеш, используйте выражения типа [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md). + +[Оригинальная статья](https://clickhouse.tech/docs/ru/operations/caches/) diff --git a/docs/ru/operations/system-tables/distributed_ddl_queue.md b/docs/ru/operations/system-tables/distributed_ddl_queue.md index 058ed06f639..71be69e98d7 100644 --- a/docs/ru/operations/system-tables/distributed_ddl_queue.md +++ b/docs/ru/operations/system-tables/distributed_ddl_queue.md @@ -14,7 +14,7 @@ - `initiator` ([String](../../sql-reference/data-types/string.md)) — узел, выполнивший запрос. - `query_start_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время начала запроса. - `query_finish_time` ([DateTime](../../sql-reference/data-types/datetime.md)) — время окончания запроса. -- `query_duration_ms` ([UInt64](../../sql-reference/data-types/datetime64.md)) — продолжительность выполнения запроса (в миллисекундах). +- `query_duration_ms` ([UInt64](../../sql-reference/data-types/int-uint.md)) — продолжительность выполнения запроса (в миллисекундах). - `exception_code` ([Enum8](../../sql-reference/data-types/enum.md)) — код исключения из [ZooKeeper](../../operations/tips.md#zookeeper). **Пример** diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index d3e6e106125..ca1ac8b29db 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -5,23 +5,27 @@ toc_title: file # file {#file} -Создаёт таблицу из файла. Данная табличная функция похожа на табличные функции [file](file.md) и [hdfs](hdfs.md). +Создаёт таблицу из файла. Данная табличная функция похожа на табличные функции [url](../../sql-reference/table-functions/url.md) and [hdfs](../../sql-reference/table-functions/hdfs.md). + +Функция `file` может использоваться в запросах `SELECT` и `INSERT` движка таблиц [File](../../engines/table-engines/special/file.md). + +**Синтаксис** ``` sql file(path, format, structure) ``` -**Входные параметры** +**Параметры** -- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, \``'abc', 'def'` — строки. +- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. - `format` — [формат](../../interfaces/formats.md#formats) файла. -- `structure` — структура таблицы. Формат `'colunmn1_name column1_ype, column2_name column2_type, ...'`. +- `structure` — структура таблицы. Формат: `'colunmn1_name column1_ype, column2_name column2_type, ...'`. **Возвращаемое значение** Таблица с указанной структурой, предназначенная для чтения или записи данных в указанном файле. -**Пример** +**Примеры** Настройка `user_files_path` и содержимое файла `test.csv`: @@ -35,12 +39,10 @@ $ cat /var/lib/clickhouse/user_files/test.csv 78,43,45 ``` -Таблица из `test.csv` и выборка первых двух строк из неё: +Получение данных из таблицы в файле `test.csv` и выборка первых двух строк из неё: ``` sql -SELECT * -FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') -LIMIT 2 +SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 2; ``` ``` text @@ -50,45 +52,61 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` +Получение первых 10 строк таблицы, содержащей 3 столбца типа [UInt32](../../sql-reference/data-types/int-uint.md), из CSV-файла: + +``` sql +SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10; +``` + +Вставка данных из файла в таблицу: + +``` sql +INSERT INTO FUNCTION file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') VALUES (1, 2, 3), (3, 2, 1); +SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32'); +``` + +``` text +┌─column1─┬─column2─┬─column3─┐ +│ 1 │ 2 │ 3 │ +│ 3 │ 2 │ 1 │ +└─────────┴─────────┴─────────┘ +``` + +## Шаблоны в компонентах пути {#globs-in-path} + Шаблоны могут содержаться в нескольких компонентах пути. Обрабатываются только существующие файлы, название которых целиком удовлетворяет шаблону (не только суффиксом или префиксом). -- `*` — Заменяет любое количество любых символов кроме `/`, включая отсутствие символов. -- `?` — Заменяет ровно один любой символ. -- `{some_string,another_string,yet_another_one}` — Заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. -- `{N..M}` — Заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). +- `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. +- `?` — заменяет ровно один любой символ. +- `{some_string,another_string,yet_another_one}` — заменяет любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — заменяет любое число в интервале от `N` до `M` включительно (может содержать ведущие нули). Конструкция с `{}` аналогична табличной функции [remote](remote.md). **Пример** -1. Предположим у нас есть несколько файлов со следующими относительными путями: +Предположим у нас есть несколько файлов со следующими относительными путями: -- ‘some_dir/some_file_1’ -- ‘some_dir/some_file_2’ -- ‘some_dir/some_file_3’ -- ‘another_dir/some_file_1’ -- ‘another_dir/some_file_2’ -- ‘another_dir/some_file_3’ +- 'some_dir/some_file_1' +- 'some_dir/some_file_2' +- 'some_dir/some_file_3' +- 'another_dir/some_file_1' +- 'another_dir/some_file_2' +- 'another_dir/some_file_3' -1. Запросим количество строк в этих файлах: - - +Запросим количество строк в этих файлах: ``` sql -SELECT count(*) -FROM file('{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32') +SELECT count(*) FROM file('{some,another}_dir/some_file_{1..3}', 'TSV', 'name String, value UInt32'); ``` -1. Запросим количество строк во всех файлах этих двух директорий: - - +Запросим количество строк во всех файлах этих двух директорий: ``` sql -SELECT count(*) -FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32') +SELECT count(*) FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32'); ``` -!!! warning "Warning" +!!! warning "Предупреждение" Если ваш список файлов содержит интервал с ведущими нулями, используйте конструкцию с фигурными скобками для каждой цифры по отдельности или используйте `?`. **Пример** @@ -96,17 +114,16 @@ FROM file('{some,another}_dir/*', 'TSV', 'name String, value UInt32') Запрос данных из файлов с именами `file000`, `file001`, … , `file999`: ``` sql -SELECT count(*) -FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32') +SELECT count(*) FROM file('big_dir/file{0..9}{0..9}{0..9}', 'CSV', 'name String, value UInt32'); ``` ## Виртуальные столбцы {#virtualnye-stolbtsy} -- `_path` — Путь к файлу. -- `_file` — Имя файла. +- `_path` — путь к файлу. +- `_file` — имя файла. **Смотрите также** - [Виртуальные столбцы](index.md#table_engines-virtual_columns) -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/file/) +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table-functions/file/) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index 901317a805d..435fb5bb6d7 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -5,9 +5,11 @@ toc_title: remote # remote, remoteSecure {#remote-remotesecure} -Позволяет обратиться к удалённым серверам без создания таблицы типа `Distributed`. +Позволяет обратиться к удалённым серверам без создания таблицы типа [Distributed](../../engines/table-engines/special/distributed.md). Функция `remoteSecure` такая же, как и `remote`, но с защищенным соединением. -Сигнатуры: +Обе функции могут быть использованы в запросах типа `SELECT` и `INSERT`. + +**Синтаксис** ``` sql remote('addresses_expr', db, table[, 'user'[, 'password']]) @@ -16,12 +18,40 @@ remoteSecure('addresses_expr', db, table[, 'user'[, 'password']]) remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) ``` -`addresses_expr` - выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера - это `хост:порт`, или только `хост`. Хост может быть указан в виде имени сервера, или в виде IPv4 или IPv6 адреса. IPv6 адрес указывается в квадратных скобках. Порт - TCP-порт удалённого сервера. Если порт не указан, используется `tcp_port` из конфигурационного файла сервера (по умолчанию - 9000). +**Параметры** + +- `addresses_expr` — выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера — это `хост:порт`, или только `хост`. + + Хост может быть указан в виде имени сервера, или в виде IPv4 или IPv6 адреса. IPv6 адрес указывается в квадратных скобках. + + Порт — TCP-порт удалённого сервера. Если порт не указан, используется [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) из конфигурационного файла сервера, к которому обратились через функцию `remote` (по умолчанию - 9000), и [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure), к которому обратились через функцию `remoteSecure` (по умолчанию — 9440). -!!! important "Важно" С IPv6-адресом обязательно нужно указывать порт. -Примеры: + Тип: [String](../../sql-reference/data-types/string.md). + +- `db` — имя базы данных. Тип: [String](../../sql-reference/data-types/string.md). +- `table` — имя таблицы. Тип: [String](../../sql-reference/data-types/string.md). +- `user` — имя пользователя. Если пользователь не указан, то по умолчанию `default`. Тип: [String](../../sql-reference/data-types/string.md). +- `password` — пароль. Если пароль не указан, то используется пустой пароль. Тип: [String](../../sql-reference/data-types/string.md). +- `sharding_key` — ключ шардирования для поддержки распределения данных между узлами. Например: `insert into remote('127.0.0.1:9000,127.0.0.2', db, table, 'default', rand())`. Тип: [UInt32](../../sql-reference/data-types/int-uint.md). + +**Возвращаемое значение** + +Набор данных с удаленных серверов. + +**Использование** + +Использование табличной функции `remote` менее оптимально, чем создание таблицы типа `Distributed`, так как в этом случае соединения с серверами устанавливаются заново при каждом запросе. В случае задания имён хостов делается резолвинг имён, а также не ведётся подсчёт ошибок при работе с разными репликами. При обработке большого количества запросов всегда создавайте таблицу типа `Distributed` заранее, не используйте табличную функцию `remote`. + +Табличная функция `remote` может быть полезна в следующих случаях: + +- Обращение на конкретный сервер в целях сравнения данных, отладки и тестирования. +- Запросы между разными кластерами ClickHouse в целях исследований. +- Нечастые распределённые запросы, задаваемые вручную. +- Распределённые запросы, где набор серверов определяется каждый раз заново. + +**Адреса** ``` text example01-01-1 @@ -32,9 +62,7 @@ localhost [2a02:6b8:0:1111::11]:9000 ``` -Адреса можно указать через запятую, в этом случае ClickHouse обработает запрос как распределённый, т.е. отправит его по всем указанным адресам как на шарды с разными данными. - -Пример: +Адреса можно указать через запятую. В этом случае ClickHouse обработает запрос как распределённый, т.е. отправит его по всем указанным адресам как на шарды с разными данными. Пример: ``` text example01-01-1,example01-02-1 @@ -46,38 +74,36 @@ example01-01-1,example01-02-1 example01-0{1,2}-1 ``` -В фигурных скобках может быть указан диапазон (неотрицательных целых) чисел через две точки. В этом случае, диапазон раскрывается в множество значений, генерирующих адреса шардов. Если запись первого числа начинается с нуля, то значения формируются с таким же выравниванием нулями. Предыдущий пример может быть записан следующим образом: +В фигурных скобках может быть указан диапазон (неотрицательных целых) чисел через две точки. В этом случае диапазон раскрывается в множество значений, генерирующих адреса шардов. Если запись первого числа начинается с нуля, то значения формируются с таким же выравниванием нулями. Предыдущий пример может быть записан следующим образом: ``` text example01-{01..02}-1 ``` -При наличии нескольких пар фигурных скобок, генерируется прямое произведение соответствующих множеств. +При наличии нескольких пар фигурных скобок генерируется прямое произведение соответствующих множеств. -Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае, соответствующие множества адресов понимаются как реплики - запрос будет отправлен на первую живую реплику. При этом, реплики перебираются в порядке, согласно текущей настройке [load_balancing](../../operations/settings/settings.md). - -Пример: +Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае соответствующие множества адресов понимаются как реплики — запрос будет отправлен на первую живую реплику. При этом реплики перебираются в порядке, согласно текущей настройке [load_balancing](../../operations/settings/settings.md#settings-load_balancing). В этом примере указано два шарда, в каждом из которых имеется две реплики: ``` text example01-{01..02}-{1|2} ``` -В этом примере указано два шарда, в каждом из которых имеется две реплики. +Количество генерируемых адресов ограничено константой. Сейчас это 1000 адресов. -Количество генерируемых адресов ограничено константой - сейчас это 1000 штук. +**Примеры** -Использование табличной функции `remote` менее оптимально, чем создание таблицы типа `Distributed`, так как в этом случае, соединения с серверами устанавливаются заново при каждом запросе, в случае задания имён хостов, делается резолвинг имён, а также не ведётся подсчёт ошибок при работе с разными репликами. При обработке большого количества запросов, всегда создавайте `Distributed` таблицу заранее, не используйте табличную функцию `remote`. +Выборка данных с удаленного сервера: -Табличная функция `remote` может быть полезна для следующих случаях: +``` sql +SELECT * FROM remote('127.0.0.1', db.remote_engine_table) LIMIT 3; +``` -- обращение на конкретный сервер в целях сравнения данных, отладки и тестирования; -- запросы между разными кластерами ClickHouse в целях исследований; -- нечастых распределённых запросов, задаваемых вручную; -- распределённых запросов, где набор серверов определяется каждый раз заново. +Вставка данных с удаленного сервера в таблицу: -Если пользователь не задан,то используется `default`. -Если пароль не задан, то используется пустой пароль. +``` sql +CREATE TABLE remote_table (name String, value UInt32) ENGINE=Memory; +INSERT INTO FUNCTION remote('127.0.0.1', currentDatabase(), 'remote_table') VALUES ('test', 42); +SELECT * FROM remote_table; +``` -`remoteSecure` - аналогично функции `remote`, но с соединением по шифрованному каналу. Порт по умолчанию - `tcp_port_secure` из конфига или 9440. - -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/remote/) +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table-functions/remote/) diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index 0cd7c24c663..afb4a23b88e 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -5,21 +5,40 @@ toc_title: url # url {#url} -`url(URL, format, structure)` - возвращает таблицу со столбцами, указанными в -`structure`, созданную из данных находящихся по `URL` в формате `format`. +Функция `url` создает таблицу с помощью адреса `URL`, формата данных и структуры таблицы. -URL - адрес, по которому сервер принимает `GET` и/или `POST` запросы по -протоколу HTTP или HTTPS. +Функция `url` может быть использована в запросах `SELECT` и `INSERT` в таблицах движка [URL](../../engines/table-engines/special/url.md). -format - [формат](../../interfaces/formats.md#formats) данных. - -structure - структура таблицы в форме `'UserID UInt64, Name String'`. Определяет имена и типы столбцов. - -**Пример** +**Синтаксис** ``` sql --- получение 3-х строк таблицы, состоящей из двух колонк типа String и UInt32 от сервера, отдающего данные в формате CSV -SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3 +url(URL, format, structure) ``` -[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/table_functions/url/) +**Параметры** + +- `URL` — HTTP или HTTPS-адрес сервера, который может принимать запросы `GET` (для `SELECT`) или `POST` (для `INSERT`). Тип: [String](../../sql-reference/data-types/string.md). +- `format` — [формат](../../interfaces/formats.md#formats) данных. Тип: [String](../../sql-reference/data-types/string.md). +- `structure` — структура таблицы в формате `'UserID UInt64, Name String'`. Определяет имена и типы столбцов. Тип: [String](../../sql-reference/data-types/string.md). + +**Возвращаемое значение** + +Таблица с указанными форматом и структурой, а также с данными, полученными из указанного адреса `URL`. + +**Примеры** + +Получение первых 3 строк таблицы, содержащей столбцы типа `String` и [UInt32](../../sql-reference/data-types/int-uint.md), с HTTP-сервера в формате [CSV](../../interfaces/formats.md/#csv). + +``` sql +SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; +``` + +Вставка данных в таблицу: + +``` sql +CREATE TABLE test_table (column1 String, column2 UInt32) ENGINE=Memory; +INSERT INTO FUNCTION url('http://127.0.0.1:8123/?query=INSERT+INTO+test_table+FORMAT+CSV', 'CSV', 'column1 String, column2 UInt32') VALUES ('http interface', 42); +SELECT * FROM test_table; +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/table-functions/url/) From 45e90961f7612f46a315d80b6b8e7a856c2bed33 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 14 Feb 2021 21:16:40 +0400 Subject: [PATCH 132/510] Store and process alias_actions in FilterInfo --- src/Interpreters/InterpreterSelectQuery.cpp | 13 +++++++++++++ .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 3 +++ .../MergeTree/MergeTreeBlockReadUtils.cpp | 16 ++++++++++++---- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 3 +++ src/Storages/MergeTree/MergeTreeReadPool.h | 3 ++- src/Storages/SelectQueryInfo.h | 1 + src/Storages/StorageBuffer.cpp | 10 ++++++++++ 7 files changed, 44 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 0a639922e55..d13c28e8ff2 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1215,6 +1215,16 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { auto & filter_info = *prewhere_info.filter_info; + if (filter_info.alias_actions) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + filter_info.alias_actions); + }); + } + pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -1599,6 +1609,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc { query_info.prewhere_info->filter_info = std::make_shared(); + if (alias_actions) + query_info.prewhere_info->filter_info->alias_actions = std::make_shared(std::move(alias_actions)); + if (expressions.filter_info->actions) query_info.prewhere_info->filter_info->actions = std::make_shared(expressions.filter_info->actions); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 2ab275a7bd6..96993e4a106 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -338,6 +338,9 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P { auto & filter_info = *prewhere_info->filter_info; + if (filter_info.alias_actions) + filter_info.alias_actions->execute(block); + if (filter_info.actions) filter_info.actions->execute(block); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index d5fb2f3300c..f4a5b1fcb9e 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -269,10 +269,18 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { - if (prewhere_info->filter_info && prewhere_info->filter_info->actions) + if (prewhere_info->filter_info) { - const auto required_column_names = prewhere_info->filter_info->actions->getRequiredColumns(); - pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); + if (prewhere_info->filter_info->alias_actions) + { + const auto required_column_names = prewhere_info->filter_info->alias_actions->getRequiredColumns(); + pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); + } + else if (prewhere_info->filter_info->actions) + { + const auto required_column_names = prewhere_info->filter_info->actions->getRequiredColumns(); + pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); + } } if (prewhere_info->alias_actions) @@ -280,7 +288,7 @@ MergeTreeReadTaskColumns getReadTaskColumns( const auto required_column_names = prewhere_info->alias_actions->getRequiredColumns(); pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); } - else + else if (prewhere_info->prewhere_actions) { const auto required_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 5995e2318c0..3c79ed73a16 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -899,6 +899,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (prewhere_info->filter_info) { + if (prewhere_info->filter_info->alias_actions) + prewhere_info->filter_info->alias_actions->execute(block); + if (prewhere_info->filter_info->actions) prewhere_info->filter_info->actions->execute(block); diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index aa6811661e6..366e9a2381a 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -71,7 +71,8 @@ private: public: MergeTreeReadPool( const size_t threads_, const size_t sum_marks_, const size_t min_marks_for_concurrent_read_, - RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, const PrewhereInfoPtr & prewhere_info_, + RangesInDataParts && parts_, const MergeTreeData & data_, const StorageMetadataPtr & metadata_snapshot_, + const PrewhereInfoPtr & prewhere_info_, const bool check_columns_, const Names & column_names_, const BackoffSettings & backoff_settings_, size_t preferred_block_size_bytes_, const bool do_not_steal_tasks_ = false); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 325f54435ed..a87ff2f40d3 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -74,6 +74,7 @@ struct PrewhereDAGInfo /// Helper struct to store all the information about the filter expression. struct FilterInfo { + ExpressionActionsPtr alias_actions; ExpressionActionsPtr actions; String column_name; bool do_remove_column = false; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 2ed7fe377c4..64bcdd2d145 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -323,6 +323,16 @@ void StorageBuffer::read( { if (query_info.prewhere_info->filter_info) { + if (query_info.prewhere_info->filter_info->alias_actions) + { + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + query_info.prewhere_info->filter_info->alias_actions); + }); + } + pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( From c24221b04f1bc511cc0a9524e6e2388c03d08246 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:53:44 +0300 Subject: [PATCH 133/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 268a7565b81..c80f8934f72 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -546,7 +546,7 @@ accurateCastOrNull(x, T) **Returned value** -- The value in specified data type `T`. +- The value, converted to the specified data type `T`. **Example** From cdac3cf9ce17391479681444b48e005dc24327d7 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:53:51 +0300 Subject: [PATCH 134/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index e16fa438aed..985dd16c231 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -423,7 +423,7 @@ SELECT uuid = uuid2; ## CAST(x, T) {#type_conversion_function-cast} -Преобразует вхожное значение `x` в указананный тип данных `T`. +Преобразует входное значение `x` в указанный тип данных `T`. Поддерживается также синтаксис `CAST(x AS t)`. From cda9dc7600880ee35582cfe1d98d15bd4df43c28 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:54:02 +0300 Subject: [PATCH 135/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 985dd16c231..3c9d3993120 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -427,7 +427,7 @@ SELECT uuid = uuid2; Поддерживается также синтаксис `CAST(x AS t)`. -Обратите внимание, что если значение `x` не соответствует границам типа `T`, функция переполняется. Например, `CAST(-1, 'UInt8')` возвращает 255. +Обратите внимание, что если значение `x` не может быть преобразовано к типу `T`, возникает переполнение. Например, `CAST(-1, 'UInt8')` возвращает 255. **Пример** From b82bf79c5245092fea0a866f3cae2934262d66d6 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:54:10 +0300 Subject: [PATCH 136/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 3c9d3993120..16e52efceec 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -494,7 +494,7 @@ SELECT toTypeName(CAST(x, 'Nullable(UInt16)')) FROM t_null; Преобразует входное значение `x` в указанный тип данных `T`. -Отличие от [cast(x, T)](#type_conversion_function-cast) в том, что `accurateCast` не допускает переполнения числовых типов, если значение типа `x` не соответствует границам типа `T`. Например, `accurateCast(-1, 'UInt8')` вернет ошибку. +В отличие от функции [cast(x, T)](#type_conversion_function-cast), `accurateCast` не допускает переполнения при преобразовании числовых типов. Например, `accurateCast(-1, 'UInt8')` вызовет исключение. **Примеры** From 82701ecbeccf88f38a73ccb0ea556267d2fa99a0 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:54:15 +0300 Subject: [PATCH 137/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 16e52efceec..0723ed2c752 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -527,7 +527,7 @@ Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in c Преобразует входное значение `x` в указанный тип данных `T`. -Всегда возвращает тип [Nullable](../../sql-reference/data-types/nullable.md), и возвращает [NULL](../../sql-reference/syntax.md#null-literal), если приведенное значение не может быть представлено в целевом типе. +Всегда возвращает тип [Nullable](../../sql-reference/data-types/nullable.md). Если исходное значение не может быть преобразовано к целевому типу, возвращает [NULL](../../sql-reference/syntax.md#null-literal). **Синтаксис** From 994b998df9863e772b438a858a2cdabdb2ce27ea Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:54:20 +0300 Subject: [PATCH 138/510] Update docs/ru/sql-reference/operators/in.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/operators/in.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/operators/in.md b/docs/ru/sql-reference/operators/in.md index c2d88a729be..e0412747898 100644 --- a/docs/ru/sql-reference/operators/in.md +++ b/docs/ru/sql-reference/operators/in.md @@ -17,8 +17,7 @@ SELECT (CounterID, UserID) IN ((34, 123), (101500, 456)) FROM ... В качестве правой части оператора может быть множество константных выражений, множество кортежей с константными выражениями (показано в примерах выше), а также имя таблицы или подзапрос SELECT в скобках. -ClickHouse допускает различные типы в левой и правой частях подзапроса `IN`. -В этом случае он преобразует левую сторону в тип правой стороны, применяя функцию [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null). +Если типы данных в левой и правой частях подзапроса `IN` различаются, ClickHouse преобразует значение в левой части к типу данных из правой части. Преобразование выполняется по аналогии с функцией [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null), т.е. тип данных становится [Nullable](../../sql-reference/data-types/nullable.md), а если преобразование не может быть выполнено, возвращается значение [NULL](../../sql-reference/syntax.md#null-literal). **Пример** From 2a71053c695ee6deb84d8583c51dec0cc74dcdb1 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 01:54:25 +0300 Subject: [PATCH 139/510] Update docs/en/sql-reference/operators/in.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/operators/in.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/in.md b/docs/en/sql-reference/operators/in.md index 4796c0f6bc0..34866f3d09a 100644 --- a/docs/en/sql-reference/operators/in.md +++ b/docs/en/sql-reference/operators/in.md @@ -17,7 +17,7 @@ Don’t list too many values explicitly (i.e. millions). If a data set is large The right side of the operator can be a set of constant expressions, a set of tuples with constant expressions (shown in the examples above), or the name of a database table or SELECT subquery in brackets. -ClickHouse allows different types in the left and right parts of `IN` subquery. In this case it converts the left hand side to the type of the right hand side as if the [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null) function is applied. +ClickHouse allows types to differ in the left and the right parts of `IN` subquery. In this case it converts the left side value to the type of the right side, as if the [accurateCastOrNull](../functions/type-conversion-functions.md#type_conversion_function-accurate-cast_or_null) function is applied. That means, that the data type becomes [Nullable](../../sql-reference/data-types/nullable.md), and if the conversion cannot be performed, it returns [NULL](../../sql-reference/syntax.md#null-literal). **Example** From df09a5cac91f216ea360dc95b0afdc50b3e6fe44 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 15 Feb 2021 16:21:36 +0300 Subject: [PATCH 140/510] Refactor 3.0 and fix tests --- src/Client/Connection.cpp | 305 ++++++++------- src/Client/ConnectionPoolWithFailover.cpp | 32 +- src/Client/ConnectionPoolWithFailover.h | 3 - src/Client/HedgedConnections.cpp | 211 +++++----- src/Client/HedgedConnections.h | 60 +-- src/Client/HedgedConnectionsFactory.cpp | 370 +++++++++--------- src/Client/HedgedConnectionsFactory.h | 73 ++-- src/Common/Epoll.cpp | 11 +- src/Common/Epoll.h | 9 +- src/Common/TimerDescriptor.cpp | 8 +- src/Common/TimerDescriptor.h | 2 +- .../configs/remote_servers.xml | 4 + .../integration/test_hedged_requests/test.py | 253 ++++++++++-- .../configs/remote_servers.xml | 4 + .../configs/users1.xml | 1 - .../test_hedged_requests_parallel/test.py | 113 ++++-- 16 files changed, 882 insertions(+), 577 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 2820046782f..5ef326acb73 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -59,19 +59,87 @@ namespace ErrorCodes void Connection::connect(const ConnectionTimeouts & timeouts) +{ + if (connected) + disconnect(); + + prepare(timeouts); + sendHello(); + receiveHello(); + + LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", + server_name, server_version_major, server_version_minor, server_version_patch); +} + + +void Connection::disconnect() +{ + maybe_compressed_out = nullptr; + in = nullptr; + last_input_packet_type.reset(); + out = nullptr; // can write to socket + if (socket) + socket->close(); + socket = nullptr; + connected = false; +} + +void Connection::prepare(const ConnectionTimeouts & timeouts) { try { - if (connected) - disconnect(); + LOG_TRACE( + log_wrapper.get(), + "Connecting. Database: {}. User: {}{}{}", + default_database.empty() ? "(not specified)" : default_database, + user, + static_cast(secure) ? ". Secure" : "", + static_cast(compression) ? "" : ". Uncompressed"); - prepare(timeouts); + if (static_cast(secure)) + { +#if USE_SSL + socket = std::make_unique(); - sendHello(); - receiveHello(); + /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) + /// work we need to pass host name separately. It will be send into TLS Hello packet to let + /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). + static_cast(socket.get())->setPeerHostName(host); +#else + throw Exception{ + "tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; +#endif + } + else + { + socket = std::make_unique(); + } - LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", - server_name, server_version_major, server_version_minor, server_version_patch); + current_resolved_address = DNSResolver::instance().resolveAddress(host, port); + + const auto & connection_timeout = static_cast(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout; + socket->connect(*current_resolved_address, connection_timeout); + socket->setReceiveTimeout(timeouts.receive_timeout); + socket->setSendTimeout(timeouts.send_timeout); + socket->setNoDelay(true); + if (timeouts.tcp_keep_alive_timeout.totalSeconds()) + { + socket->setKeepAlive(true); + socket->setOption( + IPPROTO_TCP, +#if defined(TCP_KEEPALIVE) + TCP_KEEPALIVE +#else + TCP_KEEPIDLE // __APPLE__ +#endif + , + timeouts.tcp_keep_alive_timeout); + } + + in = std::make_shared(*socket); + out = std::make_shared(*socket); + + connected = true; } catch (Poco::Net::NetException & e) { @@ -90,73 +158,11 @@ void Connection::connect(const ConnectionTimeouts & timeouts) } -void Connection::disconnect() -{ - maybe_compressed_out = nullptr; - in = nullptr; - last_input_packet_type.reset(); - out = nullptr; // can write to socket - if (socket) - socket->close(); - socket = nullptr; - connected = false; -} - -void Connection::prepare(const ConnectionTimeouts & timeouts) -{ - LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", - default_database.empty() ? "(not specified)" : default_database, - user, - static_cast(secure) ? ". Secure" : "", - static_cast(compression) ? "" : ". Uncompressed"); - - if (static_cast(secure)) - { -#if USE_SSL - socket = std::make_unique(); - - /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) - /// work we need to pass host name separately. It will be send into TLS Hello packet to let - /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). - static_cast(socket.get())->setPeerHostName(host); -#else - throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; -#endif - } - else - { - socket = std::make_unique(); - } - - current_resolved_address = DNSResolver::instance().resolveAddress(host, port); - - const auto & connection_timeout = static_cast(secure) ? timeouts.secure_connection_timeout : timeouts.connection_timeout; - socket->connect(*current_resolved_address, connection_timeout); - socket->setReceiveTimeout(timeouts.receive_timeout); - socket->setSendTimeout(timeouts.send_timeout); - socket->setNoDelay(true); - if (timeouts.tcp_keep_alive_timeout.totalSeconds()) - { - socket->setKeepAlive(true); - socket->setOption(IPPROTO_TCP, -#if defined(TCP_KEEPALIVE) - TCP_KEEPALIVE -#else - TCP_KEEPIDLE // __APPLE__ -#endif - , timeouts.tcp_keep_alive_timeout); - } - - in = std::make_shared(*socket); - out = std::make_shared(*socket); - - connected = true; -} - - void Connection::sendHello() { - /** Disallow control characters in user controlled parameters + try + { + /** Disallow control characters in user controlled parameters * to mitigate the possibility of SSRF. * The user may do server side requests with 'remote' table function. * Malicious user with full r/w access to ClickHouse @@ -165,85 +171,116 @@ void Connection::sendHello() * Limiting number of possible characters in user-controlled part of handshake * will mitigate this possibility but doesn't solve it completely. */ - auto has_control_character = [](const std::string & s) - { - for (auto c : s) - if (isControlASCII(c)) - return true; - return false; - }; + auto has_control_character = [](const std::string & s) { + for (auto c : s) + if (isControlASCII(c)) + return true; + return false; + }; - if (has_control_character(default_database) - || has_control_character(user) - || has_control_character(password)) - throw Exception("Parameters 'default_database', 'user' and 'password' must not contain ASCII control characters", ErrorCodes::BAD_ARGUMENTS); + if (has_control_character(default_database) || has_control_character(user) || has_control_character(password)) + throw Exception( + "Parameters 'default_database', 'user' and 'password' must not contain ASCII control characters", + ErrorCodes::BAD_ARGUMENTS); - writeVarUInt(Protocol::Client::Hello, *out); - writeStringBinary((DBMS_NAME " ") + client_name, *out); - writeVarUInt(DBMS_VERSION_MAJOR, *out); - writeVarUInt(DBMS_VERSION_MINOR, *out); - // NOTE For backward compatibility of the protocol, client cannot send its version_patch. - writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); - writeStringBinary(default_database, *out); - /// If interserver-secret is used, one do not need password - /// (NOTE we do not check for DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET, since we cannot ignore inter-server secret if it was requested) - if (!cluster_secret.empty()) - { - writeStringBinary(USER_INTERSERVER_MARKER, *out); - writeStringBinary("" /* password */, *out); + writeVarUInt(Protocol::Client::Hello, *out); + writeStringBinary((DBMS_NAME " ") + client_name, *out); + writeVarUInt(DBMS_VERSION_MAJOR, *out); + writeVarUInt(DBMS_VERSION_MINOR, *out); + // NOTE For backward compatibility of the protocol, client cannot send its version_patch. + writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); + writeStringBinary(default_database, *out); + /// If interserver-secret is used, one do not need password + /// (NOTE we do not check for DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET, since we cannot ignore inter-server secret if it was requested) + if (!cluster_secret.empty()) + { + writeStringBinary(USER_INTERSERVER_MARKER, *out); + writeStringBinary("" /* password */, *out); #if USE_SSL - sendClusterNameAndSalt(); + sendClusterNameAndSalt(); #else - throw Exception( - "Inter-server secret support is disabled, because ClickHouse was built without SSL library", - ErrorCodes::SUPPORT_IS_DISABLED); + throw Exception( + "Inter-server secret support is disabled, because ClickHouse was built without SSL library", + ErrorCodes::SUPPORT_IS_DISABLED); #endif - } - else - { - writeStringBinary(user, *out); - writeStringBinary(password, *out); - } + } + else + { + writeStringBinary(user, *out); + writeStringBinary(password, *out); + } - out->next(); + out->next(); + } + catch (Poco::Net::NetException & e) + { + disconnect(); + + /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::NETWORK_ERROR); + } + catch (Poco::TimeoutException & e) + { + disconnect(); + + /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); + } } void Connection::receiveHello() { - /// Receive hello packet. - UInt64 packet_type = 0; - - /// Prevent read after eof in readVarUInt in case of reset connection - /// (Poco should throw such exception while reading from socket but - /// sometimes it doesn't for unknown reason) - if (in->eof()) - throw Poco::Net::NetException("Connection reset by peer"); - - readVarUInt(packet_type, *in); - if (packet_type == Protocol::Server::Hello) + try { - readStringBinary(server_name, *in); - readVarUInt(server_version_major, *in); - readVarUInt(server_version_minor, *in); - readVarUInt(server_revision, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - readStringBinary(server_timezone, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) - readStringBinary(server_display_name, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) - readVarUInt(server_version_patch, *in); + /// Receive hello packet. + UInt64 packet_type = 0; + + /// Prevent read after eof in readVarUInt in case of reset connection + /// (Poco should throw such exception while reading from socket but + /// sometimes it doesn't for unknown reason) + if (in->eof()) + throw Poco::Net::NetException("Connection reset by peer"); + + readVarUInt(packet_type, *in); + if (packet_type == Protocol::Server::Hello) + { + readStringBinary(server_name, *in); + readVarUInt(server_version_major, *in); + readVarUInt(server_version_minor, *in); + readVarUInt(server_revision, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + readStringBinary(server_timezone, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) + readStringBinary(server_display_name, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) + readVarUInt(server_version_patch, *in); + else + server_version_patch = server_revision; + } + else if (packet_type == Protocol::Server::Exception) + receiveException()->rethrow(); else - server_version_patch = server_revision; + { + /// Close connection, to not stay in unsynchronised state. + disconnect(); + throwUnexpectedPacket(packet_type, "Hello or Exception"); + } } - else if (packet_type == Protocol::Server::Exception) - receiveException()->rethrow(); - else + catch (Poco::Net::NetException & e) { - /// Close connection, to not stay in unsynchronised state. disconnect(); - throwUnexpectedPacket(packet_type, "Hello or Exception"); + + /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::NETWORK_ERROR); + } + catch (Poco::TimeoutException & e) + { + disconnect(); + + /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. + throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); } } diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index 15344b3b18b..a027f7a186b 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -358,19 +358,6 @@ void ConnectionEstablisher::resetResult() } } -void ConnectionEstablisher::processFail(bool add_description) -{ - if (action_before_disconnect) - action_before_disconnect(socket_fd); - - fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); - if (add_description) - fail_message += " (" + result.entry->getDescription() + ")"; - resetResult(); - socket_fd = -1; - stage = Stage::FAILED; -} - void ConnectionEstablisher::run() { try @@ -463,20 +450,19 @@ void ConnectionEstablisher::run() stage = Stage::FINISHED; } - catch (Poco::Net::NetException &) - { - processFail(true); - } - catch (Poco::TimeoutException &) - { - processFail(true); - } catch (const Exception & e) { - if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT + && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) throw; - processFail(false); + if (action_before_disconnect) + action_before_disconnect(socket_fd); + + fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); + resetResult(); + socket_fd = -1; + stage = Stage::FAILED; } } diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 44b06e871ec..b25eee6e33d 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -65,9 +65,6 @@ public: /// disconnect. It may be useful for removing file descriptor from epoll. void setActionBeforeDisconnect(std::function action) { action_before_disconnect = action; } - /// Process fail connection. - void processFail(bool add_description = false); - IConnectionPool * pool; const ConnectionTimeouts * timeouts; std::string fail_message; diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 32a91af6179..ad00c60b302 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -26,15 +26,20 @@ HedgedConnections::HedgedConnections( { std::vector connections = hedged_connections_factory.getManyConnections(pool_mode); - ReplicaState replica; + if (connections.empty()) + return; + for (size_t i = 0; i != connections.size(); ++i) { + ReplicaState replica; replica.connection = connections[i]; replica.connection->setThrottler(throttler_); - int socket_fd = replica.connection->getSocket()->impl()->sockfd(); - epoll.add(socket_fd); - fd_to_replica_location[socket_fd] = ReplicaLocation{i, 0}; - offset_states.push_back(OffsetState{{replica}, 1, false}); + replica.epoll.add(replica.connection->getSocket()->impl()->sockfd()); + epoll.add(replica.epoll.getFileDescriptor()); + fd_to_replica_location[replica.epoll.getFileDescriptor()] = ReplicaLocation{i, 0}; + offset_states.emplace_back(); + offset_states[i].replicas.emplace_back(std::move(replica)); + offset_states[i].active_connection_count = 1; } active_connection_count = connections.size(); @@ -148,12 +153,12 @@ void HedgedConnections::sendQuery( if (offset_states.size() > 1) { modified_settings.parallel_replicas_count = offset_states.size(); - modified_settings.parallel_replica_offset = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()].offset; + modified_settings.parallel_replica_offset = fd_to_replica_location[replica.epoll.getFileDescriptor()].offset; } replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, replica); + replica.receive_timeout.setRelative(timeouts.receive_timeout); + replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout); }; for (auto & offset_status : offset_states) @@ -234,7 +239,8 @@ Packet HedgedConnections::drain() while (!epoll.empty()) { - Packet packet = receivePacketImpl(); + ReplicaLocation location = getReadyReplicaLocation(); + Packet packet = receivePacketFromReplica(location); switch (packet.type) { case Protocol::Server::PartUUIDs: @@ -273,64 +279,110 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) if (epoll.empty()) throw Exception("No pending events in epoll.", ErrorCodes::LOGICAL_ERROR); - return receivePacketImpl(std::move(async_callback)); + ReplicaLocation location = getReadyReplicaLocation(async_callback); + return receivePacketFromReplica(location, std::move(async_callback)); } -Packet HedgedConnections::receivePacketImpl(AsyncCallback async_callback) +HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback) { int event_fd; - Packet packet; - bool finish = false; - while (!finish) + while (true) { + /// Check connections for pending data. + ReplicaLocation location; + if (checkPendingData(location)) + return location; + + /// Get ready file descriptor from epoll and process it. event_fd = getReadyFileDescriptor(async_callback); - if (fd_to_replica_location.contains(event_fd)) + if (event_fd == hedged_connections_factory.getFileDescriptor()) { - ReplicaLocation location = fd_to_replica_location[event_fd]; - packet = receivePacketFromReplica(location, async_callback); - finish = true; - } - else if (timeout_fd_to_replica_location.contains(event_fd)) - { - ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; - processTimeoutEvent(location, offset_states[location.offset].replicas[location.index].active_timeouts[event_fd]); - } - else if (event_fd == hedged_connections_factory.getFileDescriptor()) tryGetNewReplica(false); - else - throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); - } + continue; + } - return packet; + if (!fd_to_replica_location.contains(event_fd)) + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); + + location = fd_to_replica_location[event_fd]; + + /// Read all events from replica epoll. + /// If socket is ready and timeout is alarmed simultaneously, skip timeout. + bool is_socket_ready = false; + bool is_change_replica_timeout_alarmed = false; + bool is_receive_timeout_alarmed = false; + + epoll_event events[3]; + events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; + ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; + size_t ready_count = replica_state.epoll.getManyReady(3, events, true); + + for (size_t i = 0; i != ready_count; ++i) + { + if (events[i].data.fd == replica_state.connection->getSocket()->impl()->sockfd()) + is_socket_ready = true; + if (events[i].data.fd == replica_state.change_replica_timeout.getDescriptor()) + is_change_replica_timeout_alarmed = true; + if (events[i].data.fd == replica_state.receive_timeout.getDescriptor()) + is_receive_timeout_alarmed = true; + } + + if (is_socket_ready) + return location; + + /// We reach this point only if there is an alarmed timeout. + + if (is_change_replica_timeout_alarmed) + { + replica_state.change_replica_timeout.reset(); + offsets_queue.push(location.offset); + tryGetNewReplica(true); + } + if (is_receive_timeout_alarmed) + { + finishProcessReplica(replica_state, true); + + /// Check if there is no more active connections with the same offset and there is no new replica in process. + if (offset_states[location.offset].active_connection_count == 0 && !next_replica_in_process) + throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); + } + } }; int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { - for (auto & [fd, location] : fd_to_replica_location) - { - ReplicaState & replica = offset_states[location.offset].replicas[location.index]; - if (replica.connection->hasReadPendingData()) - return replica.connection->getSocket()->impl()->sockfd(); - } - epoll_event event; event.data.fd = -1; epoll.getManyReady(1, &event, true, std::move(async_callback)); return event.data.fd; } -Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback) +bool HedgedConnections::checkPendingData(ReplicaLocation & location_out) +{ + for (auto & [fd, location] : fd_to_replica_location) + { + if (offset_states[location.offset].replicas[location.index].connection->hasReadPendingData()) + { + location_out = location; + return true; + } + } + + return false; +} + +Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & replica_location, AsyncCallback async_callback) { ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; - removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + replica.receive_timeout.reset(); Packet packet = replica.connection->receivePacket(std::move(async_callback)); switch (packet.type) { case Protocol::Server::Data: if (!offset_states[replica_location.offset].first_packet_of_data_received) processReceivedFirstDataPacket(replica_location); - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + replica.receive_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_timeout); break; case Protocol::Server::PartUUIDs: case Protocol::Server::Progress: @@ -338,7 +390,7 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_loc case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica); + replica.receive_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_timeout); break; case Protocol::Server::EndOfStream: @@ -354,12 +406,12 @@ Packet HedgedConnections::receivePacketFromReplica(ReplicaLocation & replica_loc return packet; } -void HedgedConnections::processReceivedFirstDataPacket(ReplicaLocation & replica_location) +void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & replica_location) { /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. OffsetState & offset_state = offset_states[replica_location.offset]; - removeTimeoutFromReplica(ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT, offset_state.replicas[replica_location.index]); + offset_state.replicas[replica_location.index].change_replica_timeout.reset(); ++offsets_with_received_first_data_packet; offset_state.first_packet_of_data_received = true; @@ -384,28 +436,6 @@ void HedgedConnections::processReceivedFirstDataPacket(ReplicaLocation & replica } } -void HedgedConnections::processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor) -{ - ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; - epoll.remove(timeout_descriptor->timer.getDescriptor()); - replica.active_timeouts.erase(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica_location.erase(timeout_descriptor->timer.getDescriptor()); - - if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) - { - finishProcessReplica(replica, true); - - /// Check if there is no active connections with the same offset and there is no new replica in process. - if (offset_states[replica_location.offset].active_connection_count == 0 && !next_replica_in_process) - throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); - } - else if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT) - { - offsets_queue.push(replica_location.offset); - tryGetNewReplica(true); - } -} - void HedgedConnections::tryGetNewReplica(bool start_new_connection) { Connection * connection = nullptr; @@ -414,24 +444,22 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) /// Skip replicas that doesn't support two-level aggregation if we didn't disable it in sendQuery. while (state == HedgedConnectionsFactory::State::READY && !disable_two_level_aggregation && connection->getServerRevision(hedged_connections_factory.getConnectionTimeouts()) - < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) + < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) state = hedged_connections_factory.getNextConnection(true, false, connection); if (state == HedgedConnectionsFactory::State::READY) { size_t offset = offsets_queue.front(); offsets_queue.pop(); - size_t index = offset_states[offset].replicas.size(); ReplicaState replica; replica.connection = connection; - int socket_fd = replica.connection->getSocket()->impl()->sockfd(); - epoll.add(socket_fd); - fd_to_replica_location[socket_fd] = ReplicaLocation{offset, index}; - ++offset_states[offset].active_connection_count; + replica.epoll.add(replica.connection->getSocket()->impl()->sockfd()); + epoll.add(replica.epoll.getFileDescriptor()); + fd_to_replica_location[replica.epoll.getFileDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size()}; ++active_connection_count; pipeline_for_new_replicas.run(replica); - offset_states[offset].replicas.push_back(replica); + offset_states[offset].replicas.push_back(std::move(replica)); } else if (state == HedgedConnectionsFactory::State::NOT_READY && !next_replica_in_process) { @@ -460,11 +488,9 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { - removeTimeoutsFromReplica(replica); - int socket_fd = replica.connection->getSocket()->impl()->sockfd(); - epoll.remove(socket_fd); - --offset_states[fd_to_replica_location[socket_fd].offset].active_connection_count; - fd_to_replica_location.erase(socket_fd); + epoll.remove(replica.epoll.getFileDescriptor()); + --offset_states[fd_to_replica_location[replica.epoll.getFileDescriptor()].offset].active_connection_count; + fd_to_replica_location.erase(replica.epoll.getFileDescriptor()); --active_connection_count; if (disconnect) @@ -472,38 +498,5 @@ void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool discon replica.connection = nullptr; } -void HedgedConnections::addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica) -{ - ConnectionTimeoutDescriptorPtr timeout_descriptor - = createConnectionTimeoutDescriptor(type, hedged_connections_factory.getConnectionTimeouts()); - epoll.add(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica_location[timeout_descriptor->timer.getDescriptor()] - = fd_to_replica_location[replica.connection->getSocket()->impl()->sockfd()]; - replica.active_timeouts[timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); -} - -void HedgedConnections::removeTimeoutsFromReplica(ReplicaState & replica) -{ - for (auto & [fd, _] : replica.active_timeouts) - { - epoll.remove(fd); - timeout_fd_to_replica_location.erase(fd); - } - replica.active_timeouts.clear(); -} - -void HedgedConnections::removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica) -{ - auto it = std::find_if( - replica.active_timeouts.begin(), replica.active_timeouts.end(), [type](auto & value) { return value.second->type == type; }); - - if (it != replica.active_timeouts.end()) - { - epoll.remove(it->first); - timeout_fd_to_replica_location.erase(it->first); - replica.active_timeouts.erase(it); - } -} - } #endif diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 4e3b6a67169..249c41a7a06 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -20,21 +20,40 @@ class HedgedConnections : public IConnections public: struct ReplicaState { - Connection * connection = nullptr; - std::unordered_map active_timeouts; - }; + ReplicaState() + { + epoll.add(receive_timeout.getDescriptor()); + epoll.add(change_replica_timeout.getDescriptor()); + } - struct ReplicaLocation - { - size_t offset; - size_t index; + Connection * connection = nullptr; + TimerDescriptor receive_timeout; + TimerDescriptor change_replica_timeout; + /// We store socket and timeout descriptors in epoll + /// and use it's fd outside. + Epoll epoll; }; struct OffsetState { + /// Replicas with the same offset. std::vector replicas; - size_t active_connection_count; - bool first_packet_of_data_received; + /// An amount of active replicas, when first_packet_of_data_received is true, + /// active_connection_count is always <= 1 (because we stop working with + /// other replicas when we receive first data packet from one of them) + size_t active_connection_count = 0; + bool first_packet_of_data_received = false; + }; + + /// We process events in epoll, so we need to determine replica by it's + /// file descriptor. We store map fd -> replica location. To determine + /// where replica is, we need a replica offset + /// (the same as parallel_replica_offset), and index, which is needed because + /// we can have many replicas with same offset (when receive_data_timeout has expired). + struct ReplicaLocation + { + size_t offset; + size_t index; }; HedgedConnections(const ConnectionPoolWithFailoverPtr & pool_, @@ -75,7 +94,11 @@ public: bool hasActiveConnections() const override { return active_connection_count > 0; } private: - /// We will save actions with replicas in pipeline to perform them on the new replicas. + /// If we don't receive data from replica for receive_data_timeout, we are trying + /// to get new replica and send query to it. Beside sending query, there are some + /// additional actions like sendScalarsData or sendExternalTablesData and we need + /// to perform these actions in the same order on the new replica. So, we will + /// save actions with replicas in pipeline to perform them on the new replicas. class Pipeline { public: @@ -86,13 +109,11 @@ private: std::vector> pipeline; }; - Packet receivePacketFromReplica(ReplicaLocation & replica_location, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(const ReplicaLocation & replica_location, AsyncCallback async_callback = {}); - Packet receivePacketImpl(AsyncCallback async_callback = {}); + ReplicaLocation getReadyReplicaLocation(AsyncCallback async_callback = {}); - void processReceivedFirstDataPacket(ReplicaLocation & replica_location); - - void processTimeoutEvent(ReplicaLocation & replica_location, ConnectionTimeoutDescriptorPtr timeout_descriptor); + void processReceivedFirstDataPacket(const ReplicaLocation & replica_location); void tryGetNewReplica(bool start_new_connection); @@ -100,12 +121,7 @@ private: int getReadyFileDescriptor(AsyncCallback async_callback = {}); - void addTimeoutToReplica(ConnectionTimeoutType type, ReplicaState & replica); - - void removeTimeoutsFromReplica(ReplicaState & replica); - - void removeTimeoutFromReplica(ConnectionTimeoutType type, ReplicaState & replica); - + bool checkPendingData(ReplicaLocation & location_out); HedgedConnectionsFactory hedged_connections_factory; @@ -116,8 +132,6 @@ private: /// Map socket file descriptor to replica location (it's offset and index in OffsetState.replicas). std::unordered_map fd_to_replica_location; - /// Map timeout file descriptor to replica location (it's offset and index in OffsetState.replicas). - std::unordered_map timeout_fd_to_replica_location; /// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from /// the replica, we push it's offset to this queue and start trying to get diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 12362635904..c4a10379985 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -22,9 +22,10 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( { shuffled_pools = pool->getShuffledPools(settings); for (size_t i = 0; i != shuffled_pools.size(); ++i) - connection_establishers.emplace_back(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); - - replicas_timeouts.resize(shuffled_pools.size()); + { + ConnectionEstablisher establisher(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); + replicas.emplace_back(std::move(establisher)); + } max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); @@ -62,15 +63,12 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode Connection * connection = nullptr; /// Try to start establishing connections with max_entries replicas. - int index; for (size_t i = 0; i != max_entries; ++i) { - index = getNextIndex(); + int index = startEstablishingNewConnection(connection); if (index == -1) break; - - auto state = startEstablishingConnection(index, connection); - if (state == State::READY) + if (replicas[index].is_ready) connections.push_back(connection); } @@ -104,27 +102,17 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out) { - int index = -1; - if (start_new_connection) - index = getNextIndex(); - - while (index != -1 || !epoll.empty()) { - if (index != -1) - { - State state = startEstablishingConnection(index, connection_out); - if (state == State::READY) - return state; - } - - State state = processEpollEvents(blocking, connection_out); - if (state != State::EMPTY) - return state; - - index = getNextIndex(); + int index = startEstablishingNewConnection(connection_out); + if (index != -1 && replicas[index].is_ready) + return State::READY; } + State state = processEpollEvents(blocking, connection_out); + if (state != State::CANNOT_CHOOSE) + return state; + /// We reach this point only if there was no free up to date replica. /// We will try to use usable replica. @@ -139,9 +127,9 @@ void HedgedConnectionsFactory::stopChoosingReplicas() { for (auto & [fd, replica_index] : fd_to_replica_index) { - removeTimeoutsFromReplica(replica_index); + resetReplicaTimeouts(replica_index); epoll.remove(fd); - connection_establishers[replica_index].reset(); + replicas[replica_index].connection_establisher.reset(); } fd_to_replica_index.clear(); @@ -150,7 +138,7 @@ void HedgedConnectionsFactory::stopChoosingReplicas() int HedgedConnectionsFactory::getNextIndex() { /// Check if there is no free replica. - if (entries_count + indexes_in_process.size() + failed_pools_count >= shuffled_pools.size()) + if (entries_count + replicas_in_process_count + failed_pools_count >= shuffled_pools.size()) return -1; /// Check if it's the first time. @@ -167,8 +155,8 @@ int HedgedConnectionsFactory::getNextIndex() next_index = (next_index + 1) % shuffled_pools.size(); /// Check if we can try this replica. - if (indexes_in_process.find(next_index) == indexes_in_process.end() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) - && connection_establishers[next_index].stage != ConnectionEstablisher::Stage::FINISHED) + if (!replicas[next_index].is_in_process && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) + && replicas[next_index].connection_establisher.stage != ConnectionEstablisher::Stage::FINISHED) finish = true; /// If we made a complete round, there is no replica to connect. @@ -180,83 +168,93 @@ int HedgedConnectionsFactory::getNextIndex() return next_index; } -HedgedConnectionsFactory::State HedgedConnectionsFactory::startEstablishingConnection(int replica_index, Connection *& connection_out) +int HedgedConnectionsFactory::startEstablishingNewConnection(Connection *& connection_out) { - State state; + int index; do { - ConnectionEstablisher & connection_establisher = connection_establishers[replica_index]; + index = getNextIndex(); + if (index == -1) + return -1; - state = State::NOT_READY; - indexes_in_process.insert(replica_index); + ReplicaStatus & replica = replicas[index]; - connection_establisher.reset(); - connection_establisher.run(); + ++replicas_in_process_count; + replica.is_in_process = true; + replica.connection_establisher.reset(); + replica.connection_establisher.run(); - state = processConnectionEstablisherStage(replica_index); + processConnectionEstablisherStage(index); - if (state == State::NOT_READY) + if (replica.is_in_process) { - epoll.add(connection_establisher.socket_fd); - fd_to_replica_index[connection_establisher.socket_fd] = replica_index; - connection_establisher.setActionBeforeDisconnect([&](int fd) - { - epoll.remove(fd); - fd_to_replica_index.erase(fd); - }); - addTimeouts(replica_index); + replica.epoll.add(replica.connection_establisher.socket_fd); + replica.connection_establisher.setActionBeforeDisconnect([&](int fd){ replica.epoll.remove(fd); }); + addTimeouts(index); + epoll.add(replica.epoll.getFileDescriptor()); + fd_to_replica_index[replica.epoll.getFileDescriptor()] = index; } } - while (state == State::EMPTY && (replica_index = getNextIndex()) != -1); + while (!replicas[index].is_ready && !replicas[index].is_in_process); - if (state == State::READY) - connection_out = &*connection_establishers[replica_index].result.entry; + if (replicas[index].is_ready) + connection_out = &*replicas[index].connection_establisher.result.entry; - return state; + return index; } -HedgedConnectionsFactory::State HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_index, bool remove_from_epoll) +void HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_index, bool remove_from_epoll) { - ConnectionEstablisher & connection_establisher = connection_establishers[replica_index]; + ReplicaStatus & replica = replicas[replica_index]; - if (connection_establisher.stage == ConnectionEstablisher::Stage::FINISHED) + if (replica.connection_establisher.stage == ConnectionEstablisher::Stage::FINISHED) { - indexes_in_process.erase(replica_index); + replica.is_in_process = false; + --replicas_in_process_count; ++entries_count; if (remove_from_epoll) { - epoll.remove(connection_establisher.socket_fd); - fd_to_replica_index.erase(connection_establisher.socket_fd); + epoll.remove(replica.epoll.getFileDescriptor()); + fd_to_replica_index.erase(replica.epoll.getFileDescriptor()); } - if (connection_establisher.result.is_usable) + if (replica.connection_establisher.result.is_usable) { ++usable_count; - if (connection_establisher.result.is_up_to_date) + if (replica.connection_establisher.result.is_up_to_date) { - ready_indexes.insert(replica_index); - return State::READY; + ++ready_replicas_count; + replica.is_ready = true; + return; } } - - /// This replica is not up to date, we will try to find up to date. - return State::EMPTY; + else + { + std::string & fail_message = replica.connection_establisher.fail_message; + if (!fail_message.empty()) + fail_messages += fail_message + "\n"; + } } - else if (connection_establisher.stage == ConnectionEstablisher::Stage::FAILED) - { - processFailedConnection(replica_index); - return State::EMPTY; - } - - return State::NOT_READY; + else if (replica.connection_establisher.stage == ConnectionEstablisher::Stage::FAILED) + processFailedConnection(replica_index, remove_from_epoll); } -void HedgedConnectionsFactory::processFailedConnection(int replica_index) +void HedgedConnectionsFactory::processFailedConnection(int replica_index, bool remove_from_epoll) { + if (remove_from_epoll) + { + epoll.remove(replicas[replica_index].epoll.getFileDescriptor()); + fd_to_replica_index.erase(replicas[replica_index].epoll.getFileDescriptor()); + } + + std::string & fail_message = replicas[replica_index].connection_establisher.fail_message; + if (!fail_message.empty()) + fail_messages += fail_message + "\n"; + ShuffledPool & shuffled_pool = shuffled_pools[replica_index]; LOG_WARNING( - log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), connection_establishers[replica_index].fail_message); + log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message); ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); @@ -267,139 +265,157 @@ void HedgedConnectionsFactory::processFailedConnection(int replica_index) ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); } - std::string & fail_message = connection_establishers[replica_index].fail_message; - if (!fail_message.empty()) - fail_messages += fail_message + "\n"; - - indexes_in_process.erase(replica_index); + --replicas_in_process_count; + replicas[replica_index].is_in_process = false; } void HedgedConnectionsFactory::addTimeouts(int replica_index) { - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TIMEOUT, replica_index); - - auto stage = connection_establishers[replica_index].stage; + auto stage = replicas[replica_index].connection_establisher.stage; if (stage == ConnectionEstablisher::Stage::RECEIVE_HELLO) - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT, replica_index); - else if (stage == ConnectionEstablisher::Stage::RECEIVE_TABLES_STATUS) - addTimeoutToReplica(ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT, replica_index); -} - -void HedgedConnectionsFactory::addTimeoutToReplica(ConnectionTimeoutType type, int replica_index) -{ - ConnectionTimeoutDescriptorPtr timeout_descriptor = createConnectionTimeoutDescriptor(type, timeouts); - epoll.add(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica_index[timeout_descriptor->timer.getDescriptor()] = replica_index; - replicas_timeouts[replica_index][timeout_descriptor->timer.getDescriptor()] = std::move(timeout_descriptor); -} - -void HedgedConnectionsFactory::removeTimeoutsFromReplica(int replica_index) -{ - for (auto & [fd, _] : replicas_timeouts[replica_index]) { - epoll.remove(fd); - timeout_fd_to_replica_index.erase(fd); + replicas[replica_index].receive_timeout.setRelative(timeouts.receive_timeout); + replicas[replica_index].change_replica_timeout.setRelative(timeouts.receive_hello_timeout); } - replicas_timeouts[replica_index].clear(); + else if (stage == ConnectionEstablisher::Stage::RECEIVE_TABLES_STATUS) + { + replicas[replica_index].receive_timeout.setRelative(Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0)); + replicas[replica_index].change_replica_timeout.setRelative(timeouts.receive_tables_status_timeout); + } +} + +void HedgedConnectionsFactory::resetReplicaTimeouts(int replica_index) +{ + replicas[replica_index].receive_timeout.reset(); + replicas[replica_index].change_replica_timeout.reset(); } HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out) { int event_fd; - while (true) + while (!epoll.empty()) { + /// Firstly, check connections for pending data. + int replica_index = checkPendingData(); + if (replica_index != -1) + { + processSocketEvent(replica_index, connection_out); + /// Return only if replica is ready. + if (replicas[replica_index].is_ready) + return State::READY; + + continue; + } + + /// Get ready descriptor fro epoll. event_fd = getReadyFileDescriptor(blocking); /// Check if there is no events. if (event_fd == -1) return State::NOT_READY; - if (fd_to_replica_index.find(event_fd) != fd_to_replica_index.end()) + if (!fd_to_replica_index.contains(event_fd)) + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); + + replica_index = fd_to_replica_index[event_fd]; + + /// Read all events from replica epoll. + /// If socket is ready and timeout is alarmed simultaneously, skip timeout. + bool is_socket_ready = false; + bool is_receive_timeout_alarmed = false; + bool is_change_replica_timeout_alarmed = false; + + epoll_event events[3]; + events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; + size_t ready_count = replicas[replica_index].epoll.getManyReady(3, events, true); + for (size_t i = 0; i != ready_count; ++i) { - int replica_index = fd_to_replica_index[event_fd]; - State state = processReplicaEvent(replica_index, connection_out); - /// Return only if replica is ready or we need to try next replica. - if (state != State::NOT_READY) - return state; + if (events[i].data.fd == replicas[replica_index].connection_establisher.socket_fd) + is_socket_ready = true; + if (events[i].data.fd == replicas[replica_index].receive_timeout.getDescriptor()) + is_receive_timeout_alarmed = true; + if (events[i].data.fd == replicas[replica_index].change_replica_timeout.getDescriptor()) + is_change_replica_timeout_alarmed = true; } - else if (timeout_fd_to_replica_index.find(event_fd) != timeout_fd_to_replica_index.end()) + + if (is_socket_ready) { - int replica_index = timeout_fd_to_replica_index[event_fd]; - /// Process received timeout. If retured values is true, we need to try new replica. - if (processTimeoutEvent(replica_index, replicas_timeouts[replica_index][event_fd])) - return State::EMPTY; + processSocketEvent(replica_index, connection_out); + /// Return only if replica is ready. + if (replicas[replica_index].is_ready) + return State::READY; + if (replicas[replica_index].is_in_process) + continue; } else - throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); + { + if (is_receive_timeout_alarmed) + processReceiveTimeout(replica_index); + + if (is_change_replica_timeout_alarmed) + replicas[replica_index].change_replica_timeout.reset(); + } + + /// We reach this point only if we need to start new connection. + replica_index = startEstablishingNewConnection(connection_out); + /// Return only if replica is ready. + if (replica_index != -1 && replicas[replica_index].is_ready) + return State::READY; } + + return State::CANNOT_CHOOSE; } int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) { - for (auto & [fd, replica_index] : fd_to_replica_index) - if (connection_establishers[replica_index].result.entry->hasReadPendingData()) - return connection_establishers[replica_index].socket_fd; - epoll_event event; event.data.fd = -1; epoll.getManyReady(1, &event, blocking); return event.data.fd; } -HedgedConnectionsFactory::State HedgedConnectionsFactory::processReplicaEvent(int replica_index, Connection *& connection_out) +int HedgedConnectionsFactory::checkPendingData() { - removeTimeoutsFromReplica(replica_index); - connection_establishers[replica_index].run(); - State state = processConnectionEstablisherStage(replica_index, true); - if (state == State::NOT_READY) - addTimeouts(replica_index); - if (state == State::READY) - connection_out = &*connection_establishers[replica_index].result.entry; - return state; + for (auto & [fd, replica_index] : fd_to_replica_index) + if (replicas[replica_index].connection_establisher.result.entry->hasReadPendingData()) + return replica_index; + + return -1; } -bool HedgedConnectionsFactory::processTimeoutEvent(int replica_index, ConnectionTimeoutDescriptorPtr timeout_descriptor) +void HedgedConnectionsFactory::processSocketEvent(int replica_index, Connection *& connection_out) { - epoll.remove(timeout_descriptor->timer.getDescriptor()); - replicas_timeouts[replica_index].erase(timeout_descriptor->timer.getDescriptor()); - timeout_fd_to_replica_index[timeout_descriptor->timer.getDescriptor()]; + resetReplicaTimeouts(replica_index); + replicas[replica_index].connection_establisher.run(); + processConnectionEstablisherStage(replica_index, true); + if (replicas[replica_index].is_in_process) + addTimeouts(replica_index); + if (replicas[replica_index].is_ready) + connection_out = &*replicas[replica_index].connection_establisher.result.entry; +} - if (timeout_descriptor->type == ConnectionTimeoutType::RECEIVE_TIMEOUT) - { - removeTimeoutsFromReplica(replica_index); - int fd = connection_establishers[replica_index].socket_fd; - epoll.remove(fd); - fd_to_replica_index.erase(fd); +void HedgedConnectionsFactory::processReceiveTimeout(int replica_index) +{ + resetReplicaTimeouts(replica_index); + ReplicaStatus & replica = replicas[replica_index]; - ConnectionEstablisher & connection_establisher = connection_establishers[replica_index]; - connection_establisher.fail_message = "Receive timeout expired (" + connection_establisher.result.entry->getDescription() + ")"; - connection_establisher.resetResult(); - connection_establisher.stage = ConnectionEstablisher::Stage::FAILED; - processFailedConnection(replica_index); - return true; - } - - /// Return true if we can try to start one more connection. - return entries_count + indexes_in_process.size() + failed_pools_count < shuffled_pools.size(); + replica.connection_establisher.fail_message = + "Code: 209, e.displayText() = DB::NetException: Timeout exceeded while reading from socket (" + replica.connection_establisher.result.entry->getDescription() + ")"; + replica.connection_establisher.resetResult(); + replica.connection_establisher.stage = ConnectionEstablisher::Stage::FAILED; + processFailedConnection(replica_index, true); } HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(Connection *& connection_out) { - std::vector indexes(connection_establishers.size()); - for (size_t i = 0; i != indexes.size(); ++i) - indexes[i] = i; - - /// Remove unusable, failed replicas and replicas that are ready or in process. - indexes.erase( - std::remove_if( - indexes.begin(), - indexes.end(), - [&](int i) - { - return connection_establishers[i].result.entry.isNull() || !connection_establishers[i].result.is_usable || - indexes_in_process.find(i) != indexes_in_process.end() || ready_indexes.find(i) != ready_indexes.end(); - }), - indexes.end()); + std::vector indexes; + for (size_t i = 0; i != replicas.size(); ++i) + { + /// Don't add unusable, failed replicas and replicas that are ready or in process. + if (!replicas[i].connection_establisher.result.entry.isNull() && replicas[i].connection_establisher.result.is_usable && + !replicas[i].is_in_process && !replicas[i].is_ready) + indexes.push_back(i); + } if (indexes.empty()) return State::CANNOT_CHOOSE; @@ -410,38 +426,14 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C indexes.end(), [&](size_t lhs, size_t rhs) { - return connection_establishers[lhs].result.staleness < connection_establishers[rhs].result.staleness; + return replicas[lhs].connection_establisher.result.staleness < replicas[rhs].connection_establisher.result.staleness; }); - ready_indexes.insert(indexes[0]); - connection_out = &*connection_establishers[indexes[0]].result.entry; + ++ready_replicas_count; + replicas[indexes[0]].is_ready = true; + connection_out = &*replicas[indexes[0]].connection_establisher.result.entry; return State::READY; } -ConnectionTimeoutDescriptorPtr createConnectionTimeoutDescriptor(ConnectionTimeoutType type, const ConnectionTimeouts & timeouts) -{ - Poco::Timespan timeout; - switch (type) - { - case ConnectionTimeoutType::RECEIVE_HELLO_TIMEOUT: - timeout = timeouts.receive_hello_timeout; - break; - case ConnectionTimeoutType::RECEIVE_TABLES_STATUS_TIMEOUT: - timeout = timeouts.receive_tables_status_timeout; - break; - case ConnectionTimeoutType::RECEIVE_DATA_TIMEOUT: - timeout = timeouts.receive_data_timeout; - break; - case ConnectionTimeoutType::RECEIVE_TIMEOUT: - timeout = timeouts.receive_timeout; - break; - } - - ConnectionTimeoutDescriptorPtr timeout_descriptor = std::make_shared(); - timeout_descriptor->type = type; - timeout_descriptor->timer.setRelative(timeout); - return timeout_descriptor; -} - } #endif diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 45a03e212c0..b821a9c925e 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -12,21 +12,6 @@ namespace DB { -enum class ConnectionTimeoutType -{ - RECEIVE_HELLO_TIMEOUT, - RECEIVE_TABLES_STATUS_TIMEOUT, - RECEIVE_DATA_TIMEOUT, - RECEIVE_TIMEOUT, -}; - -struct ConnectionTimeoutDescriptor -{ - ConnectionTimeoutType type; - TimerDescriptor timer; -}; - -using ConnectionTimeoutDescriptorPtr = std::shared_ptr; using TimerDescriptorPtr = std::shared_ptr; /** Class for establishing hedged connections with replicas. @@ -40,12 +25,27 @@ class HedgedConnectionsFactory public: using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool; + struct ReplicaStatus + { + ReplicaStatus(const ConnectionEstablisher & establisher) : connection_establisher(establisher) + { + epoll.add(receive_timeout.getDescriptor()); + epoll.add(change_replica_timeout.getDescriptor()); + } + + ConnectionEstablisher connection_establisher; + TimerDescriptor receive_timeout; + TimerDescriptor change_replica_timeout; + bool is_ready = false; + bool is_in_process = false; + Epoll epoll; + }; + enum class State { - EMPTY = 0, - READY = 1, - NOT_READY = 2, - CANNOT_CHOOSE = 3, + READY, + NOT_READY, + CANNOT_CHOOSE, }; HedgedConnectionsFactory(const ConnectionPoolWithFailoverPtr & pool_, @@ -64,7 +64,7 @@ public: State getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out); /// Check if we can try to produce new READY replica. - bool canGetNewConnection() const { return ready_indexes.size() + failed_pools_count < shuffled_pools.size(); } + bool canGetNewConnection() const { return ready_replicas_count + failed_pools_count < shuffled_pools.size(); } /// Stop working with all replicas that are not READY. void stopChoosingReplicas(); @@ -78,9 +78,11 @@ public: ~HedgedConnectionsFactory(); private: - State startEstablishingConnection(int index, Connection *& connection_out); + /// Try to start establishing connection to the new replica. Return + /// the index of the new replica or -1 if cannot start new connection. + int startEstablishingNewConnection(Connection *& connection_out); - State processConnectionEstablisherStage(int replica_index, bool remove_from_epoll = false); + void processConnectionEstablisherStage(int replica_index, bool remove_from_epoll = false); /// Find an index of the next free replica to start connection. /// Return -1 if there is no free replica. @@ -88,20 +90,20 @@ private: int getReadyFileDescriptor(bool blocking); + int checkPendingData(); + void addTimeouts(int replica_index); - void addTimeoutToReplica(ConnectionTimeoutType type, int replica_index); + void resetReplicaTimeouts(int replica_index); - void removeTimeoutsFromReplica(int replica_index); + void processFailedConnection(int replica_index, bool remove_from_epoll); - void processFailedConnection(int replica_index); + void processSocketEvent(int replica_index, Connection *& connection_out); - State processReplicaEvent(int replica_index, Connection *& connection_out); - - bool processTimeoutEvent(int replica_index, ConnectionTimeoutDescriptorPtr timeout_descriptor); + void processReceiveTimeout(int replica_index); /// Return NOT_READY state if there is no ready events, READY if replica is ready - /// and EMPTY if we need to try next replica. + /// and CANNOT_CHOOSE if there is no more events in epoll. State processEpollEvents(bool blocking, Connection *& connection_out); State setBestUsableReplica(Connection *& connection_out); @@ -111,20 +113,16 @@ private: const ConnectionTimeouts timeouts; std::shared_ptr table_to_check; - std::vector connection_establishers; + std::vector replicas; std::vector shuffled_pools; - std::vector> replicas_timeouts; - /// Map socket file descriptor to replica index. std::unordered_map fd_to_replica_index; - /// Map timeout file descriptor to replica index. - std::unordered_map timeout_fd_to_replica_index; /// Indexes of replicas, that are in process of connection. - std::unordered_set indexes_in_process; + size_t replicas_in_process_count = 0; /// Indexes of ready replicas. - std::unordered_set ready_indexes; + size_t ready_replicas_count = 0; int last_used_index = -1; bool fallback_to_stale_replicas; @@ -137,8 +135,5 @@ private: size_t max_tries; }; -/// Create ConnectionTimeoutDescriptor with particular type. -ConnectionTimeoutDescriptorPtr createConnectionTimeoutDescriptor(ConnectionTimeoutType type, const ConnectionTimeouts & timeouts); - } #endif diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index da3a4c4c04b..770807d7c9c 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -20,6 +20,11 @@ Epoll::Epoll() : events_count(0) throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR); } +Epoll::Epoll(Epoll && other) : epoll_fd(other.epoll_fd), events_count(other.events_count) +{ + other.epoll_fd = -1; +} + void Epoll::add(int fd, void * ptr) { epoll_event event; @@ -45,6 +50,9 @@ void Epoll::remove(int fd) size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback) const { + if (events_count == 0) + throw Exception("There is no events in epoll", ErrorCodes::LOGICAL_ERROR); + int ready_size = 0; int timeout = blocking && !async_callback ? -1 : 0; do @@ -64,7 +72,8 @@ size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocki Epoll::~Epoll() { - close(epoll_fd); + if (epoll_fd != -1) + close(epoll_fd); } } diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index 3a91199799b..eb168c22a92 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -11,11 +11,18 @@ namespace DB using AsyncCallback = std::function; -class Epoll : boost::noncopyable +class Epoll { public: Epoll(); + Epoll(const Epoll & other) = delete; + Epoll & operator=(const Epoll & other) = delete; + + Epoll(Epoll && other); + + Epoll & operator=(Epoll && other) = default; + /// Add new file descriptor to epoll. If ptr set to nullptr, epoll_event.data.fd = fd, /// otherwise epoll_event.data.ptr = ptr. void add(int fd, void * ptr = nullptr); diff --git a/src/Common/TimerDescriptor.cpp b/src/Common/TimerDescriptor.cpp index e2b8a0ec928..791e6380a89 100644 --- a/src/Common/TimerDescriptor.cpp +++ b/src/Common/TimerDescriptor.cpp @@ -27,10 +27,16 @@ TimerDescriptor::TimerDescriptor(int clockid, int flags) throwFromErrno("Cannot set O_NONBLOCK for timer_fd", ErrorCodes::CANNOT_FCNTL); } +TimerDescriptor::TimerDescriptor(TimerDescriptor && other) : timer_fd(other.timer_fd) +{ + other.timer_fd = -1; +} + TimerDescriptor::~TimerDescriptor() { /// Do not check for result cause cannot throw exception. - close(timer_fd); + if (timer_fd != -1) + close(timer_fd); } void TimerDescriptor::reset() const diff --git a/src/Common/TimerDescriptor.h b/src/Common/TimerDescriptor.h index 6f7003f6980..42f8eb386af 100644 --- a/src/Common/TimerDescriptor.h +++ b/src/Common/TimerDescriptor.h @@ -17,7 +17,7 @@ public: TimerDescriptor(const TimerDescriptor &) = delete; TimerDescriptor & operator=(const TimerDescriptor &) = delete; - TimerDescriptor(TimerDescriptor &&) = default; + TimerDescriptor(TimerDescriptor && other); TimerDescriptor & operator=(TimerDescriptor &&) = default; int getDescriptor() const { return timer_fd; } diff --git a/tests/integration/test_hedged_requests/configs/remote_servers.xml b/tests/integration/test_hedged_requests/configs/remote_servers.xml index 60d2f5891d7..9d753ca2b6a 100644 --- a/tests/integration/test_hedged_requests/configs/remote_servers.xml +++ b/tests/integration/test_hedged_requests/configs/remote_servers.xml @@ -11,6 +11,10 @@ node_2 9000 + + node_3 + 9000 + diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 20602b1af0a..24dc9202880 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -11,60 +11,253 @@ from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) -# Cluster with 1 shard of 2 replicas. node is the instance with Distributed table. -node = cluster.add_instance( - 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) -node_1 = cluster.add_instance('node_1', with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) -node_2 = cluster.add_instance('node_2', with_zookeeper=True) +NODES = {'node_' + str(i): None for i in (1, 2, 3)} +NODES['node'] = None -config = ''' - - - <{setting}>30 - - -''' +sleep_time = 30 @pytest.fixture(scope="module") def started_cluster(): + cluster = ClickHouseCluster(__file__) + NODES['node'] = cluster.add_instance( + 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) + + for name in NODES: + if name != 'node': + NODES[name] = cluster.add_instance(name, with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) + try: cluster.start() - node_1.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_1') ORDER BY id PARTITION BY toYYYYMM(date)''') + for node_id, node in list(NODES.items()): + node.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', '{}') ORDER BY id PARTITION BY toYYYYMM(date)'''.format(node_id)) - node_2.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_2') ORDER BY id PARTITION BY toYYYYMM(date)''') - - node.query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = + NODES['node'].query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')''') + NODES['node'].query("INSERT INTO distributed VALUES (1, '2020-01-01')") + yield cluster finally: cluster.shutdown() -def process_test(sleep_setting_name, receive_timeout_name): - node_1.replace_config('/etc/clickhouse-server/users.d/users1.xml', config.format(setting=sleep_setting_name)) - # Restart node to make new config relevant - node_1.restart_clickhouse(30) - +config = ''' + + + {sleep_before_send_hello} + {sleep_before_send_tables_status} + {sleep_before_send_data} + + +''' + + +def check_query(): # Without hedged requests select query will last more than 30 seconds, # with hedged requests it will last just around 1-2 second start = time.time() - node.query("SELECT * FROM distributed"); + NODES['node'].query("SELECT * FROM distributed"); query_time = time.time() - start - print(query_time) + + assert query_time < 5 -def test_hedged_requests(started_cluster): - node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") +def test_send_hello_sleep(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - process_test("sleep_before_send_hello", "receive_hello_timeout") - process_test("sleep_before_send_tables_status", "receive_tables_status_timeout") - process_test("sleep_before_send_data", "receive_data_timeout") + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_send_hello_sleep2(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_send_table_status_sleep(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_send_table_status_sleep2(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + +def test_send_data(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_send_data2(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_combination1(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_combination2(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_combination3(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_combination4(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + +def test_combination5(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_combination6(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() diff --git a/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml b/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml index 9d753ca2b6a..63767185b34 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/remote_servers.xml @@ -15,6 +15,10 @@ node_3 9000 + + node_4 + 9000 + diff --git a/tests/integration/test_hedged_requests_parallel/configs/users1.xml b/tests/integration/test_hedged_requests_parallel/configs/users1.xml index 5fe444b94ff..2a54396feca 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users1.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users1.xml @@ -2,7 +2,6 @@ - 30 diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 08c5c0d3cd1..95e32a0f3fc 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -11,45 +11,114 @@ from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) -# Cluster with 1 shard of 3 replicas. node is the instance with Distributed table. -node = cluster.add_instance( - 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) -node_1 = cluster.add_instance('node_1', with_zookeeper=True, user_configs=['configs/users1.xml']) -node_2 = cluster.add_instance('node_2', with_zookeeper=True) -node_3 = cluster.add_instance('node_3', with_zookeeper=True) +NODES = {'node_' + str(i): None for i in (1, 2, 3, 4)} +NODES['node'] = None +sleep_time = 30 @pytest.fixture(scope="module") def started_cluster(): + cluster = ClickHouseCluster(__file__) + NODES['node'] = cluster.add_instance( + 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) + + for name in NODES: + if name != 'node': + NODES[name] = cluster.add_instance(name, with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) + try: cluster.start() - node_1.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_1') ORDER BY id PARTITION BY toYYYYMM(date)''') + for node_id, node in list(NODES.items()): + node.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/replicated', '{}') ORDER BY id PARTITION BY toYYYYMM(date)'''.format(node_id)) - node_2.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_2') ORDER BY id PARTITION BY toYYYYMM(date)''') - - node_3.query('''CREATE TABLE replicated (id UInt32, date Date) ENGINE = - ReplicatedMergeTree('/clickhouse/tables/replicated', 'node_3') ORDER BY id PARTITION BY toYYYYMM(date)''') - - node.query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = + NODES['node'].query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')''') + NODES['node'].query("INSERT INTO distributed VALUES (1, '2020-01-01'), (2, '2020-01-02')") + yield cluster finally: cluster.shutdown() -def test_hedged_requests_with_max_parallel_replicas(started_cluster): - node.query("INSERT INTO distributed VALUES (1, '2020-01-01')") - - # Without hedged requests select query will last more 30 seconds, - # with hedged requests it will last just over 2 seconds + +config = ''' + + + {sleep_before_send_hello} + {sleep_before_send_tables_status} + {sleep_before_send_data} + + +''' + + +def check_query(): + # Without hedged requests select query will last more than 30 seconds, + # with hedged requests it will last just around 1-2 second start = time.time() - node.query("SELECT * FROM distributed"); + NODES['node'].query("SELECT * FROM distributed"); query_time = time.time() - start - print(query_time) + + assert query_time < 5 + + +def test_send_hello_sleep(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_send_table_status_sleep(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + + +def test_send_data(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_1'].restart_clickhouse(sleep_time) + NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + + NODES['node_2'].restart_clickhouse(sleep_time) + NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + + check_query() + From eb0847e2b6c9246b4f6d3e9ecf35242b7622f79e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 15 Feb 2021 17:44:05 +0300 Subject: [PATCH 141/510] Style --- src/Client/HedgedConnectionsFactory.cpp | 2 +- src/Common/Epoll.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index c4a10379985..c881c2723df 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -307,7 +307,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(boo continue; } - /// Get ready descriptor fro epoll. + /// Get ready descriptor from epoll. event_fd = getReadyFileDescriptor(blocking); /// Check if there is no events. diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 770807d7c9c..628bb45e796 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -11,6 +11,7 @@ namespace DB namespace ErrorCodes { extern const int EPOLL_ERROR; + extern const int LOGICAL_ERROR; } Epoll::Epoll() : events_count(0) From 3ce33603795d0649ae4fca41ae11aa9918d8b143 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Feb 2021 18:36:25 +0300 Subject: [PATCH 142/510] Some initial code --- src/Coordination/Changelog.cpp | 315 ++++++++++++++++++++++++++ src/Coordination/Changelog.h | 81 +++++++ src/Coordination/InMemoryLogStore.cpp | 8 +- src/Coordination/NuKeeperLogStore.h | 24 ++ 4 files changed, 424 insertions(+), 4 deletions(-) create mode 100644 src/Coordination/Changelog.cpp create mode 100644 src/Coordination/Changelog.h create mode 100644 src/Coordination/NuKeeperLogStore.h diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp new file mode 100644 index 00000000000..a38f039fa40 --- /dev/null +++ b/src/Coordination/Changelog.cpp @@ -0,0 +1,315 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CHECKSUM_DOESNT_MATCH; + extern const int CORRUPTED_DATA; + extern const int UNKNOWN_FORMAT_VERSION; + extern const int LOGICAL_ERROR; + extern const int UNIMPLEMENTED; +} + + +std::string toString(const ChangelogVersion & version) +{ + if (version == ChangelogVersion::V0) + return "V0"; + + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown chagelog version {}", static_cast(version)); +} + +ChangelogVersion fromString(const std::string & version_str) +{ + if (version == "V0") + return ChangelogVersion::V0; + + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown chagelog version {}", version_str); +} + +namespace +{ + +static constexpr auto DEFAULT_PREFIX = "changelog"; + +struct ChangelogName +{ + std::string prefix; + ChangelogVersion version; + size_t from_log_idx; + size_t to_log_idx; +}; + +std::string formatChangelogPath(const std::string & prefix, const ChangelogVersion & version, const ChangelogName & name) +{ + std::filesystem::path path(prefix); + path /= std::filesystem::path(name.prefix + "_" + toString(version) + "_" + std::to_string(name.from_log_idx) + "_" + std::to_string(name.to_log_idx) + ".log"); + return path.to_string(); +} + +ChangelogName getChangelogName(const std::string & path_str) +{ + std::filesystem::path path(path_str); + std:string filename = path.stem(); + Strings filename_parts; + boost::split(filename_parts, filename, boost::is_any_of("_")); + if (filename_parts.size() < 4) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path_str); + + ChangelogName result; + result.prefix = filename_parts[0]; + result.version = fromString(filename_parts[1]); + result.form_log_idx = parse(filename_parts[2]); + result.to_log_idx = parse(filename_parts[3]); + return result; +} + +} + +class ChangelogWriter +{ +public: + ChangelogWriter(const std::string & filepath_, WriteMode mode, size_t start_index_) + : filepath(filepath_) + , plain_buf(filepath, DBMS_DEFAULT_BUFFER_SIZE, mode == WriteMode::Rewrite ? -1 : (O_APPEND | O_CREAT | O_WRONLY)) + , start_index(start_index_) + {} + + + off_t appendRecord(ChangelogRecord && record, bool sync) + { + off_t result = plain_buf.count(); + writeIntBinary(record.header.version, plain_buf); + writeIntBinary(record.header.index, plain_buf); + writeIntBinary(record.header.term, plain_buf); + writeIntBinary(record.header.value_type, plain_buf); + writeIntBinary(record.header.blob_size, plain_buf); + writeIntBinary(record.header.blob_checksum, plain_buf); + + if (record.blob_size != 0) + plain_buf.write(reinterpret_cast(record.blob->data_begin()), record.blob->size()); + + entries_written++; + + if (sync) + plain_buf.sync(); + reeturn result; + } + + void truncateToLength(off_t new_length) + { + flush(); + plain_buf.truncate(new_length); + } + + void flush() + { + plain_buf.sync(); + } + + size_t getEntriesWritten() const + { + return entries_written; + } + + size_t setEntriesWritten(size_t entries_written_) + { + entries_written = entries_written_; + } + + size_t getStartIndex() const + { + return start_index; + } + + void setStartIndex(size_t start_index_) + { + start_index = start_index_; + } + +private: + std::string filepath; + WriteBufferFromFile plain_buf; + size_t entries_written = 0; + size_t start_index; +}; + + +class ChangelogReader +{ +public: + explicit ChangelogReader(const std::string & filepath_) + : filepath(filepath_) + , read_buf(filepath) + {} + + size_t readChangelog(Changelog & changelog, IndexToOffset & index_to_offset) + { + size_t total_read = 0; + while (!read_buf.eof()) + { + total_read += 1; + off_t pos = read_buf.count(); + ChangelogRecord record; + readIntBinary(record.header.version, read_buf); + readIntBinary(record.header.index, read_buf); + readIntBinary(record.header.term, read_buf); + readIntBinary(record.header.value_type, read_buf); + readIntBinary(record.header.blob_size, read_buf); + readIntBinary(record.header.blob_checksum, read_buf); + auto buffer = nuraft::buffer::alloc(record.header.blob_size); + auto buffer_begin = reinterpret_cast(buffer->data_begin()); + read_buf.readStrict(buffer_begin, record.header.blob_size); + index_to_offset[record.header.index] = pos; + + Checksum checksum = CityHash_v1_0_2::CityHash128(buffer_begin, record.header.blob_size); + if (checksum != record.header.blob_checksum) + { + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, + "Checksums doesn't match for log {} (version {}), index {}, blob_size {}", + filepath, record.header.version, record.header.index, record.header.blob_size); + } + + if (changlog.start_idx == 0) + changelog.start_idx = record.header.index; + + if (!changelog.try_emplace(record.header.index, buffer).second) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filename); + } + return total_read; + } +private: + std::string filepath; + ReadBufferFromFile read_buf; +}; + +ChangelogOnDiskHelper::ChangelogOnDiskHelper(const std::string & changelogs_dir, size_t rotate_interval_) + : changelogs_dir(chagelogs_dir_) + , rotate_interval(rotate_interval_) +{ + namespace fs = std::filesystem; + for(const auto & p : fs::directory_iterator(changelogs_dir)) + existing_changelogs.push_back(p.path()); +} + +Changelog ChangelogOnDiskHelper::readChangelogAndInitWriter(size_t from_log_idx) +{ + Changelog result; + size_t read_from_last = 0; + for (const std::string & changelog_file : existing_changelogs) + { + ChangelogName parsed_name = getChangelogName(changelog_file); + if (parsed_name.to_log_idx >= from_log_idx) + { + ChangelogReader reader(changelog_file); + read_from_last = reader.readChangelog(result, index_to_start_pos); + } + } + if (existing_changelogs.size() > 0 && read_from_last < rotate_interval) + { + auto parsed_name = getChangelogName(existing_changelogs.back()); + current_writer = std::make_unique(existing_changelogs.back(), WriteMode::Append, parsed_name.from_log_idx); + current_writer->setEntriesWritten(read_from_last); + } + else + { + rotate(from_log_idx); + } + return result; +} + +void ChangelogOnDiskHelper::rotate(size_t new_start_log_idx) +{ + if (current_writer) + current_writer->flush(); + + ChangelogName new_name; + new_name.prefix = changelogs_dir; + new_name.version = CURRENT_CHANGELOG_VERSION; + new_name.from_log_idx = new_start_log_idx; + new_name.to_log_idx = new_start_log_idx; + + auto new_log_path = formatChagelogPath(changelogs_dir, CURRENT_CHANGELOG_VERSION, new_name); + existing_changelogs.push_back(new_log_path); + current_writer = std::make_unique(existing_changelogs.back(), WriteMode::Rewrite, new_start_log_idx); +} + +ChangelogRecord ChangelogOnDiskHelper::buildRecord(size_t index, nuraft::ptr log_entry) const +{ + ChangelogRecordHeader header; + header.index = index; + header.term = log_entry->get_term(); + header.value_type = log_entry->get_val_type(); + auto buffer = log_entry->get_buf_ptr(); + if (buffer) + { + header.blob_size = buffer->size(); + header.blob_checksum = CityHash_v1_0_2::CityHash128(reinterpret_cast(buffer->data_begin()), buffer->size()); + } + else + { + header.blob_size = 0; + header.blob_checksum = 0; + } + + return ChangelogRecord{header, buffer}; +} + +void ChangelogOnDiskHelper::appendRecord(size_t index, nuraft::ptr log_entry) +{ + if (!current_writer) + throw Exception(ErrorCodes::LOGICAL_ERROR, "ChangelogOnDiskHelper must be initialized before appending records"); + + if (current_writer->getEntriesWritten() == rotate_interval) + rotate(index); + + auto offset = current_writer->appendRecord(buildRecord(index, log_entry), true); + if (!index_to_start_pos.try_emplace(index, offset).second) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Record with index {} already exists", index); + +} + +void ChangelogOnDiskHelper::writeAt(size_t index, nuraft::ptr log_entry) +{ + if (index < current_writer->getStartIndex()) + throw Exception(ErrorCodes::UNIMPLEMENTED, "Currently cannot overwrite index from previous file"); + + auto entries_written = current_writer->getEntriesWritten(); + current_writer->truncateToLength(index_to_start_pos(index)); + for (auto itr = index_to_start_pos.begin(); itr != index_to_start_pos.end();) + { + if (itr->first >= index) + { + entries_written--; + itr = index_to_start_pos.erase(itr); + } + else + itr++; + } + + current_writer->setEntriesWritten(entries_written); + + appendRecord(index, log_entry); +} + +void ChangelogOnDiskHelper::compact(size_t up_to_log_idx) +{ + for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();) + { + ChangelogName parsed_name = getChangelogName(*itr); + if (parsed_name.to_log_idx <= up_to_log_idx) + { + std::filesystem::remove(itr); + itr = existing_changelogs.erase(itr); + for (size_t idx = parsed_name.from_log_idx; idx <= parsed_name.to_log_idx; ++idx) + index_to_start_pos.erase(idx); + } + } +} + +} diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h new file mode 100644 index 00000000000..ffcd2a353bb --- /dev/null +++ b/src/Coordination/Changelog.h @@ -0,0 +1,81 @@ +#pragma once + +#include // Y_IGNORE +#include +#include +#include +#include +#include + +namespace DB +{ + +using Checksum = CityHash_v1_0_2::uint128; + +enum class ChangelogVersion : uint8_t +{ + V0 = 0, +}; + +std::string toString(const ChangelogVersion & version); +ChangelogVersion fromString(const std::string & version_str); + +static constexpr auto CURRENT_CHANGELOG_VERSION = ChangeLogVersion::V0; + +struct ChangelogRecordHeader +{ + ChangelogVersion version = CURRENT_CHANGELOG_VERSION; + size_t index; + size_t term; + nuraft::log_val_type value_type; + size_t blob_size; + Checksum blob_checksum; +}; + +struct ChangelogRecord +{ + ChangelogRecordHeader header; + nuraft::ptr blob; +}; + +using IndexToOffset = std::unordered_map; +using IndexToLogEntry = std::map>; + +struct Changelog +{ +public: +private: + IndexToLogEntry logs; + size_t start_idx = 0; +}; + +class ChangelogWriter; + +class ChangelogOnDiskHelper +{ + +public: + ChangelogOnDiskHelper(const std::string & changelogs_dir_, size_t rotate_interval_); + + Changelog readChangelogAndInitWriter(size_t from_log_idx); + + void appendRecord(size_t index, nuraft::ptr log_entry); + + void writeAt(size_t index, nuraft::ptr log_entry); + + void compact(size_t up_to_log_idx); + +private: + void rotate(size_t new_start_log_idex); + + ChangelogRecord buildRecord(size_t index, nuraft::ptr log_entry) const; + +private: + std::string changelogs_dir; + std::deque existing_changelogs; + std::unique_ptr current_writer; + IndexToOffset index_to_start_pos; + const size_t rotate_interval; +}; + +} diff --git a/src/Coordination/InMemoryLogStore.cpp b/src/Coordination/InMemoryLogStore.cpp index 101458891e7..877c8a60a2a 100644 --- a/src/Coordination/InMemoryLogStore.cpp +++ b/src/Coordination/InMemoryLogStore.cpp @@ -72,12 +72,12 @@ nuraft::ptr>> InMemoryLogStore::log_e ret->resize(end - start); size_t cc = 0; - for (size_t ii = start; ii < end; ++ii) + for (size_t i = start; i < end; ++i) { nuraft::ptr src = nullptr; { std::lock_guard l(logs_lock); - auto entry = logs.find(ii); + auto entry = logs.find(i); if (entry == logs.end()) { entry = logs.find(0); @@ -152,9 +152,9 @@ void InMemoryLogStore::apply_pack(size_t index, nuraft::buffer & pack) pack.pos(0); Int32 num_logs = pack.get_int(); - for (Int32 ii = 0; ii < num_logs; ++ii) + for (Int32 i = 0; i < num_logs; ++i) { - size_t cur_idx = index + ii; + size_t cur_idx = index + i; Int32 buf_size = pack.get_int(); nuraft::ptr buf_local = nuraft::buffer::alloc(buf_size); diff --git a/src/Coordination/NuKeeperLogStore.h b/src/Coordination/NuKeeperLogStore.h new file mode 100644 index 00000000000..2d066ac3e3a --- /dev/null +++ b/src/Coordination/NuKeeperLogStore.h @@ -0,0 +1,24 @@ +#pragma once +#include // Y_IGNORE +#include +#include +#include +#include + +namespace DB +{ + +class NuKeeperLogStore : public nuraft::log_store +{ +public: + NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_); + + +private: + mutable std::mutex logs_lock; + std::atomic start_idx; + Changelog in_memory_changelog; + ChangelogOnDiskHelper on_disk_changelog_helper; +}; + +} From 28dec516acb4921cbd2703b1706bfd67964fe651 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 15 Feb 2021 19:12:03 +0300 Subject: [PATCH 143/510] Style --- src/Client/Connection.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 5ef326acb73..0e8b94ef1cb 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -171,7 +171,8 @@ void Connection::sendHello() * Limiting number of possible characters in user-controlled part of handshake * will mitigate this possibility but doesn't solve it completely. */ - auto has_control_character = [](const std::string & s) { + auto has_control_character = [](const std::string & s) + { for (auto c : s) if (isControlASCII(c)) return true; From 5401116988b83cee6e4cf136d95843494c5523f0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Feb 2021 20:59:40 +0300 Subject: [PATCH 144/510] Compileable code --- src/Coordination/Changelog.cpp | 183 +++++++++++++++++---- src/Coordination/Changelog.h | 54 ++++-- src/Coordination/NuKeeperLogStore.h | 31 +++- src/Coordination/tests/gtest_for_build.cpp | 26 ++- 4 files changed, 238 insertions(+), 56 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index a38f039fa40..f06185124da 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -1,7 +1,11 @@ #include #include #include +#include #include +#include +#include +#include namespace DB { @@ -26,7 +30,7 @@ std::string toString(const ChangelogVersion & version) ChangelogVersion fromString(const std::string & version_str) { - if (version == "V0") + if (version_str == "V0") return ChangelogVersion::V0; throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown chagelog version {}", version_str); @@ -49,13 +53,13 @@ std::string formatChangelogPath(const std::string & prefix, const ChangelogVersi { std::filesystem::path path(prefix); path /= std::filesystem::path(name.prefix + "_" + toString(version) + "_" + std::to_string(name.from_log_idx) + "_" + std::to_string(name.to_log_idx) + ".log"); - return path.to_string(); + return path; } ChangelogName getChangelogName(const std::string & path_str) { std::filesystem::path path(path_str); - std:string filename = path.stem(); + std::string filename = path.stem(); Strings filename_parts; boost::split(filename_parts, filename, boost::is_any_of("_")); if (filename_parts.size() < 4) @@ -64,11 +68,16 @@ ChangelogName getChangelogName(const std::string & path_str) ChangelogName result; result.prefix = filename_parts[0]; result.version = fromString(filename_parts[1]); - result.form_log_idx = parse(filename_parts[2]); + result.from_log_idx = parse(filename_parts[2]); result.to_log_idx = parse(filename_parts[3]); return result; } +LogEntryPtr makeClone(const LogEntryPtr & entry) +{ + return cs_new(entry->get_term(), nuraft::buffer::clone(entry->get_buf()), entry->get_val_type()); +} + } class ChangelogWriter @@ -91,14 +100,14 @@ public: writeIntBinary(record.header.blob_size, plain_buf); writeIntBinary(record.header.blob_checksum, plain_buf); - if (record.blob_size != 0) + if (record.header.blob_size != 0) plain_buf.write(reinterpret_cast(record.blob->data_begin()), record.blob->size()); entries_written++; if (sync) plain_buf.sync(); - reeturn result; + return result; } void truncateToLength(off_t new_length) @@ -117,7 +126,7 @@ public: return entries_written; } - size_t setEntriesWritten(size_t entries_written_) + void setEntriesWritten(size_t entries_written_) { entries_written = entries_written_; } @@ -148,7 +157,7 @@ public: , read_buf(filepath) {} - size_t readChangelog(Changelog & changelog, IndexToOffset & index_to_offset) + size_t readChangelog(IndexToLogEntry & logs, size_t start_log_idx, IndexToOffset & index_to_offset) { size_t total_read = 0; while (!read_buf.eof()) @@ -174,12 +183,12 @@ public: "Checksums doesn't match for log {} (version {}), index {}, blob_size {}", filepath, record.header.version, record.header.index, record.header.blob_size); } + if (record.header.index < start_log_idx) + continue; - if (changlog.start_idx == 0) - changelog.start_idx = record.header.index; - - if (!changelog.try_emplace(record.header.index, buffer).second) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filename); + auto log_entry = nuraft::cs_new(record.header.term, buffer, record.header.value_type); + if (!logs.try_emplace(record.header.index, log_entry).second) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filepath); } return total_read; } @@ -188,8 +197,8 @@ private: ReadBufferFromFile read_buf; }; -ChangelogOnDiskHelper::ChangelogOnDiskHelper(const std::string & changelogs_dir, size_t rotate_interval_) - : changelogs_dir(chagelogs_dir_) +Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval_) + : changelogs_dir(changelogs_dir_) , rotate_interval(rotate_interval_) { namespace fs = std::filesystem; @@ -197,9 +206,8 @@ ChangelogOnDiskHelper::ChangelogOnDiskHelper(const std::string & changelogs_dir, existing_changelogs.push_back(p.path()); } -Changelog ChangelogOnDiskHelper::readChangelogAndInitWriter(size_t from_log_idx) +void Changelog::readChangelogAndInitWriter(size_t from_log_idx) { - Changelog result; size_t read_from_last = 0; for (const std::string & changelog_file : existing_changelogs) { @@ -207,9 +215,12 @@ Changelog ChangelogOnDiskHelper::readChangelogAndInitWriter(size_t from_log_idx) if (parsed_name.to_log_idx >= from_log_idx) { ChangelogReader reader(changelog_file); - read_from_last = reader.readChangelog(result, index_to_start_pos); + read_from_last = reader.readChangelog(logs, from_log_idx, index_to_start_pos); } } + + start_index = from_log_idx == 0 ? 1 : from_log_idx; + if (existing_changelogs.size() > 0 && read_from_last < rotate_interval) { auto parsed_name = getChangelogName(existing_changelogs.back()); @@ -220,26 +231,25 @@ Changelog ChangelogOnDiskHelper::readChangelogAndInitWriter(size_t from_log_idx) { rotate(from_log_idx); } - return result; } -void ChangelogOnDiskHelper::rotate(size_t new_start_log_idx) +void Changelog::rotate(size_t new_start_log_idx) { if (current_writer) current_writer->flush(); ChangelogName new_name; - new_name.prefix = changelogs_dir; + new_name.prefix = DEFAULT_PREFIX; new_name.version = CURRENT_CHANGELOG_VERSION; new_name.from_log_idx = new_start_log_idx; new_name.to_log_idx = new_start_log_idx; - auto new_log_path = formatChagelogPath(changelogs_dir, CURRENT_CHANGELOG_VERSION, new_name); + auto new_log_path = formatChangelogPath(changelogs_dir, CURRENT_CHANGELOG_VERSION, new_name); existing_changelogs.push_back(new_log_path); current_writer = std::make_unique(existing_changelogs.back(), WriteMode::Rewrite, new_start_log_idx); } -ChangelogRecord ChangelogOnDiskHelper::buildRecord(size_t index, nuraft::ptr log_entry) const +ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) const { ChangelogRecordHeader header; header.index = index; @@ -254,16 +264,16 @@ ChangelogRecord ChangelogOnDiskHelper::buildRecord(size_t index, nuraft::ptr log_entry) +void Changelog::appendEntry(size_t index, nuraft::ptr log_entry) { if (!current_writer) - throw Exception(ErrorCodes::LOGICAL_ERROR, "ChangelogOnDiskHelper must be initialized before appending records"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); if (current_writer->getEntriesWritten() == rotate_interval) rotate(index); @@ -271,16 +281,19 @@ void ChangelogOnDiskHelper::appendRecord(size_t index, nuraft::ptrappendRecord(buildRecord(index, log_entry), true); if (!index_to_start_pos.try_emplace(index, offset).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "Record with index {} already exists", index); - + logs[index] = makeClone(log_entry); } -void ChangelogOnDiskHelper::writeAt(size_t index, nuraft::ptr log_entry) +void Changelog::writeAt(size_t index, nuraft::ptr log_entry) { if (index < current_writer->getStartIndex()) throw Exception(ErrorCodes::UNIMPLEMENTED, "Currently cannot overwrite index from previous file"); + if (index_to_start_pos.count(index) == 0) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); + auto entries_written = current_writer->getEntriesWritten(); - current_writer->truncateToLength(index_to_start_pos(index)); + current_writer->truncateToLength(index_to_start_pos[index]); for (auto itr = index_to_start_pos.begin(); itr != index_to_start_pos.end();) { if (itr->first >= index) @@ -294,22 +307,128 @@ void ChangelogOnDiskHelper::writeAt(size_t index, nuraft::ptr current_writer->setEntriesWritten(entries_written); - appendRecord(index, log_entry); + auto itr = logs.lower_bound(index); + while (itr != logs.end()) + itr = logs.erase(itr); + + appendEntry(index, log_entry); } -void ChangelogOnDiskHelper::compact(size_t up_to_log_idx) +void Changelog::compact(size_t up_to_log_idx) { for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();) { ChangelogName parsed_name = getChangelogName(*itr); if (parsed_name.to_log_idx <= up_to_log_idx) { - std::filesystem::remove(itr); + std::filesystem::remove(*itr); itr = existing_changelogs.erase(itr); for (size_t idx = parsed_name.from_log_idx; idx <= parsed_name.to_log_idx; ++idx) + { + auto logs_itr = logs.find(idx); + if (logs_itr != logs.end()) + logs.erase(idx); + else + break; index_to_start_pos.erase(idx); + } } } } +LogEntryPtr Changelog::getLastEntry() const +{ + + static LogEntryPtr fake_entry = nuraft::cs_new(0, nuraft::buffer::alloc(sizeof(size_t))); + + size_t next_idx = getNextEntryIndex() - 1; + auto entry = logs.find(next_idx); + if (entry == logs.end()) + return fake_entry; + + return makeClone(entry->second); +} + +LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end) +{ + LogEntriesPtr ret = nuraft::cs_new>>(); + + ret->resize(end - start); + size_t result_pos = 0; + for (size_t i = start; i < end; ++i) + { + (*ret)[result_pos] = entryAt(i); + result_pos++; + } + return ret; +} + +LogEntryPtr Changelog::entryAt(size_t idx) +{ + nuraft::ptr src = nullptr; + auto entry = logs.find(idx); + if (entry == logs.end()) + return nullptr; + + src = entry->second; + return makeClone(src); +} + +nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, int32_t cnt) +{ + std::vector> returned_logs; + + size_t size_total = 0; + for (size_t i = index; i < index + cnt; ++i) + { + auto entry = logs.find(i); + if (entry == logs.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Don't have log entry {}", i); + + nuraft::ptr buf = entry->second->serialize(); + size_total += buf->size(); + returned_logs.push_back(buf); + } + + nuraft::ptr buf_out = nuraft::buffer::alloc(sizeof(int32_t) + cnt * sizeof(int32_t) + size_total); + buf_out->pos(0); + buf_out->put(static_cast(cnt)); + + for (auto & entry : returned_logs) + { + nuraft::ptr & bb = entry; + buf_out->put(static_cast(bb->size())); + buf_out->put(*bb); + } + return buf_out; +} + +void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer) +{ + buffer.pos(0); + int num_logs = buffer.get_int(); + + for (int i = 0; i < num_logs; ++i) + { + size_t cur_idx = index + i; + int buf_size = buffer.get_int(); + + nuraft::ptr buf_local = nuraft::buffer::alloc(buf_size); + buffer.get(buf_local); + + LogEntryPtr log_entry = nuraft::log_entry::deserialize(*buf_local); + if (i == 0 && logs.count(cur_idx)) + writeAt(cur_idx, log_entry); + else + appendEntry(cur_idx, log_entry); + } +} + +void Changelog::flush() +{ + current_writer->flush(); +} + +Changelog::~Changelog() = default; + } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index ffcd2a353bb..c58f35cb4a1 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -12,6 +12,13 @@ namespace DB using Checksum = CityHash_v1_0_2::uint128; +using LogEntryPtr = nuraft::ptr; +using LogEntries = std::vector; +using LogEntriesPtr = nuraft::ptr; + +using IndexToOffset = std::unordered_map; +using IndexToLogEntry = std::map; + enum class ChangelogVersion : uint8_t { V0 = 0, @@ -20,7 +27,7 @@ enum class ChangelogVersion : uint8_t std::string toString(const ChangelogVersion & version); ChangelogVersion fromString(const std::string & version_str); -static constexpr auto CURRENT_CHANGELOG_VERSION = ChangeLogVersion::V0; +static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0; struct ChangelogRecordHeader { @@ -38,33 +45,48 @@ struct ChangelogRecord nuraft::ptr blob; }; -using IndexToOffset = std::unordered_map; -using IndexToLogEntry = std::map>; -struct Changelog -{ -public: -private: - IndexToLogEntry logs; - size_t start_idx = 0; -}; class ChangelogWriter; -class ChangelogOnDiskHelper +class Changelog { public: - ChangelogOnDiskHelper(const std::string & changelogs_dir_, size_t rotate_interval_); + Changelog(const std::string & changelogs_dir_, size_t rotate_interval_); - Changelog readChangelogAndInitWriter(size_t from_log_idx); + void readChangelogAndInitWriter(size_t from_log_idx); - void appendRecord(size_t index, nuraft::ptr log_entry); + void appendEntry(size_t index, LogEntryPtr log_entry); - void writeAt(size_t index, nuraft::ptr log_entry); + void writeAt(size_t index, LogEntryPtr log_entry); void compact(size_t up_to_log_idx); + size_t getNextEntryIndex() const + { + return start_index + logs.size() - 1; + } + + size_t getStartIndex() const + { + return start_index; + } + + LogEntryPtr getLastEntry() const; + + LogEntriesPtr getLogEntriesBetween(size_t start_index, size_t end_idx); + + LogEntryPtr entryAt(size_t idx); + + nuraft::ptr serializeEntriesToBuffer(size_t index, Int32 cnt); + + void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer); + + void flush(); + + ~Changelog(); + private: void rotate(size_t new_start_log_idex); @@ -76,6 +98,8 @@ private: std::unique_ptr current_writer; IndexToOffset index_to_start_pos; const size_t rotate_interval; + IndexToLogEntry logs; + size_t start_index = 0; }; } diff --git a/src/Coordination/NuKeeperLogStore.h b/src/Coordination/NuKeeperLogStore.h index 2d066ac3e3a..981dc3f24e7 100644 --- a/src/Coordination/NuKeeperLogStore.h +++ b/src/Coordination/NuKeeperLogStore.h @@ -13,12 +13,35 @@ class NuKeeperLogStore : public nuraft::log_store public: NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_); + void init(size_t from_log_idx); + + size_t start_index() const override; + + size_t next_slot() const override; + + nuraft::ptr last_entry() const override; + + size_t append(nuraft::ptr & entry) override; + + void write_at(size_t index, nuraft::ptr & entry) override; + + nuraft::ptr>> log_entries(size_t start, size_t end) override; + + nuraft::ptr entry_at(size_t index) override; + + size_t term_at(size_t index) override; + + nuraft::ptr pack(size_t index, int32_t cnt) override; + + void apply_pack(size_t index, nuraft::buffer & pack) override; + + bool compact(size_t last_log_index) override; + + bool flush() override; private: - mutable std::mutex logs_lock; - std::atomic start_idx; - Changelog in_memory_changelog; - ChangelogOnDiskHelper on_disk_changelog_helper; + mutable std::mutex changelog_lock; + Changelog changelog; }; } diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index ed9777350c5..6142ee0b5c0 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -22,6 +22,8 @@ #include #include // Y_IGNORE #include +#include +#include TEST(CoordinationTest, BuildTest) @@ -134,7 +136,7 @@ struct SimpliestRaftServer using SummingRaftServer = SimpliestRaftServer; -nuraft::ptr getLogEntry(int64_t number) +nuraft::ptr getBuffer(int64_t number) { nuraft::ptr ret = nuraft::buffer::alloc(sizeof(number)); nuraft::buffer_serializer bs(ret); @@ -151,7 +153,7 @@ TEST(CoordinationTest, TestSummingRaft1) /// Single node is leader EXPECT_EQ(s1.raft_instance->get_leader(), 1); - auto entry1 = getLogEntry(143); + auto entry1 = getBuffer(143); auto ret = s1.raft_instance->append_entries({entry1}); EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code(); EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code(); @@ -209,7 +211,7 @@ TEST(CoordinationTest, TestSummingRaft3) EXPECT_EQ(s3.raft_instance->get_leader(), 2); std::cerr << "Starting to add entries\n"; - auto entry = getLogEntry(1); + auto entry = getBuffer(1); auto ret = s2.raft_instance->append_entries({entry}); EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code(); EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code(); @@ -236,7 +238,7 @@ TEST(CoordinationTest, TestSummingRaft3) EXPECT_EQ(s2.state_machine->getValue(), 1); EXPECT_EQ(s3.state_machine->getValue(), 1); - auto non_leader_entry = getLogEntry(3); + auto non_leader_entry = getBuffer(3); auto ret_non_leader1 = s1.raft_instance->append_entries({non_leader_entry}); EXPECT_FALSE(ret_non_leader1->get_accepted()); @@ -245,7 +247,7 @@ TEST(CoordinationTest, TestSummingRaft3) EXPECT_FALSE(ret_non_leader3->get_accepted()); - auto leader_entry = getLogEntry(77); + auto leader_entry = getBuffer(77); auto ret_leader = s2.raft_instance->append_entries({leader_entry}); EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate: entry 78" << ret_leader->get_result_code(); EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 78" << ret_leader->get_result_code(); @@ -333,4 +335,18 @@ TEST(CoordinationTest, TestStorageSerialization) EXPECT_EQ(new_storage.ephemerals[1].size(), 1); } +DB::LogEntryPtr getLogEntry(const std::string & s) +{ + DB::WriteBufferFromNuraftBuffer bufwriter; + writeText(s, bufwriter); + return nuraft::cs_new(0, bufwriter.getBuffer()); +} + +TEST(CoordinationTest, ChangelogTestSimple) +{ + DB::Changelog changelog("./logs", 5); + auto entry = getLogEntry("hello world"); + changelog.appendEntry(1, entry); +} + #endif From ed9f2b5eb99335471c9f0b60bf9633e1d75a5204 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 15 Feb 2021 21:01:01 +0300 Subject: [PATCH 145/510] Linkable code --- src/Coordination/Changelog.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index f06185124da..d3ba176f209 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -16,7 +16,7 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; extern const int UNKNOWN_FORMAT_VERSION; extern const int LOGICAL_ERROR; - extern const int UNIMPLEMENTED; + extern const int NOT_IMPLEMENTED; } @@ -287,7 +287,7 @@ void Changelog::appendEntry(size_t index, nuraft::ptr log_ent void Changelog::writeAt(size_t index, nuraft::ptr log_entry) { if (index < current_writer->getStartIndex()) - throw Exception(ErrorCodes::UNIMPLEMENTED, "Currently cannot overwrite index from previous file"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Currently cannot overwrite index from previous file"); if (index_to_start_pos.count(index) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); From e34d6b0f37da637e2fa68fc05945c6a3e4e57e5a Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 21:25:10 +0300 Subject: [PATCH 146/510] Update docs/ru/sql-reference/functions/date-time-functions.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index d019c18a688..bb4c49e898e 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -706,7 +706,7 @@ SELECT FROM_UNIXTIME(423543535); └──────────────────────────┘ ``` -В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает таким же образом, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). +В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает также, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). Запрос: From e8889463a6351316c1d0ae1cc0b99c8424c767d5 Mon Sep 17 00:00:00 2001 From: Roman Bug Date: Mon, 15 Feb 2021 21:25:31 +0300 Subject: [PATCH 147/510] Update docs/ru/operations/utilities/clickhouse-local.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/operations/utilities/clickhouse-local.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/utilities/clickhouse-local.md b/docs/ru/operations/utilities/clickhouse-local.md index 8ecbbfcce8c..15d069c9acf 100644 --- a/docs/ru/operations/utilities/clickhouse-local.md +++ b/docs/ru/operations/utilities/clickhouse-local.md @@ -77,7 +77,7 @@ $ clickhouse-local --query " 1 2 ``` -Объём оперативной памяти, занимаемой пользователями (Unix): +Объём оперативной памяти, занимаемой процессами, которые запустил пользователь (Unix): Запрос: From ae73600fb0b1b673973199e9213db2b535572458 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 15 Feb 2021 22:48:06 +0300 Subject: [PATCH 148/510] Refactor row level security actions. --- src/Interpreters/ActionsDAG.cpp | 4 +- src/Interpreters/ActionsDAG.h | 4 +- src/Interpreters/ExpressionAnalyzer.cpp | 6 + src/Interpreters/InterpreterSelectQuery.cpp | 155 +++++++++++------- src/Interpreters/InterpreterSelectQuery.h | 2 +- .../getHeaderForProcessingStage.cpp | 12 +- .../MergeTreeBaseSelectProcessor.cpp | 44 +++-- .../MergeTree/MergeTreeBlockReadUtils.cpp | 26 +-- .../MergeTree/MergeTreeRangeReader.cpp | 41 ++--- src/Storages/SelectQueryInfo.h | 8 +- src/Storages/StorageBuffer.cpp | 34 ++-- 11 files changed, 161 insertions(+), 175 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 12942371d4f..bd092bc0296 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -80,14 +80,14 @@ ActionsDAG::Node & ActionsDAG::getNode(const std::string & name) return **it; } -const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type, bool can_replace) +const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type, bool can_replace, bool add_to_index) { Node node; node.type = ActionType::INPUT; node.result_type = std::move(type); node.result_name = std::move(name); - return addNode(std::move(node), can_replace); + return addNode(std::move(node), can_replace, add_to_index); } const ActionsDAG::Node & ActionsDAG::addInput(ColumnWithTypeAndName column, bool can_replace) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 3c8778e239a..d3f1d65d454 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -196,7 +196,7 @@ public: std::string dumpNames() const; std::string dumpDAG() const; - const Node & addInput(std::string name, DataTypePtr type, bool can_replace = false); + const Node & addInput(std::string name, DataTypePtr type, bool can_replace = false, bool add_to_index = true); const Node & addInput(ColumnWithTypeAndName column, bool can_replace = false); const Node & addColumn(ColumnWithTypeAndName column, bool can_replace = false, bool materialize = false); const Node & addAlias(const std::string & name, std::string alias, bool can_replace = false); @@ -220,7 +220,7 @@ public: /// Return true if column was removed from inputs. bool removeUnusedResult(const std::string & column_name); - void projectInput() { settings.project_input = true; } + void projectInput(bool project = true) { settings.project_input = project; } void removeUnusedActions(const Names & required_names); bool hasArrayJoin() const; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index d5a6876f8ef..3145df23b95 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -855,6 +855,10 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( if (!select_query->prewhere()) return prewhere_actions; + Names first_action_names; + if (!chain.steps.empty()) + first_action_names = chain.steps.front()->getRequiredColumns().getNames(); + auto & step = chain.lastStep(sourceColumns()); getRootActions(select_query->prewhere(), only_types, step.actions()); String prewhere_column_name = select_query->prewhere()->getColumnName(); @@ -879,6 +883,7 @@ ActionsDAGPtr SelectQueryExpressionAnalyzer::appendPrewhere( auto tmp_actions = std::make_shared(tmp_actions_dag); auto required_columns = tmp_actions->getRequiredColumns(); NameSet required_source_columns(required_columns.begin(), required_columns.end()); + required_source_columns.insert(first_action_names.begin(), first_action_names.end()); /// Add required columns to required output in order not to remove them after prewhere execution. /// TODO: add sampling and final execution to common chain. @@ -1579,6 +1584,7 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si { const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0); + prewhere_info->prewhere_actions->projectInput(false); NameSet columns_to_remove; for (size_t i = 1; i < step.required_output.size(); ++i) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d13c28e8ff2..2c960b6983a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -107,6 +107,10 @@ namespace ErrorCodes String InterpreterSelectQuery::generateFilterActions( ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const { + std::cerr << "----- InterpreterSelectQuery::generateFilterActions\n"; + for (const auto & name : prerequisite_columns) + std::cerr << name << std::endl; + const auto & db_name = table_id.getDatabaseName(); const auto & table_name = table_id.getTableName(); @@ -141,6 +145,7 @@ String InterpreterSelectQuery::generateFilterActions( auto syntax_result = TreeRewriter(*context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, metadata_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot); actions = analyzer.simpleSelectActions(); + //std::cerr << actions-> return expr_list->children.at(0)->getColumnName(); } @@ -524,6 +529,10 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, input, std::move(input_pipe)); + WriteBufferFromOwnString buf; + query_plan.explainPlan(buf, {.header = true, .actions = true}); + std::cerr << buf.str(); + /// We must guarantee that result structure is the same as in getSampleBlock() if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) { @@ -811,20 +820,54 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu bool intermediate_stage = false; bool to_aggregation_stage = false; bool from_aggregation_stage = false; - const bool execute_row_level_filter_in_prewhere = ( - ( - settings.optimize_move_to_prewhere || // ...when it is allowed to move things to prewhere, so we do it for row-level filter actions too. - expressions.prewhere_info // ...or when we already have prewhere and must execute row-level filter before it. - ) && - !input && !input_pipe && storage && storage->supportsPrewhere() // Check that prewhere can be used at all. - ); + + if (expressions.filter_info) + { + if (!expressions.prewhere_info) + { + const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere(); + if (does_storage_support_prewhere && settings.optimize_move_to_prewhere) + { + /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. + expressions.prewhere_info = std::make_shared( + std::move(expressions.filter_info->actions), + std::move(expressions.filter_info->column_name)); + expressions.prewhere_info->remove_prewhere_column = expressions.filter_info->do_remove_column; + expressions.prewhere_info->need_filter = true; + expressions.filter_info = nullptr; + } + } + else + { + /// Add row level security actions to prewhere. + std::cerr << expressions.filter_info->actions->dumpDAG() << std::endl; + expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions); + expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); + expressions.prewhere_info->row_level_filter_actions->projectInput(false); + if (expressions.filter_info->do_remove_column) + { + /// Instead of removing column, add it to prewhere_actions input (but not in index). + /// It will be removed at prewhere_actions execution. + const auto & index = expressions.prewhere_info->row_level_filter_actions->getIndex(); + auto it = index.find(expressions.prewhere_info->row_level_column_name); + if (it == index.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found column {} in row level security filter {}", + expressions.prewhere_info->row_level_column_name, expressions.prewhere_info->row_level_filter_actions->dumpDAG()); + const auto & node = *it; + + expressions.prewhere_info->prewhere_actions->addInput(node->result_name, node->result_type, true, false); + } + + expressions.filter_info = nullptr; + } + } if (options.only_analyze) { auto read_nothing = std::make_unique(source_header); query_plan.addStep(std::move(read_nothing)); - if (expressions.filter_info && execute_row_level_filter_in_prewhere) + if (expressions.filter_info) { auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -832,12 +875,24 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu expressions.filter_info->column_name, expressions.filter_info->do_remove_column); - row_level_security_step->setStepDescription("Row-level security filter (PREWHERE)"); + row_level_security_step->setStepDescription("Row-level security filter"); query_plan.addStep(std::move(row_level_security_step)); } if (expressions.prewhere_info) { + if (expressions.prewhere_info->row_level_filter_actions) + { + auto row_level_filter_step = std::make_unique( + query_plan.getCurrentDataStream(), + expressions.prewhere_info->row_level_filter_actions, + expressions.prewhere_info->row_level_column_name, + false); + + row_level_filter_step->setStepDescription("Row-level security filter (PREWHERE)"); + query_plan.addStep(std::move(row_level_filter_step)); + } + auto prewhere_step = std::make_unique( query_plan.getCurrentDataStream(), expressions.prewhere_info->prewhere_actions, @@ -887,7 +942,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu to_aggregation_stage = true; /// Read the data from Storage. from_stage - to what stage the request was completed in Storage. - executeFetchColumns(from_stage, query_plan, execute_row_level_filter_in_prewhere); + executeFetchColumns(from_stage, query_plan); LOG_TRACE(log, "{} -> {}", QueryProcessingStage::toString(from_stage), QueryProcessingStage::toString(options.to_stage)); } @@ -952,7 +1007,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu if (expressions.first_stage) { - if (expressions.filter_info && !execute_row_level_filter_in_prewhere) + if (expressions.filter_info) { auto row_level_security_step = std::make_unique( query_plan.getCurrentDataStream(), @@ -1211,30 +1266,6 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c { auto & prewhere_info = *query_info.prewhere_info; - if (prewhere_info.filter_info) - { - auto & filter_info = *prewhere_info.filter_info; - - if (filter_info.alias_actions) - { - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - filter_info.alias_actions); - }); - } - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - filter_info.actions, - filter_info.column_name, - filter_info.do_remove_column); - }); - } - if (prewhere_info.alias_actions) { pipe.addSimpleTransform([&](const Block & header) @@ -1245,6 +1276,18 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c }); } + if (prewhere_info.row_level_filter) + { + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + prewhere_info.row_level_filter, + prewhere_info.row_level_column_name, + false); + }); + } + pipe.addSimpleTransform([&](const Block & header) { return std::make_shared( @@ -1274,7 +1317,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c query_plan.addStep(std::move(read_from_pipe)); } -void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool execute_row_level_filter_in_prewhere) +void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan) { auto & query = getSelectQuery(); const Settings & settings = context->getSettingsRef(); @@ -1351,13 +1394,15 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (storage) { /// Append columns from the table filter to required - auto row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); - if (row_policy_filter) + ActionsDAG * row_policy_filter = nullptr; + if (expressions.filter_info) + row_policy_filter = expressions.filter_info->actions.get(); + // else if (expressions.prewhere_info && expressions.prewhere_info->row_level_filter_actions) + // row_policy_filter = expressions.prewhere_info->row_level_filter_actions.get(); + + if (expressions.filter_info) { - auto initial_required_columns = required_columns; - ActionsDAGPtr actions_dag; - generateFilterActions(actions_dag, row_policy_filter, initial_required_columns); - auto required_columns_from_filter = actions_dag->getRequiredColumns(); + auto required_columns_from_filter = expressions.filter_info->actions->getRequiredColumns(); for (const auto & column : required_columns_from_filter) { @@ -1394,7 +1439,10 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (prewhere_info) { /// Get some columns directly from PREWHERE expression actions - auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); + auto prewhere_required_columns = ( + prewhere_info->row_level_filter_actions ? + prewhere_info->row_level_filter_actions : + prewhere_info->prewhere_actions)->getRequiredColumns().getNames(); required_columns_from_prewhere.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); } @@ -1605,31 +1653,18 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc { query_info.prewhere_info = std::make_shared(); - if (expressions.filter_info && execute_row_level_filter_in_prewhere) - { - query_info.prewhere_info->filter_info = std::make_shared(); - - if (alias_actions) - query_info.prewhere_info->filter_info->alias_actions = std::make_shared(std::move(alias_actions)); - - if (expressions.filter_info->actions) - query_info.prewhere_info->filter_info->actions = std::make_shared(expressions.filter_info->actions); - - query_info.prewhere_info->filter_info->column_name = expressions.filter_info->column_name; - query_info.prewhere_info->filter_info->do_remove_column = expressions.filter_info->do_remove_column; - } + query_info.prewhere_info->prewhere_actions = std::make_shared(prewhere_info->prewhere_actions); + if (prewhere_info->row_level_filter_actions) + query_info.prewhere_info->row_level_filter = std::make_shared(prewhere_info->row_level_filter_actions); if (prewhere_info->alias_actions) query_info.prewhere_info->alias_actions = std::make_shared(prewhere_info->alias_actions); - - if (prewhere_info->prewhere_actions) - query_info.prewhere_info->prewhere_actions = std::make_shared(prewhere_info->prewhere_actions); - if (prewhere_info->remove_columns_actions) query_info.prewhere_info->remove_columns_actions = std::make_shared(prewhere_info->remove_columns_actions); query_info.prewhere_info->prewhere_column_name = prewhere_info->prewhere_column_name; query_info.prewhere_info->remove_prewhere_column = prewhere_info->remove_prewhere_column; + query_info.prewhere_info->row_level_column_name = prewhere_info->row_level_column_name; query_info.prewhere_info->need_filter = prewhere_info->need_filter; } diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 793df612103..20cffdf5702 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -108,7 +108,7 @@ private: /// Different stages of query execution. - void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan, bool execute_row_level_filter_in_prewhere); + void executeFetchColumns(QueryProcessingStage::Enum processing_stage, QueryPlan & query_plan); void executeWhere(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool remove_filter); void executeAggregation(QueryPlan & query_plan, const ActionsDAGPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPlan & query_plan, bool overflow_row, bool final); diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 2aef3c25c3c..3adbab8413f 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -46,16 +46,8 @@ Block getHeaderForProcessingStage( { auto & prewhere_info = *query_info.prewhere_info; - if (prewhere_info.filter_info) - { - auto & filter_info = *prewhere_info.filter_info; - - if (filter_info.actions) - filter_info.actions->execute(header); - - if (filter_info.do_remove_column) - header.erase(filter_info.column_name); - } + if (prewhere_info.row_level_filter) + prewhere_info.row_level_filter->execute(header); if (prewhere_info.prewhere_actions) prewhere_info.prewhere_actions->execute(header); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 96993e4a106..5a46ed29e3d 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -334,38 +334,30 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P { if (prewhere_info) { - if (prewhere_info->filter_info) - { - auto & filter_info = *prewhere_info->filter_info; - - if (filter_info.alias_actions) - filter_info.alias_actions->execute(block); - - if (filter_info.actions) - filter_info.actions->execute(block); - - auto & filter_column = block.getByName(filter_info.column_name); - if (!filter_column.type->canBeUsedInBooleanContext()) - { - throw Exception("Invalid type for row-level security filter: " + filter_column.type->getName(), - ErrorCodes::LOGICAL_ERROR); - } - - if (filter_info.do_remove_column) - block.erase(filter_info.column_name); - else - { - auto & ctn = block.getByName(filter_info.column_name); - ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); - } - } + std::cerr << "0: " << block.dumpStructure() << std::endl; if (prewhere_info->alias_actions) prewhere_info->alias_actions->execute(block); + std::cerr << "1: " << block.dumpStructure() << std::endl; + + if (prewhere_info->row_level_filter) + { + prewhere_info->row_level_filter->execute(block); + auto & row_level_column = block.getByName(prewhere_info->row_level_column_name); + if (!row_level_column.type->canBeUsedInBooleanContext()) + { + throw Exception("Invalid type for filter in PREWHERE: " + row_level_column.type->getName(), + ErrorCodes::LOGICAL_ERROR); + } + } + std::cerr << "2: " << block.dumpStructure() << std::endl; + if (prewhere_info->prewhere_actions) prewhere_info->prewhere_actions->execute(block); + std::cerr << "3: " << block.dumpStructure() << std::endl; + auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) { @@ -380,6 +372,8 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P auto & ctn = block.getByName(prewhere_info->prewhere_column_name); ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); } + + std::cerr << "4: " << block.dumpStructure() << std::endl; } } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index f4a5b1fcb9e..ed5fc48dad1 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -269,30 +269,12 @@ MergeTreeReadTaskColumns getReadTaskColumns( if (prewhere_info) { - if (prewhere_info->filter_info) - { - if (prewhere_info->filter_info->alias_actions) - { - const auto required_column_names = prewhere_info->filter_info->alias_actions->getRequiredColumns(); - pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); - } - else if (prewhere_info->filter_info->actions) - { - const auto required_column_names = prewhere_info->filter_info->actions->getRequiredColumns(); - pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); - } - } - if (prewhere_info->alias_actions) - { - const auto required_column_names = prewhere_info->alias_actions->getRequiredColumns(); - pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); - } + pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); + else if (prewhere_info->row_level_filter) + pre_column_names = prewhere_info->row_level_filter->getRequiredColumns(); else if (prewhere_info->prewhere_actions) - { - const auto required_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); - pre_column_names.insert(pre_column_names.end(), required_column_names.begin(), required_column_names.end()); - } + pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 3c79ed73a16..b4b8e4309b5 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -561,18 +561,12 @@ MergeTreeRangeReader::MergeTreeRangeReader( if (prewhere_info) { - if (prewhere_info->filter_info) - { - if (prewhere_info->filter_info->actions) - prewhere_info->filter_info->actions->execute(sample_block, true); - - if (prewhere_info->filter_info->do_remove_column) - sample_block.erase(prewhere_info->filter_info->column_name); - } - if (prewhere_info->alias_actions) prewhere_info->alias_actions->execute(sample_block, true); + if (prewhere_info->row_level_filter) + prewhere_info->row_level_filter->execute(sample_block, true); + if (prewhere_info->prewhere_actions) prewhere_info->prewhere_actions->execute(sample_block, true); @@ -897,31 +891,20 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r for (auto name_and_type = header.begin(); pos < num_columns; ++pos, ++name_and_type) block.insert({result.columns[pos], name_and_type->type, name_and_type->name}); - if (prewhere_info->filter_info) - { - if (prewhere_info->filter_info->alias_actions) - prewhere_info->filter_info->alias_actions->execute(block); - - if (prewhere_info->filter_info->actions) - prewhere_info->filter_info->actions->execute(block); - - const auto filter_column_pos = block.getPositionByName(prewhere_info->filter_info->column_name); - result.addFilter(block.getByPosition(filter_column_pos).column); - - if (prewhere_info->filter_info->do_remove_column) - block.erase(prewhere_info->filter_info->column_name); - else - block.getByPosition(filter_column_pos).column = block.getByPosition(filter_column_pos).type->createColumnConst(result.num_rows, 1); - } - if (prewhere_info->alias_actions) prewhere_info->alias_actions->execute(block); /// Columns might be projected out. We need to store them here so that default columns can be evaluated later. result.block_before_prewhere = block; - if (prewhere_info->prewhere_actions) - prewhere_info->prewhere_actions->execute(block); + if (prewhere_info->row_level_filter) + { + prewhere_info->row_level_filter->execute(block); + const auto filter_column_pos = block.getPositionByName(prewhere_info->row_level_column_name); + result.addFilter(block.getByPosition(filter_column_pos).column); + } + + prewhere_info->prewhere_actions->execute(block); prewhere_column_pos = block.getPositionByName(prewhere_info->prewhere_column_name); result.addFilter(block.getByPosition(prewhere_column_pos).column); @@ -943,7 +926,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.totalRowsPerGranule() == 0) result.setFilterConstFalse(); /// If we need to filter in PREWHERE - else if (prewhere_info->need_filter || result.need_filter || prewhere_info->remove_prewhere_column) + else if (prewhere_info->need_filter || result.need_filter || prewhere_info->row_level_filter) { /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index a87ff2f40d3..fea9a7bad68 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -41,14 +41,16 @@ using ClusterPtr = std::shared_ptr; struct PrewhereInfo { - /// Information about the preliminary filter expression, if any. - FilterInfoPtr filter_info; /// Actions which are executed in order to alias columns are used for prewhere actions. ExpressionActionsPtr alias_actions; + /// Actions for row level security filter. Applied separately before prewhere_actions. + /// This actions are separate because prewhere condition should not be executed over filtered rows. + ExpressionActionsPtr row_level_filter; /// Actions which are executed on block in order to get filter column for prewhere step. ExpressionActionsPtr prewhere_actions; /// Actions which are executed after reading from storage in order to remove unused columns. ExpressionActionsPtr remove_columns_actions; + String row_level_column_name; String prewhere_column_name; bool remove_prewhere_column = false; bool need_filter = false; @@ -58,8 +60,10 @@ struct PrewhereInfo struct PrewhereDAGInfo { ActionsDAGPtr alias_actions; + ActionsDAGPtr row_level_filter_actions; ActionsDAGPtr prewhere_actions; ActionsDAGPtr remove_columns_actions; + String row_level_column_name; String prewhere_column_name; bool remove_prewhere_column = false; bool need_filter = false; diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 64bcdd2d145..15dec77caf3 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -321,28 +321,6 @@ void StorageBuffer::read( { if (query_info.prewhere_info) { - if (query_info.prewhere_info->filter_info) - { - if (query_info.prewhere_info->filter_info->alias_actions) - { - pipe_from_buffers.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - query_info.prewhere_info->filter_info->alias_actions); - }); - } - - pipe_from_buffers.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - query_info.prewhere_info->filter_info->actions, - query_info.prewhere_info->filter_info->column_name, - query_info.prewhere_info->filter_info->do_remove_column); - }); - } - if (query_info.prewhere_info->alias_actions) { pipe_from_buffers.addSimpleTransform([&](const Block & header) @@ -353,6 +331,18 @@ void StorageBuffer::read( }); } + if (query_info.prewhere_info->row_level_filter) + { + pipe_from_buffers.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + query_info.prewhere_info->row_level_filter, + query_info.prewhere_info->row_level_column_name, + false); + }); + } + pipe_from_buffers.addSimpleTransform([&](const Block & header) { return std::make_shared( From 85277d6a417b633431ac18f2bdbdb7ca7da57568 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 15 Feb 2021 22:54:47 +0300 Subject: [PATCH 149/510] Comment debug output. --- .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 5a46ed29e3d..90da45cc6d8 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -334,12 +334,12 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P { if (prewhere_info) { - std::cerr << "0: " << block.dumpStructure() << std::endl; + // std::cerr << "0: " << block.dumpStructure() << std::endl; if (prewhere_info->alias_actions) prewhere_info->alias_actions->execute(block); - std::cerr << "1: " << block.dumpStructure() << std::endl; + // std::cerr << "1: " << block.dumpStructure() << std::endl; if (prewhere_info->row_level_filter) { @@ -351,12 +351,12 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P ErrorCodes::LOGICAL_ERROR); } } - std::cerr << "2: " << block.dumpStructure() << std::endl; + // std::cerr << "2: " << block.dumpStructure() << std::endl; if (prewhere_info->prewhere_actions) prewhere_info->prewhere_actions->execute(block); - std::cerr << "3: " << block.dumpStructure() << std::endl; + // std::cerr << "3: " << block.dumpStructure() << std::endl; auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) @@ -373,7 +373,7 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); } - std::cerr << "4: " << block.dumpStructure() << std::endl; + // std::cerr << "4: " << block.dumpStructure() << std::endl; } } From e7bbb6cb23446791cabdd1ab315d29107e857324 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Feb 2021 23:09:06 +0300 Subject: [PATCH 150/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index f752bb9f6cb..189cf74049c 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -701,7 +701,7 @@ parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); **Parameters** -- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md). +- `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`). [String](../../sql-reference/data-types/string.md). - `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md). **Supported non-standard formats** From 5eda6169902306fb4e9f07e28327aff9531b3052 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Feb 2021 23:14:01 +0300 Subject: [PATCH 151/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 189cf74049c..06ac64646ae 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -702,7 +702,7 @@ parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); **Parameters** - `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`). [String](../../sql-reference/data-types/string.md). -- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md). +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). **Supported non-standard formats** From a09c9be48b6ba4d42029459486639b3c6b504429 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Feb 2021 23:30:39 +0300 Subject: [PATCH 152/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../en/sql-reference/functions/type-conversion-functions.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 06ac64646ae..24ac8d91d22 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -707,10 +707,10 @@ parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); **Supported non-standard formats** - A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). -- A string with a date and a time component: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. +- A string with a date and a time components: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. - A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. -- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted as `2000-01`. -- A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. +- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted with `2000-01`. +- A string that includes date and time along with timezone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. **Returned values** From f6cbad65e82267b6c6e9bc0fcc672f0802085384 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Feb 2021 23:33:35 +0300 Subject: [PATCH 153/510] Update docs/en/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../en/sql-reference/functions/type-conversion-functions.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 24ac8d91d22..6cc0fe52442 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -714,8 +714,10 @@ parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); **Returned values** -- `time_string` converted to the `DateTime` data type. -- `NULL`. +Possible values: + +- `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type. +- `NULL` if the input string cannot be converted to the `DateTime` data type. **Examples** From c9a6b21fc8c20f08c4abbe62398d635deb5de3d4 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 15 Feb 2021 23:47:12 +0300 Subject: [PATCH 154/510] Fix the English version MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил английскую версию согласно комментариям в PR. --- .../functions/type-conversion-functions.md | 52 ++++++++----------- 1 file changed, 23 insertions(+), 29 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 6cc0fe52442..08e83771af7 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -691,12 +691,12 @@ Same as for [parseDateTimeBestEffort](#parsedatetimebesteffort) except that it r ## parseDateTimeBestEffortUSOrNull {#parsedatetimebesteffortusornull} -Same as for [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) except that it returns `NULL` when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns `NULL` when it encounters a date format that cannot be processed. **Syntax** ``` sql -parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); +parseDateTimeBestEffortUSOrNull(time_string[, time_zone]) ``` **Parameters** @@ -716,16 +716,15 @@ parseDateTimeBestEffortUSOrNull(time_string [, time_zone]); Possible values: -- `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type. -- `NULL` if the input string cannot be converted to the `DateTime` data type. +- `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type. +- `NULL` if the input string cannot be converted to the `DateTime` data type. **Examples** Query: ``` sql -SELECT parseDateTimeBestEffortUSOrNull('02/10/2021 21:12:57') -AS parseDateTimeBestEffortUSOrNull; +SELECT parseDateTimeBestEffortUSOrNull('02/10/2021 21:12:57') AS parseDateTimeBestEffortUSOrNull; ``` Result: @@ -739,8 +738,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrNull('02-10-2021 21:12:57') -AS parseDateTimeBestEffortUSOrNull; +SELECT parseDateTimeBestEffortUSOrNull('02-10-2021 21:12:57') AS parseDateTimeBestEffortUSOrNull; ``` Result: @@ -754,8 +752,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrNull('02.10.2021 21:12:57') -AS parseDateTimeBestEffortUSOrNull; +SELECT parseDateTimeBestEffortUSOrNull('02.10.2021 21:12:57') AS parseDateTimeBestEffortUSOrNull; ``` Result: @@ -769,8 +766,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrNull('02.2021 21:12:57') -AS parseDateTimeBestEffortUSOrNull; +SELECT parseDateTimeBestEffortUSOrNull('02.10.2021') AS parseDateTimeBestEffortUSOrNull; ``` Result: @@ -783,30 +779,32 @@ Result: ## parseDateTimeBestEffortUSOrZero {#parsedatetimebesteffortusorzero} -Same as for [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) except that it returns zero date or zero date time when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns zero date or zero date with time when it encounters a date format that cannot be processed. **Syntax** ``` sql -parseDateTimeBestEffortUSOrZero(time_string [, time_zone]); +parseDateTimeBestEffortUSOrZero(time_string[, time_zone]) ``` **Parameters** -- `time_string` — String containing a date and time to convert. [String](../../sql-reference/data-types/string.md). -- `time_zone` — Time zone. The function parses `time_string` according to the time zone. [String](../../sql-reference/data-types/string.md). +- `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`). [String](../../sql-reference/data-types/string.md). +- `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). **Supported non-standard formats** - A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). -- A string with a date and a time component: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. +- A string with a date and a time components: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. - A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. -- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted as `2000-01`. -- A string that includes the date and time along with time zone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. +- A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted with `2000-01`. +- A string that includes date and time along with timezone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. -**Returned value** +**Returned values** -- `time_string` converted to the `DateTime` data type. +Possible values: + +- `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type. - `zero date time`. **Examples** @@ -814,8 +812,7 @@ parseDateTimeBestEffortUSOrZero(time_string [, time_zone]); Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02/10/2021 21:12:57') -AS parseDateTimeBestEffortUSOrZero; +SELECT parseDateTimeBestEffortUSOrZero('02/10/2021 21:12:57') AS parseDateTimeBestEffortUSOrZero; ``` Result: @@ -829,8 +826,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02-10-2021 21:12:57') -AS parseDateTimeBestEffortUSOrZero; +SELECT parseDateTimeBestEffortUSOrZero('02-10-2021 21:12:57') AS parseDateTimeBestEffortUSOrZero; ``` Result: @@ -844,8 +840,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02.10.2021 21:12:57') -AS parseDateTimeBestEffortUS; +SELECT parseDateTimeBestEffortUSOrZero('02.10.2021 21:12:57') AS parseDateTimeBestEffortUS; ``` Result: @@ -859,8 +854,7 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02.2021 21:12:57') -AS parseDateTimeBestEffortUSOrZero; +SELECT parseDateTimeBestEffortUSOrZero('02.2021 21:12:57') AS parseDateTimeBestEffortUSOrZero; ``` Result: From 937a3192eb6d5fad2ccdb4294f91f5c6d7af53b8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 15 Feb 2021 22:54:23 +0300 Subject: [PATCH 155/510] Fix data race --- src/Common/Epoll.cpp | 24 ++++++++++++++++++------ src/Common/Epoll.h | 9 ++++----- 2 files changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 628bb45e796..d085315b1a0 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -21,9 +21,21 @@ Epoll::Epoll() : events_count(0) throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR); } -Epoll::Epoll(Epoll && other) : epoll_fd(other.epoll_fd), events_count(other.events_count) +Epoll::Epoll(Epoll && other) { + epoll_fd = other.epoll_fd; other.epoll_fd = -1; + int count = other.events_count; + events_count = count; +} + +Epoll & Epoll::operator=(Epoll && other) +{ + epoll_fd = other.epoll_fd; + other.epoll_fd = -1; + int count = other.events_count; + events_count = count; + return *this; } void Epoll::add(int fd, void * ptr) @@ -35,18 +47,18 @@ void Epoll::add(int fd, void * ptr) else event.data.fd = fd; + ++events_count; + if (epoll_ctl(epoll_fd, EPOLL_CTL_ADD, fd, &event) == -1) throwFromErrno("Cannot add new descriptor to epoll", DB::ErrorCodes::EPOLL_ERROR); - - ++events_count; } void Epoll::remove(int fd) { + --events_count; + if (epoll_ctl(epoll_fd, EPOLL_CTL_DEL, fd, nullptr) == -1) throwFromErrno("Cannot remove descriptor from epoll", DB::ErrorCodes::EPOLL_ERROR); - - --events_count; } size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback) const @@ -54,7 +66,7 @@ size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocki if (events_count == 0) throw Exception("There is no events in epoll", ErrorCodes::LOGICAL_ERROR); - int ready_size = 0; + int ready_size; int timeout = blocking && !async_callback ? -1 : 0; do { diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index eb168c22a92..a7090bdb9b6 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -16,13 +16,12 @@ class Epoll public: Epoll(); - Epoll(const Epoll & other) = delete; - Epoll & operator=(const Epoll & other) = delete; + Epoll(const Epoll &) = delete; + Epoll & operator=(const Epoll &) = delete; + Epoll & operator=(Epoll && other); Epoll(Epoll && other); - Epoll & operator=(Epoll && other) = default; - /// Add new file descriptor to epoll. If ptr set to nullptr, epoll_event.data.fd = fd, /// otherwise epoll_event.data.ptr = ptr. void add(int fd, void * ptr = nullptr); @@ -47,7 +46,7 @@ public: private: int epoll_fd; - int events_count; + std::atomic events_count; }; } From 21f80a9367760528b12c0639d3c4faacf7c100e0 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 16 Feb 2021 00:42:16 +0300 Subject: [PATCH 156/510] Add examples MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил примеры. --- .../functions/type-conversion-functions.md | 42 +++++++++---------- 1 file changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 08e83771af7..81b5649db32 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -738,28 +738,14 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrNull('02-10-2021 21:12:57') AS parseDateTimeBestEffortUSOrNull; +SELECT parseDateTimeBestEffortUSOrNull('02-10-2021 21:12:57 GMT', 'Europe/Moscow') AS parseDateTimeBestEffortUSOrNull; ``` Result: ``` text ┌─parseDateTimeBestEffortUSOrNull─┐ -│ 2021-02-10 21:12:57 │ -└─────────────────────────────────┘ -``` - -Query: - -``` sql -SELECT parseDateTimeBestEffortUSOrNull('02.10.2021 21:12:57') AS parseDateTimeBestEffortUSOrNull; -``` - -Result: - -``` text -┌─parseDateTimeBestEffortUSOrNull─┐ -│ 2021-02-10 21:12:57 │ +│ 2021-02-11 00:12:57 │ └─────────────────────────────────┘ ``` @@ -771,6 +757,20 @@ SELECT parseDateTimeBestEffortUSOrNull('02.10.2021') AS parseDateTimeBestEffortU Result: +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-10 00:00:00 │ +└─────────────────────────────────┘ +``` + +Query: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('10.2021') AS parseDateTimeBestEffortUSOrNull; +``` + +Result: + ``` text ┌─parseDateTimeBestEffortUSOrNull─┐ │ ᴺᵁᴸᴸ │ @@ -826,35 +826,35 @@ Result: Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02-10-2021 21:12:57') AS parseDateTimeBestEffortUSOrZero; +SELECT parseDateTimeBestEffortUSOrZero('02-10-2021 21:12:57 GMT', 'Europe/Moscow') AS parseDateTimeBestEffortUSOrZero; ``` Result: ``` text ┌─parseDateTimeBestEffortUSOrZero─┐ -│ 2021-02-10 21:12:57 │ +│ 2021-02-11 00:12:57 │ └─────────────────────────────────┘ ``` Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02.10.2021 21:12:57') AS parseDateTimeBestEffortUS; +SELECT parseDateTimeBestEffortUSOrZero('02.10.2021') AS parseDateTimeBestEffortUSOrZero; ``` Result: ``` text ┌─parseDateTimeBestEffortUSOrZero─┐ -│ 2021-02-10 21:12:57 │ +│ 2021-02-10 00:00:00 │ └─────────────────────────────────┘ ``` Query: ``` sql -SELECT parseDateTimeBestEffortUSOrZero('02.2021 21:12:57') AS parseDateTimeBestEffortUSOrZero; +SELECT parseDateTimeBestEffortUSOrZero('02.2021') AS parseDateTimeBestEffortUSOrZero; ``` Result: From 6dcb306060e0fb70371eb6b5d5fceb1357d29ed9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 Feb 2021 00:46:51 +0300 Subject: [PATCH 157/510] Style --- src/Client/Connection.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 0e8b94ef1cb..ee2d4474a0d 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -106,8 +106,7 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). static_cast(socket.get())->setPeerHostName(host); #else - throw Exception{ - "tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; + throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif } else From f139ad8080f66105eef83a80fef8310d31a85b2f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Feb 2021 09:15:12 +0300 Subject: [PATCH 158/510] Comment debug output. --- src/Interpreters/InterpreterSelectQuery.cpp | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 2c960b6983a..45d187c34d5 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -107,9 +107,9 @@ namespace ErrorCodes String InterpreterSelectQuery::generateFilterActions( ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const { - std::cerr << "----- InterpreterSelectQuery::generateFilterActions\n"; - for (const auto & name : prerequisite_columns) - std::cerr << name << std::endl; + // std::cerr << "----- InterpreterSelectQuery::generateFilterActions\n"; + // for (const auto & name : prerequisite_columns) + // std::cerr << name << std::endl; const auto & db_name = table_id.getDatabaseName(); const auto & table_name = table_id.getTableName(); @@ -529,9 +529,9 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, input, std::move(input_pipe)); - WriteBufferFromOwnString buf; - query_plan.explainPlan(buf, {.header = true, .actions = true}); - std::cerr << buf.str(); + // WriteBufferFromOwnString buf; + // query_plan.explainPlan(buf, {.header = true, .actions = true}); + // std::cerr << buf.str(); /// We must guarantee that result structure is the same as in getSampleBlock() if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) @@ -840,7 +840,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu else { /// Add row level security actions to prewhere. - std::cerr << expressions.filter_info->actions->dumpDAG() << std::endl; + // std::cerr << expressions.filter_info->actions->dumpDAG() << std::endl; expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions); expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); expressions.prewhere_info->row_level_filter_actions->projectInput(false); From 0e1d67ad9ac17c48fe2c7b44bd8b3a1ad485927e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 16 Feb 2021 10:56:45 +0300 Subject: [PATCH 159/510] Add LOG_DEBUG to debug test_distributed_load_balancing --- src/Client/Connection.cpp | 16 +++++++++++ src/Client/HedgedConnections.cpp | 11 ++++++++ src/Client/HedgedConnections.h | 2 ++ src/Client/HedgedConnectionsFactory.cpp | 28 +++++++++++++++++++ .../configs/users.xml | 8 ++++++ .../test_distributed_load_balancing/test.py | 1 + 6 files changed, 66 insertions(+) create mode 100644 tests/integration/test_distributed_load_balancing/configs/users.xml diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 0e8b94ef1cb..65dcdfd5fe7 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -74,6 +74,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) void Connection::disconnect() { + LOG_DEBUG(log, "disconnect"); maybe_compressed_out = nullptr; in = nullptr; last_input_packet_type.reset(); @@ -160,6 +161,7 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) void Connection::sendHello() { + LOG_DEBUG(log_wrapper.get(), "sendHello"); try { /** Disallow control characters in user controlled parameters @@ -233,6 +235,8 @@ void Connection::sendHello() void Connection::receiveHello() { + LOG_DEBUG(log_wrapper.get(), "receiveHello"); + try { /// Receive hello packet. @@ -430,6 +434,8 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) { + LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest"); + writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); out->next(); @@ -437,6 +443,8 @@ void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) TablesStatusResponse Connection::receiveTablesStatusResponse() { + LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse"); + UInt64 response_type = 0; readVarUInt(response_type, *in); @@ -459,6 +467,8 @@ void Connection::sendQuery( const ClientInfo * client_info, bool with_pending_data) { + LOG_DEBUG(log_wrapper.get(), "sendQuery"); + if (!connected) connect(timeouts); @@ -556,6 +566,8 @@ void Connection::sendQuery( void Connection::sendCancel() { + LOG_DEBUG(log_wrapper.get(), "sendCancel"); + /// If we already disconnected. if (!out) return; @@ -806,6 +818,8 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) Packet Connection::receivePacket(AsyncCallback async_callback) { + LOG_DEBUG(log_wrapper.get(), "receivePacket"); + in->setAsyncCallback(std::move(async_callback)); SCOPE_EXIT(in->setAsyncCallback({})); @@ -883,6 +897,8 @@ Packet Connection::receivePacket(AsyncCallback async_callback) Block Connection::receiveData() { + LOG_DEBUG(log_wrapper.get(), "receiveData"); + initBlockInput(); return receiveDataImpl(block_in); } diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index ad00c60b302..6d49c0f6749 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -45,6 +45,8 @@ HedgedConnections::HedgedConnections( active_connection_count = connections.size(); offsets_with_received_first_data_packet = 0; pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); }); + + log = &Poco::Logger::get("HedgedConnections"); } void HedgedConnections::Pipeline::add(std::function send_function) @@ -285,6 +287,7 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback) { + LOG_DEBUG(log, "getReadyReplicaLocation"); int event_fd; while (true) { @@ -374,6 +377,8 @@ bool HedgedConnections::checkPendingData(ReplicaLocation & location_out) Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & replica_location, AsyncCallback async_callback) { + LOG_DEBUG(log, "receivePacketFromReplica"); + ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; replica.receive_timeout.reset(); Packet packet = replica.connection->receivePacket(std::move(async_callback)); @@ -408,6 +413,8 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & replica_location) { + LOG_DEBUG(log, "processReceivedFirstDataPacket"); + /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. OffsetState & offset_state = offset_states[replica_location.offset]; @@ -438,6 +445,8 @@ void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & r void HedgedConnections::tryGetNewReplica(bool start_new_connection) { + LOG_DEBUG(log, "tryGetNewReplica"); + Connection * connection = nullptr; HedgedConnectionsFactory::State state = hedged_connections_factory.getNextConnection(start_new_connection, false, connection); @@ -488,6 +497,8 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { + LOG_DEBUG(log, "finishProcessReplica"); + epoll.remove(replica.epoll.getFileDescriptor()); --offset_states[fd_to_replica_location[replica.epoll.getFileDescriptor()].offset].active_connection_count; fd_to_replica_location.erase(replica.epoll.getFileDescriptor()); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 249c41a7a06..41c548de9ef 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -163,6 +163,8 @@ private: bool cancelled = false; mutable std::mutex cancel_mutex; + + Poco::Logger * log; }; } diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index c881c2723df..ba0e4ac7b22 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -43,6 +43,7 @@ HedgedConnectionsFactory::~HedgedConnectionsFactory() std::vector HedgedConnectionsFactory::getManyConnections(PoolMode pool_mode) { + LOG_DEBUG(log, "getManyConnections"); size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; size_t max_entries; @@ -102,6 +103,8 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out) { + LOG_DEBUG(log, "getNextConnection"); + if (start_new_connection) { int index = startEstablishingNewConnection(connection_out); @@ -125,6 +128,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool void HedgedConnectionsFactory::stopChoosingReplicas() { + LOG_DEBUG(log, "stopChoosingReplicas"); for (auto & [fd, replica_index] : fd_to_replica_index) { resetReplicaTimeouts(replica_index); @@ -137,6 +141,8 @@ void HedgedConnectionsFactory::stopChoosingReplicas() int HedgedConnectionsFactory::getNextIndex() { + LOG_DEBUG(log, "getNextIndex"); + /// Check if there is no free replica. if (entries_count + replicas_in_process_count + failed_pools_count >= shuffled_pools.size()) return -1; @@ -170,9 +176,13 @@ int HedgedConnectionsFactory::getNextIndex() int HedgedConnectionsFactory::startEstablishingNewConnection(Connection *& connection_out) { + LOG_DEBUG(log, "startEstablishingNewConnection"); + int index; do { + LOG_DEBUG(log, "startEstablishingNewConnection loop"); + index = getNextIndex(); if (index == -1) return -1; @@ -205,6 +215,8 @@ int HedgedConnectionsFactory::startEstablishingNewConnection(Connection *& conne void HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_index, bool remove_from_epoll) { + LOG_DEBUG(log, "processConnectionEstablisherStage"); + ReplicaStatus & replica = replicas[replica_index]; if (replica.connection_establisher.stage == ConnectionEstablisher::Stage::FINISHED) @@ -224,6 +236,7 @@ void HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_ind ++usable_count; if (replica.connection_establisher.result.is_up_to_date) { + LOG_DEBUG(log, "READY"); ++ready_replicas_count; replica.is_ready = true; return; @@ -242,6 +255,8 @@ void HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_ind void HedgedConnectionsFactory::processFailedConnection(int replica_index, bool remove_from_epoll) { + LOG_DEBUG(log, "processFailedConnection"); + if (remove_from_epoll) { epoll.remove(replicas[replica_index].epoll.getFileDescriptor()); @@ -271,6 +286,8 @@ void HedgedConnectionsFactory::processFailedConnection(int replica_index, bool r void HedgedConnectionsFactory::addTimeouts(int replica_index) { + LOG_DEBUG(log, "addTimeouts"); + auto stage = replicas[replica_index].connection_establisher.stage; if (stage == ConnectionEstablisher::Stage::RECEIVE_HELLO) { @@ -286,12 +303,16 @@ void HedgedConnectionsFactory::addTimeouts(int replica_index) void HedgedConnectionsFactory::resetReplicaTimeouts(int replica_index) { + LOG_DEBUG(log, "resetReplicaTimeouts"); + replicas[replica_index].receive_timeout.reset(); replicas[replica_index].change_replica_timeout.reset(); } HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out) { + LOG_DEBUG(log, "processEpollEvents"); + int event_fd; while (!epoll.empty()) { @@ -353,7 +374,10 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(boo processReceiveTimeout(replica_index); if (is_change_replica_timeout_alarmed) + { + LOG_DEBUG(log, "change_replica_timeout"); replicas[replica_index].change_replica_timeout.reset(); + } } /// We reach this point only if we need to start new connection. @@ -385,6 +409,8 @@ int HedgedConnectionsFactory::checkPendingData() void HedgedConnectionsFactory::processSocketEvent(int replica_index, Connection *& connection_out) { + LOG_DEBUG(log, "processSocketEvent"); + resetReplicaTimeouts(replica_index); replicas[replica_index].connection_establisher.run(); processConnectionEstablisherStage(replica_index, true); @@ -396,6 +422,8 @@ void HedgedConnectionsFactory::processSocketEvent(int replica_index, Connection void HedgedConnectionsFactory::processReceiveTimeout(int replica_index) { + LOG_DEBUG(log, "processReceiveTimeout"); + resetReplicaTimeouts(replica_index); ReplicaStatus & replica = replicas[replica_index]; diff --git a/tests/integration/test_distributed_load_balancing/configs/users.xml b/tests/integration/test_distributed_load_balancing/configs/users.xml new file mode 100644 index 00000000000..b2dcdbcd8f3 --- /dev/null +++ b/tests/integration/test_distributed_load_balancing/configs/users.xml @@ -0,0 +1,8 @@ + + + + + 0 + + + diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index df7b74fcae1..d3ac5c132cd 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -166,6 +166,7 @@ def test_load_balancing_priority_round_robin(dist_table): def test_distributed_replica_max_ignored_errors(): settings = { + 'use_hedged_requests' : 0, 'load_balancing': 'in_order', 'prefer_localhost_replica': 0, 'connect_timeout': 2, From a72ef6f026eb955fe43ba9c2d07e3ad6e6646983 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Feb 2021 11:26:24 +0300 Subject: [PATCH 160/510] Fix number of threads for scalar subqueries and subqueries for index. --- .../ExecuteScalarSubqueriesVisitor.cpp | 16 ++++++++++++---- src/Interpreters/ExpressionAnalyzer.cpp | 7 +++++-- .../Executors/PullingAsyncPipelineExecutor.cpp | 7 ++++++- src/Processors/Formats/LazyOutputFormat.cpp | 9 +++++++-- 4 files changed, 30 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index e6061aabe94..7ee7bb1f301 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -21,7 +21,7 @@ #include -#include +#include namespace DB { @@ -122,8 +122,10 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr try { - PullingPipelineExecutor executor(io.pipeline); - if (!executor.pull(block)) + PullingAsyncPipelineExecutor executor(io.pipeline); + while (block.rows() == 0 && executor.pull(block)); + + if (block.rows() == 0) { /// Interpret subquery with empty result as Null literal auto ast_new = std::make_unique(Null()); @@ -132,7 +134,13 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr return; } - if (block.rows() != 1 || executor.pull(block)) + if (block.rows() != 1) + throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); + + Block tmp_block; + while (tmp_block.rows() == 0 && executor.pull(tmp_block)); + + if (tmp_block.rows() != 0) throw Exception("Scalar subquery returned more than one row", ErrorCodes::INCORRECT_RESULT_OF_SCALAR_SUBQUERY); } catch (const Exception & e) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3f65a6f3f58..cea056d6a21 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -54,7 +54,7 @@ #include #include -#include +#include #include namespace DB @@ -321,7 +321,7 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, {}, query_options); auto io = interpreter_subquery->execute(); - PullingPipelineExecutor executor(io.pipeline); + PullingAsyncPipelineExecutor executor(io.pipeline); SetPtr set = std::make_shared(settings.size_limits_for_set, true, context.getSettingsRef().transform_null_in); set->setHeader(executor.getHeader()); @@ -329,6 +329,9 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr Block block; while (executor.pull(block)) { + if (block.rows() == 0) + continue; + /// If the limits have been exceeded, give up and let the default subquery processing actions take place. if (!set->insertFromBlock(block)) return; diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index e4bcf6dc0ab..21741d30dfa 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -133,7 +133,12 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) } chunk.clear(); - data->finish_event.tryWait(milliseconds); + + if (milliseconds) + data->finish_event.tryWait(milliseconds); + else + data->finish_event.wait(); + return true; } diff --git a/src/Processors/Formats/LazyOutputFormat.cpp b/src/Processors/Formats/LazyOutputFormat.cpp index 46287d1cce9..0663ff28f84 100644 --- a/src/Processors/Formats/LazyOutputFormat.cpp +++ b/src/Processors/Formats/LazyOutputFormat.cpp @@ -16,8 +16,13 @@ Chunk LazyOutputFormat::getChunk(UInt64 milliseconds) } Chunk chunk; - if (!queue.tryPop(chunk, milliseconds)) - return {}; + if (milliseconds) + { + if (!queue.tryPop(chunk, milliseconds)) + return {}; + } + else + queue.pop(chunk); if (chunk) info.update(chunk.getNumRows(), chunk.allocatedBytes()); From 10f1432c5cb1dc77c0c31cd960a275480fa380dd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Feb 2021 11:31:17 +0300 Subject: [PATCH 161/510] Added perftest. --- tests/performance/subqueries.xml | 7 +++++++ 1 file changed, 7 insertions(+) create mode 100644 tests/performance/subqueries.xml diff --git a/tests/performance/subqueries.xml b/tests/performance/subqueries.xml new file mode 100644 index 00000000000..f1481a78c7e --- /dev/null +++ b/tests/performance/subqueries.xml @@ -0,0 +1,7 @@ + + create table tab (a UInt32, b UInt32) engine = MergeTree order by (a, b) + insert into tab values (1, 1) + select a, b from tab where (a, b) in (select toUInt32(number) as x, toUInt32(sleep(0.1) + 1) from numbers_mt(16)) settings max_threads = 2, max_block_size = 4 + select a, b from tab where (1, 1) = (select min(toUInt32(number + 1)) as x, min(toUInt32(sleep(0.1) + 1)) from numbers_mt(16)) settings max_threads = 2, max_block_size = 4 + DROP TABLE tab + \ No newline at end of file From a14b6c2650015c379c811646f345c4c66c1d9afd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Feb 2021 12:37:19 +0300 Subject: [PATCH 162/510] Fix trivial count optimization --- src/Interpreters/InterpreterSelectQuery.cpp | 16 ++++++++-------- src/Interpreters/InterpreterSelectQuery.h | 4 ++-- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 45d187c34d5..a99f99cfa13 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -104,8 +104,7 @@ namespace ErrorCodes } /// Assumes `storage` is set and the table filter (row-level security) is not empty. -String InterpreterSelectQuery::generateFilterActions( - ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns) const +String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns) const { // std::cerr << "----- InterpreterSelectQuery::generateFilterActions\n"; // for (const auto & name : prerequisite_columns) @@ -357,7 +356,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( ASTSelectQuery & query = getSelectQuery(); std::shared_ptr table_join = joined_tables.makeTableJoin(query); - ASTPtr row_policy_filter; if (storage) row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); @@ -457,7 +455,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (row_policy_filter) { filter_info = std::make_shared(); - filter_info->column_name = generateFilterActions(filter_info->actions, row_policy_filter, required_columns); + filter_info->column_name = generateFilterActions(filter_info->actions, required_columns); source_header = metadata_snapshot->getSampleBlockForColumns( filter_info->actions->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); } @@ -828,6 +826,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere(); if (does_storage_support_prewhere && settings.optimize_move_to_prewhere) { + std::cerr << "----- Moving row level filter to prewhere\n"; /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. expressions.prewhere_info = std::make_shared( std::move(expressions.filter_info->actions), @@ -1331,7 +1330,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc && (settings.max_parallel_replicas <= 1) && storage && storage->getName() != "MaterializeMySQL" - && !expressions.filter_info + && !row_policy_filter && processing_stage == QueryProcessingStage::FetchColumns && query_analyzer->hasAggregation() && (query_analyzer->aggregates().size() == 1) @@ -1394,9 +1393,9 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (storage) { /// Append columns from the table filter to required - ActionsDAG * row_policy_filter = nullptr; - if (expressions.filter_info) - row_policy_filter = expressions.filter_info->actions.get(); + // ActionsDAG * row_policy_filter = nullptr; + // if (expressions.filter_info) + // row_policy_filter = expressions.filter_info->actions.get(); // else if (expressions.prewhere_info && expressions.prewhere_info->row_level_filter_actions) // row_policy_filter = expressions.prewhere_info->row_level_filter_actions.get(); @@ -1651,6 +1650,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (prewhere_info) { + std::cerr << "-------- filling prewhere info \n"; query_info.prewhere_info = std::make_shared(); query_info.prewhere_info->prewhere_actions = std::make_shared(prewhere_info->prewhere_actions); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index 20cffdf5702..49169c66d1b 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -131,8 +131,7 @@ private: void executeSubqueriesInSetsAndJoins(QueryPlan & query_plan, std::unordered_map & subqueries_for_sets); void executeMergeSorted(QueryPlan & query_plan, const SortDescription & sort_description, UInt64 limit, const std::string & description); - String generateFilterActions( - ActionsDAGPtr & actions, const ASTPtr & row_policy_filter, const Names & prerequisite_columns = {}) const; + String generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns = {}) const; enum class Modificator { @@ -157,6 +156,7 @@ private: /// Is calculated in getSampleBlock. Is used later in readImpl. ExpressionAnalysisResult analysis_result; /// For row-level security. + ASTPtr row_policy_filter; FilterDAGInfoPtr filter_info; QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; From a6322800118f9f9c27b3c239d78707af1025e97d Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 13:53:44 +0300 Subject: [PATCH 163/510] added alias for nulls --- docs/en/sql-reference/functions/functions-for-nulls.md | 2 ++ docs/ru/sql-reference/functions/functions-for-nulls.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/functions-for-nulls.md b/docs/en/sql-reference/functions/functions-for-nulls.md index c32af7194fb..fbbda2c0ecc 100644 --- a/docs/en/sql-reference/functions/functions-for-nulls.md +++ b/docs/en/sql-reference/functions/functions-for-nulls.md @@ -13,6 +13,8 @@ Checks whether the argument is [NULL](../../sql-reference/syntax.md#null-literal isNull(x) ``` +Alias: `ISNULL`. + **Parameters** - `x` — A value with a non-compound data type. diff --git a/docs/ru/sql-reference/functions/functions-for-nulls.md b/docs/ru/sql-reference/functions/functions-for-nulls.md index 17da1ea9194..0db55847631 100644 --- a/docs/ru/sql-reference/functions/functions-for-nulls.md +++ b/docs/ru/sql-reference/functions/functions-for-nulls.md @@ -13,6 +13,8 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438\u0020\u0434\u043b\u044f\u isNull(x) ``` +Синоним: `ISNULL`. + **Параметры** - `x` — значение с не составным типом данных. From bc6fdc7d4b09f290a57f7da39ba4abae2532d7c6 Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:12:12 +0300 Subject: [PATCH 164/510] added aliases for date-time functions --- .../functions/date-time-functions.md | 18 ++++++++++++++++++ .../functions/date-time-functions.md | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 4a73bdb2546..a0c89ecb035 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -61,40 +61,58 @@ int32samoa: 1546300800 Converts a date or date with time to a UInt16 number containing the year number (AD). +Alias: `Year`. + ## toQuarter {#toquarter} Converts a date or date with time to a UInt8 number containing the quarter number. +Alias: `QUARTER`. + ## toMonth {#tomonth} Converts a date or date with time to a UInt8 number containing the month number (1-12). +Alias: `MONTH`. + ## toDayOfYear {#todayofyear} Converts a date or date with time to a UInt16 number containing the number of the day of the year (1-366). +Alias: `DAYOFYEAR`. + ## toDayOfMonth {#todayofmonth} Converts a date or date with time to a UInt8 number containing the number of the day of the month (1-31). +Aliases: `DAYOFMONTH`, `DAY`. + ## toDayOfWeek {#todayofweek} Converts a date or date with time to a UInt8 number containing the number of the day of the week (Monday is 1, and Sunday is 7). +Alias: `DAYOFWEEK`. + ## toHour {#tohour} Converts a date with time to a UInt8 number containing the number of the hour in 24-hour time (0-23). This function assumes that if clocks are moved ahead, it is by one hour and occurs at 2 a.m., and if clocks are moved back, it is by one hour and occurs at 3 a.m. (which is not always true – even in Moscow the clocks were twice changed at a different time). +Alias: `HOUR`. + ## toMinute {#tominute} Converts a date with time to a UInt8 number containing the number of the minute of the hour (0-59). +Alias: `MINUTE`. + ## toSecond {#tosecond} Converts a date with time to a UInt8 number containing the number of the second in the minute (0-59). Leap seconds are not accounted for. +Alias: `SECOND`. + ## toUnixTimestamp {#to-unix-timestamp} For DateTime argument: converts value to the number with type UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time). diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 31482cde77f..add47e9dad1 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -63,40 +63,58 @@ int32samoa: 1546300800 Переводит дату или дату-с-временем в число типа UInt16, содержащее номер года (AD). +Синоним: `Year`. + ## toQuarter {#toquarter} Переводит дату или дату-с-временем в число типа UInt8, содержащее номер квартала. +Синоним: `QUARTER`. + ## toMonth {#tomonth} Переводит дату или дату-с-временем в число типа UInt8, содержащее номер месяца (1-12). +Синоним: `MONTH`. + ## toDayOfYear {#todayofyear} Переводит дату или дату-с-временем в число типа UInt16, содержащее номер дня года (1-366). +Синоним: `DAYOFYEAR`. + ## toDayOfMonth {#todayofmonth} Переводит дату или дату-с-временем в число типа UInt8, содержащее номер дня в месяце (1-31). +Синонимы: `DAYOFMONTH`, `DAY`. + ## toDayOfWeek {#todayofweek} Переводит дату или дату-с-временем в число типа UInt8, содержащее номер дня в неделе (понедельник - 1, воскресенье - 7). +Синоним: `DAYOFWEEK`. + ## toHour {#tohour} Переводит дату-с-временем в число типа UInt8, содержащее номер часа в сутках (0-23). Функция исходит из допущения, что перевод стрелок вперёд, если осуществляется, то на час, в два часа ночи, а перевод стрелок назад, если осуществляется, то на час, в три часа ночи (что, в общем, не верно - даже в Москве два раза перевод стрелок был осуществлён в другое время). +Синоним: `HOUR`. + ## toMinute {#tominute} Переводит дату-с-временем в число типа UInt8, содержащее номер минуты в часе (0-59). +Синоним: `MINUTE`. + ## toSecond {#tosecond} Переводит дату-с-временем в число типа UInt8, содержащее номер секунды в минуте (0-59). Секунды координации не учитываются. +Синоним: `SECOND`. + ## toUnixTimestamp {#to-unix-timestamp} Переводит дату-с-временем в число типа UInt32 -- Unix Timestamp (https://en.wikipedia.org/wiki/Unix_time). From 33e12f7b4a628fdd63f3a30e070cedbb0449473a Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:12:44 +0300 Subject: [PATCH 165/510] added aliases for encoding functions --- docs/en/sql-reference/functions/encoding-functions.md | 2 ++ docs/ru/sql-reference/functions/encoding-functions.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/encoding-functions.md b/docs/en/sql-reference/functions/encoding-functions.md index bc3f5ca4345..3ec6c8ec3dd 100644 --- a/docs/en/sql-reference/functions/encoding-functions.md +++ b/docs/en/sql-reference/functions/encoding-functions.md @@ -75,6 +75,8 @@ Result: Returns a string containing the argument’s hexadecimal representation. +Alias: `HEX`. + **Syntax** ``` sql diff --git a/docs/ru/sql-reference/functions/encoding-functions.md b/docs/ru/sql-reference/functions/encoding-functions.md index 6f1c2aad6cb..8c3065e5a77 100644 --- a/docs/ru/sql-reference/functions/encoding-functions.md +++ b/docs/ru/sql-reference/functions/encoding-functions.md @@ -75,6 +75,8 @@ SELECT char(0xE4, 0xBD, 0xA0, 0xE5, 0xA5, 0xBD) AS hello; Returns a string containing the argument’s hexadecimal representation. +Синоним: `HEX`. + **Syntax** ``` sql From 1bd1a97716264f668659a972861c3f172e3b1cef Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:13:01 +0300 Subject: [PATCH 166/510] added aliases for string functions --- docs/en/sql-reference/functions/string-functions.md | 4 ++++ docs/ru/sql-reference/functions/string-functions.md | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index 2b93dd924a3..c1f3625c14d 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -276,10 +276,14 @@ Returns the string ‘s’ that was converted from the encoding in ‘from’ to Encodes ‘s’ string into base64 +Alias: `TO_BASE64`. + ## base64Decode(s) {#base64decode} Decode base64-encoded string ‘s’ into original string. In case of failure raises an exception. +Alias: `FROM_BASE64`. + ## tryBase64Decode(s) {#trybase64decode} Similar to base64Decode, but in case of error an empty string would be returned. diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index aeb0652cc18..24edc3618fb 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -273,10 +273,14 @@ SELECT concat(key1, key2), sum(value) FROM key_val GROUP BY (key1, key2) Производит кодирование строки s в base64-представление. +Синоним: `TO_BASE64`. + ## base64Decode(s) {#base64decode} Декодирует base64-представление s в исходную строку. При невозможности декодирования выбрасывает исключение +Синоним: `FROM_BASE64`. + ## tryBase64Decode(s) {#trybase64decode} Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. From 3603fbd46a30e5a8f77877de5cac871ebec17564 Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:13:17 +0300 Subject: [PATCH 167/510] added aliases for ip-address functions --- .../sql-reference/functions/ip-address-functions.md | 12 +++++++++++- .../sql-reference/functions/ip-address-functions.md | 12 +++++++++++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/ip-address-functions.md b/docs/en/sql-reference/functions/ip-address-functions.md index 0c1f675304b..8e2939e9272 100644 --- a/docs/en/sql-reference/functions/ip-address-functions.md +++ b/docs/en/sql-reference/functions/ip-address-functions.md @@ -9,10 +9,14 @@ toc_title: IP Addresses Takes a UInt32 number. Interprets it as an IPv4 address in big endian. Returns a string containing the corresponding IPv4 address in the format A.B.C.d (dot-separated numbers in decimal form). +Alias: `INET_NTOA`. + ## IPv4StringToNum(s) {#ipv4stringtonums} The reverse function of IPv4NumToString. If the IPv4 address has an invalid format, it returns 0. +Alias: `INET_ATON`. + ## IPv4NumToStringClassC(num) {#ipv4numtostringclasscnum} Similar to IPv4NumToString, but using xxx instead of the last octet. @@ -49,7 +53,11 @@ Since using ‘xxx’ is highly unusual, this may be changed in the future. We r ### IPv6NumToString(x) {#ipv6numtostringx} Accepts a FixedString(16) value containing the IPv6 address in binary format. Returns a string containing this address in text format. -IPv6-mapped IPv4 addresses are output in the format ::ffff:111.222.33.44. Examples: +IPv6-mapped IPv4 addresses are output in the format ::ffff:111.222.33.44. + +Alias: `INET6_NTOA`. + +Examples: ``` sql SELECT IPv6NumToString(toFixedString(unhex('2A0206B8000000000000000000000011'), 16)) AS addr @@ -119,6 +127,8 @@ The reverse function of IPv6NumToString. If the IPv6 address has an invalid form If the IP address is a valid IPv4 address then the IPv6 equivalent of the IPv4 address is returned. HEX can be uppercase or lowercase. +Alias: `INET6_ATON`. + ``` sql SELECT cutIPv6(IPv6StringToNum('127.0.0.1'), 0, 0); ``` diff --git a/docs/ru/sql-reference/functions/ip-address-functions.md b/docs/ru/sql-reference/functions/ip-address-functions.md index 52f0a92bc9f..3b7379e9a65 100644 --- a/docs/ru/sql-reference/functions/ip-address-functions.md +++ b/docs/ru/sql-reference/functions/ip-address-functions.md @@ -9,10 +9,14 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438\u0020\u0434\u043b\u044f\u Принимает число типа UInt32. Интерпретирует его, как IPv4-адрес в big endian. Возвращает строку, содержащую соответствующий IPv4-адрес в формате A.B.C.D (числа в десятичной форме через точки). +Синоним: `INET_NTOA`. + ## IPv4StringToNum(s) {#ipv4stringtonums} Функция, обратная к IPv4NumToString. Если IPv4 адрес в неправильном формате, то возвращает 0. +Синоним: `INET_ATON`. + ## IPv4NumToStringClassC(num) {#ipv4numtostringclasscnum} Похоже на IPv4NumToString, но вместо последнего октета используется xxx. @@ -49,7 +53,11 @@ LIMIT 10 ### IPv6NumToString(x) {#ipv6numtostringx} Принимает значение типа FixedString(16), содержащее IPv6-адрес в бинарном виде. Возвращает строку, содержащую этот адрес в текстовом виде. -IPv6-mapped IPv4 адреса выводится в формате ::ffff:111.222.33.44. Примеры: +IPv6-mapped IPv4 адреса выводится в формате ::ffff:111.222.33.44. + +Примеры: `INET6_NTOA`. + +Примеры: ``` sql SELECT IPv6NumToString(toFixedString(unhex('2A0206B8000000000000000000000011'), 16)) AS addr @@ -118,6 +126,8 @@ LIMIT 10 Функция, обратная к IPv6NumToString. Если IPv6 адрес в неправильном формате, то возвращает строку из нулевых байт. HEX может быть в любом регистре. +Alias: `INET6_ATON`. + ## IPv4ToIPv6(x) {#ipv4toipv6x} Принимает число типа `UInt32`. Интерпретирует его, как IPv4-адрес в [big endian](https://en.wikipedia.org/wiki/Endianness). Возвращает значение `FixedString(16)`, содержащее адрес IPv6 в двоичном формате. Примеры: From c661760113164e74d7cb5ee5c394de3c57892d6c Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:27:52 +0300 Subject: [PATCH 168/510] fixed a typo --- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index add47e9dad1..85d7c275f27 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -63,7 +63,7 @@ int32samoa: 1546300800 Переводит дату или дату-с-временем в число типа UInt16, содержащее номер года (AD). -Синоним: `Year`. +Синоним: `YEAR`. ## toQuarter {#toquarter} From 8a7d59f0fef99281a935cad8e51f40ff8a7341bc Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:31:24 +0300 Subject: [PATCH 169/510] Added aliases for string function --- docs/en/sql-reference/functions/string-functions.md | 2 ++ docs/ru/sql-reference/functions/string-functions.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/string-functions.md b/docs/en/sql-reference/functions/string-functions.md index c1f3625c14d..a4c127507b7 100644 --- a/docs/en/sql-reference/functions/string-functions.md +++ b/docs/en/sql-reference/functions/string-functions.md @@ -98,6 +98,8 @@ SELECT toValidUTF8('\x61\xF0\x80\x80\x80b') Repeats a string as many times as specified and concatenates the replicated values as a single string. +Alias: `REPEAT`. + **Syntax** ``` sql diff --git a/docs/ru/sql-reference/functions/string-functions.md b/docs/ru/sql-reference/functions/string-functions.md index 24edc3618fb..d01d12ac8d5 100644 --- a/docs/ru/sql-reference/functions/string-functions.md +++ b/docs/ru/sql-reference/functions/string-functions.md @@ -95,6 +95,8 @@ SELECT toValidUTF8('\x61\xF0\x80\x80\x80b') Повторяет строку определенное количество раз и объединяет повторяемые значения в одну строку. +Синоним: `REPEAT`. + **Синтаксис** ``` sql From 4315cd8d26cb838553dc38a38ba35380e0eed767 Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:34:24 +0300 Subject: [PATCH 170/510] fixed a typo --- docs/en/sql-reference/functions/date-time-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index a0c89ecb035..880942a02f9 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -61,7 +61,7 @@ int32samoa: 1546300800 Converts a date or date with time to a UInt16 number containing the year number (AD). -Alias: `Year`. +Alias: `YEAR`. ## toQuarter {#toquarter} From 243ca5fe58d7b12fee746784c2f8a2f36790ff1e Mon Sep 17 00:00:00 2001 From: George Date: Tue, 16 Feb 2021 14:48:28 +0300 Subject: [PATCH 171/510] Added aliases for type conversion functions --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 ++ docs/ru/sql-reference/functions/type-conversion-functions.md | 2 ++ 2 files changed, 4 insertions(+) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 3ca36f41c78..6e21ee9774d 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -124,6 +124,8 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) ## toDate {#todate} +Alias: `DATE`. + ## toDateOrZero {#todateorzero} ## toDateOrNull {#todateornull} diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 4a314bd22d8..022b4c3ebc7 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -124,6 +124,8 @@ SELECT toUInt64(nan), toUInt32(-32), toUInt16('16'), toUInt8(8.8) ## toDate {#todate} +Cиноним: `DATE`. + ## toDateOrZero {#todateorzero} ## toDateOrNull {#todateornull} From bcf30d841262fd9316d0de1760d592c426805c5b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 16 Feb 2021 15:57:00 +0300 Subject: [PATCH 172/510] Try fix tests. --- src/Interpreters/InterpreterSelectQuery.cpp | 36 +++++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a99f99cfa13..826be1e5143 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -826,7 +826,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere(); if (does_storage_support_prewhere && settings.optimize_move_to_prewhere) { - std::cerr << "----- Moving row level filter to prewhere\n"; + // std::cerr << "----- Moving row level filter to prewhere\n"; /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. expressions.prewhere_info = std::make_shared( std::move(expressions.filter_info->actions), @@ -1393,20 +1393,28 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (storage) { /// Append columns from the table filter to required - // ActionsDAG * row_policy_filter = nullptr; - // if (expressions.filter_info) - // row_policy_filter = expressions.filter_info->actions.get(); - // else if (expressions.prewhere_info && expressions.prewhere_info->row_level_filter_actions) - // row_policy_filter = expressions.prewhere_info->row_level_filter_actions.get(); - - if (expressions.filter_info) + if (row_policy_filter) { - auto required_columns_from_filter = expressions.filter_info->actions->getRequiredColumns(); - - for (const auto & column : required_columns_from_filter) + ActionsDAG * row_policy_dag = nullptr; + if (expressions.filter_info) + row_policy_dag = expressions.filter_info->actions.get(); + else if (expressions.prewhere_info) { - if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) - required_columns.push_back(column.name); + if (expressions.prewhere_info->row_level_filter_actions) + row_policy_dag = expressions.prewhere_info->row_level_filter_actions.get(); + else if (expressions.prewhere_info->prewhere_actions) + row_policy_dag = expressions.prewhere_info->prewhere_actions.get(); + } + + if (row_policy_dag) + { + auto required_columns_from_filter = row_policy_dag->getRequiredColumns(); + + for (const auto & column : required_columns_from_filter) + { + if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) + required_columns.push_back(column.name); + } } } @@ -1650,7 +1658,7 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (prewhere_info) { - std::cerr << "-------- filling prewhere info \n"; + // std::cerr << "-------- filling prewhere info \n"; query_info.prewhere_info = std::make_shared(); query_info.prewhere_info->prewhere_actions = std::make_shared(prewhere_info->prewhere_actions); From 16bcd9d247877c55d27936e64a0d3c76dbe9cf7a Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Feb 2021 20:28:54 +0300 Subject: [PATCH 173/510] Add changelog tests --- src/Coordination/Changelog.cpp | 102 ++++--- src/Coordination/Changelog.h | 12 +- src/Coordination/tests/gtest_for_build.cpp | 325 ++++++++++++++++++++- 3 files changed, 396 insertions(+), 43 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index d3ba176f209..6fa3e0e9e03 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -44,15 +44,14 @@ static constexpr auto DEFAULT_PREFIX = "changelog"; struct ChangelogName { std::string prefix; - ChangelogVersion version; size_t from_log_idx; size_t to_log_idx; }; -std::string formatChangelogPath(const std::string & prefix, const ChangelogVersion & version, const ChangelogName & name) +std::string formatChangelogPath(const std::string & prefix, const ChangelogName & name) { std::filesystem::path path(prefix); - path /= std::filesystem::path(name.prefix + "_" + toString(version) + "_" + std::to_string(name.from_log_idx) + "_" + std::to_string(name.to_log_idx) + ".log"); + path /= std::filesystem::path(name.prefix + "_" + std::to_string(name.from_log_idx) + "_" + std::to_string(name.to_log_idx) + ".bin"); return path; } @@ -62,14 +61,13 @@ ChangelogName getChangelogName(const std::string & path_str) std::string filename = path.stem(); Strings filename_parts; boost::split(filename_parts, filename, boost::is_any_of("_")); - if (filename_parts.size() < 4) + if (filename_parts.size() < 3) throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path_str); ChangelogName result; result.prefix = filename_parts[0]; - result.version = fromString(filename_parts[1]); - result.from_log_idx = parse(filename_parts[2]); - result.to_log_idx = parse(filename_parts[3]); + result.from_log_idx = parse(filename_parts[1]); + result.to_log_idx = parse(filename_parts[2]); return result; } @@ -114,6 +112,7 @@ public: { flush(); plain_buf.truncate(new_length); + plain_buf.seek(new_length, SEEK_SET); } void flush() @@ -190,6 +189,7 @@ public: if (!logs.try_emplace(record.header.index, log_entry).second) throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filepath); } + return total_read; } private: @@ -203,13 +203,16 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval { namespace fs = std::filesystem; for(const auto & p : fs::directory_iterator(changelogs_dir)) - existing_changelogs.push_back(p.path()); + { + auto name = getChangelogName(p.path()); + existing_changelogs[name.from_log_idx] = p.path(); + } } void Changelog::readChangelogAndInitWriter(size_t from_log_idx) { size_t read_from_last = 0; - for (const std::string & changelog_file : existing_changelogs) + for (const auto & [start_id, changelog_file] : existing_changelogs) { ChangelogName parsed_name = getChangelogName(changelog_file); if (parsed_name.to_log_idx >= from_log_idx) @@ -223,8 +226,9 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) if (existing_changelogs.size() > 0 && read_from_last < rotate_interval) { - auto parsed_name = getChangelogName(existing_changelogs.back()); - current_writer = std::make_unique(existing_changelogs.back(), WriteMode::Append, parsed_name.from_log_idx); + auto str_name = existing_changelogs.rbegin()->second; + auto parsed_name = getChangelogName(str_name); + current_writer = std::make_unique(str_name, WriteMode::Append, parsed_name.from_log_idx); current_writer->setEntriesWritten(read_from_last); } else @@ -240,13 +244,12 @@ void Changelog::rotate(size_t new_start_log_idx) ChangelogName new_name; new_name.prefix = DEFAULT_PREFIX; - new_name.version = CURRENT_CHANGELOG_VERSION; new_name.from_log_idx = new_start_log_idx; - new_name.to_log_idx = new_start_log_idx; + new_name.to_log_idx = new_start_log_idx + rotate_interval - 1; - auto new_log_path = formatChangelogPath(changelogs_dir, CURRENT_CHANGELOG_VERSION, new_name); - existing_changelogs.push_back(new_log_path); - current_writer = std::make_unique(existing_changelogs.back(), WriteMode::Rewrite, new_start_log_idx); + auto new_log_path = formatChangelogPath(changelogs_dir, new_name); + existing_changelogs[new_start_log_idx] = new_log_path; + current_writer = std::make_unique(new_log_path, WriteMode::Rewrite, new_start_log_idx); } ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) const @@ -275,42 +278,62 @@ void Changelog::appendEntry(size_t index, nuraft::ptr log_ent if (!current_writer) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); + if (logs.empty()) + start_index = index; + if (current_writer->getEntriesWritten() == rotate_interval) rotate(index); auto offset = current_writer->appendRecord(buildRecord(index, log_entry), true); if (!index_to_start_pos.try_emplace(index, offset).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "Record with index {} already exists", index); + logs[index] = makeClone(log_entry); } void Changelog::writeAt(size_t index, nuraft::ptr log_entry) { - if (index < current_writer->getStartIndex()) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Currently cannot overwrite index from previous file"); - if (index_to_start_pos.count(index) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); + bool need_rollback = index < current_writer->getStartIndex(); + if (need_rollback) + { + auto index_changelog = existing_changelogs.lower_bound(index); + std::string fname; + if (index_changelog->first == index) + fname = index_changelog->second; + else + fname = std::prev(index_changelog)->second; + + current_writer = std::make_unique(fname, WriteMode::Append, index_changelog->first); + auto formated_name = getChangelogName(fname); + current_writer->setEntriesWritten(formated_name.to_log_idx - formated_name.from_log_idx + 1); + } + auto entries_written = current_writer->getEntriesWritten(); current_writer->truncateToLength(index_to_start_pos[index]); - for (auto itr = index_to_start_pos.begin(); itr != index_to_start_pos.end();) + + if (need_rollback) { - if (itr->first >= index) + auto to_remove_itr = existing_changelogs.upper_bound(index); + for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - entries_written--; - itr = index_to_start_pos.erase(itr); + std::filesystem::remove(itr->second); + itr = existing_changelogs.erase(itr); } - else - itr++; + } + + /// Rollback in memory state + for (auto itr = logs.lower_bound(index); itr != logs.end();) + { + index_to_start_pos.erase(itr->first); + itr = logs.erase(itr); + entries_written--; } current_writer->setEntriesWritten(entries_written); - auto itr = logs.lower_bound(index); - while (itr != logs.end()) - itr = logs.erase(itr); - appendEntry(index, log_entry); } @@ -318,22 +341,27 @@ void Changelog::compact(size_t up_to_log_idx) { for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();) { - ChangelogName parsed_name = getChangelogName(*itr); + ChangelogName parsed_name = getChangelogName(itr->second); if (parsed_name.to_log_idx <= up_to_log_idx) { - std::filesystem::remove(*itr); - itr = existing_changelogs.erase(itr); + for (size_t idx = parsed_name.from_log_idx; idx <= parsed_name.to_log_idx; ++idx) { - auto logs_itr = logs.find(idx); - if (logs_itr != logs.end()) - logs.erase(idx); - else + auto index_pos = index_to_start_pos.find(idx); + if (index_pos == index_to_start_pos.end()) break; - index_to_start_pos.erase(idx); + index_to_start_pos.erase(index_pos); } + std::filesystem::remove(itr->second); + itr = existing_changelogs.erase(itr); } + else + break; } + auto start = logs.begin(); + auto end = logs.upper_bound(up_to_log_idx); + logs.erase(start, end); + start_index = up_to_log_idx + 1; } LogEntryPtr Changelog::getLastEntry() const diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index c58f35cb4a1..97669d1aa19 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -65,7 +65,7 @@ public: size_t getNextEntryIndex() const { - return start_index + logs.size() - 1; + return start_index + logs.size(); } size_t getStartIndex() const @@ -79,22 +79,28 @@ public: LogEntryPtr entryAt(size_t idx); - nuraft::ptr serializeEntriesToBuffer(size_t index, Int32 cnt); + nuraft::ptr serializeEntriesToBuffer(size_t index, int32_t cnt); void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer); void flush(); + size_t size() const + { + return logs.size(); + } + ~Changelog(); private: + void rotate(size_t new_start_log_idex); ChangelogRecord buildRecord(size_t index, nuraft::ptr log_entry) const; private: std::string changelogs_dir; - std::deque existing_changelogs; + std::map existing_changelogs; std::unique_ptr current_writer; IndexToOffset index_to_start_pos; const size_t rotate_interval; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 6142ee0b5c0..6335df4b940 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -24,6 +24,7 @@ #include #include #include +#include TEST(CoordinationTest, BuildTest) @@ -335,18 +336,336 @@ TEST(CoordinationTest, TestStorageSerialization) EXPECT_EQ(new_storage.ephemerals[1].size(), 1); } -DB::LogEntryPtr getLogEntry(const std::string & s) +DB::LogEntryPtr getLogEntry(const std::string & s, size_t term) { DB::WriteBufferFromNuraftBuffer bufwriter; writeText(s, bufwriter); - return nuraft::cs_new(0, bufwriter.getBuffer()); + return nuraft::cs_new(term, bufwriter.getBuffer()); } +namespace fs = std::filesystem; +struct ChangelogDirTest +{ + std::string path; + bool drop; + ChangelogDirTest(std::string path_, bool drop_ = true) + : path(path_) + , drop(drop_) + { + if (fs::exists(path)) + EXPECT_TRUE(false) << "Path " << path << " already exists, remove it to run test"; + fs::create_directory(path); + } + + ~ChangelogDirTest() + { + if (fs::exists(path) && drop) + fs::remove_all(path); + } +}; + TEST(CoordinationTest, ChangelogTestSimple) { + ChangelogDirTest test("./logs"); DB::Changelog changelog("./logs", 5); - auto entry = getLogEntry("hello world"); + changelog.readChangelogAndInitWriter(1); + auto entry = getLogEntry("hello world", 77); changelog.appendEntry(1, entry); + EXPECT_EQ(changelog.getNextEntryIndex(), 2); + EXPECT_EQ(changelog.getStartIndex(), 1); + EXPECT_EQ(changelog.getLastEntry()->get_term(), 77); + EXPECT_EQ(changelog.entryAt(1)->get_term(), 77); + EXPECT_EQ(changelog.getLogEntriesBetween(1, 2)->size(), 1); +} + +TEST(CoordinationTest, ChangelogTestFile) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 5); + changelog.readChangelogAndInitWriter(1); + auto entry = getLogEntry("hello world", 77); + changelog.appendEntry(1, entry); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + for(const auto & p : fs::directory_iterator("./logs")) + EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin"); + + changelog.appendEntry(2, entry); + changelog.appendEntry(3, entry); + changelog.appendEntry(4, entry); + changelog.appendEntry(5, entry); + changelog.appendEntry(6, entry); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); +} + +TEST(CoordinationTest, ChangelogReadWrite) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 1000); + changelog.readChangelogAndInitWriter(1); + for (size_t i = 0; i < 10; ++i) + { + auto entry = getLogEntry("hello world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + EXPECT_EQ(changelog.size(), 10); + DB::Changelog changelog_reader("./logs", 1000); + changelog_reader.readChangelogAndInitWriter(1); + EXPECT_EQ(changelog_reader.size(), 10); + EXPECT_EQ(changelog_reader.getLastEntry()->get_term(), changelog.getLastEntry()->get_term()); + EXPECT_EQ(changelog_reader.getStartIndex(), changelog.getStartIndex()); + EXPECT_EQ(changelog_reader.getNextEntryIndex(), changelog.getNextEntryIndex()); + + for (size_t i = 0; i < 10; ++i) + EXPECT_EQ(changelog_reader.entryAt(i + 1)->get_term(), changelog.entryAt(i + 1)->get_term()); + + auto entries_from_range_read = changelog_reader.getLogEntriesBetween(1, 11); + auto entries_from_range = changelog.getLogEntriesBetween(1, 11); + EXPECT_EQ(entries_from_range_read->size(), entries_from_range->size()); + EXPECT_EQ(10, entries_from_range->size()); +} + +TEST(CoordinationTest, ChangelogWriteAt) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 1000); + changelog.readChangelogAndInitWriter(1); + for (size_t i = 0; i < 10; ++i) + { + auto entry = getLogEntry("hello world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + EXPECT_EQ(changelog.size(), 10); + + auto entry = getLogEntry("writer", 77); + changelog.writeAt(7, entry); + EXPECT_EQ(changelog.size(), 7); + EXPECT_EQ(changelog.getLastEntry()->get_term(), 77); + EXPECT_EQ(changelog.entryAt(7)->get_term(), 77); + EXPECT_EQ(changelog.getNextEntryIndex(), 8); + + DB::Changelog changelog_reader("./logs", 1000); + changelog_reader.readChangelogAndInitWriter(1); + + EXPECT_EQ(changelog_reader.size(), changelog.size()); + EXPECT_EQ(changelog_reader.getLastEntry()->get_term(), changelog.getLastEntry()->get_term()); + EXPECT_EQ(changelog_reader.getStartIndex(), changelog.getStartIndex()); + EXPECT_EQ(changelog_reader.getNextEntryIndex(), changelog.getNextEntryIndex()); +} + + +TEST(CoordinationTest, ChangelogTestAppendAfterRead) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 5); + changelog.readChangelogAndInitWriter(1); + for (size_t i = 0; i < 7; ++i) + { + auto entry = getLogEntry("hello world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + + EXPECT_EQ(changelog.size(), 7); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + + DB::Changelog changelog_reader("./logs", 5); + changelog_reader.readChangelogAndInitWriter(1); + + EXPECT_EQ(changelog_reader.size(), 7); + for (size_t i = 7; i < 10; ++i) + { + auto entry = getLogEntry("hello world", i * 10); + changelog_reader.appendEntry(changelog_reader.getNextEntryIndex(), entry); + } + EXPECT_EQ(changelog_reader.size(), 10); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + + size_t logs_count = 0; + for(const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + logs_count++; + + EXPECT_EQ(logs_count, 2); + + auto entry = getLogEntry("someentry", 77); + changelog_reader.appendEntry(changelog_reader.getNextEntryIndex(), entry); + EXPECT_EQ(changelog_reader.size(), 11); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + + logs_count = 0; + for(const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + logs_count++; + + EXPECT_EQ(logs_count, 3); +} + +TEST(CoordinationTest, ChangelogTestCompaction) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 5); + changelog.readChangelogAndInitWriter(1); + + for (size_t i = 0; i < 3; ++i) + { + auto entry = getLogEntry("hello world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + + EXPECT_EQ(changelog.size(), 3); + + changelog.compact(2); + + EXPECT_EQ(changelog.size(), 1); + EXPECT_EQ(changelog.getStartIndex(), 3); + EXPECT_EQ(changelog.getNextEntryIndex(), 4); + EXPECT_EQ(changelog.getLastEntry()->get_term(), 20); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + + changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 30)); + changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 40)); + changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 50)); + changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 60)); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + + changelog.compact(6); + + EXPECT_FALSE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + + EXPECT_EQ(changelog.size(), 1); + EXPECT_EQ(changelog.getStartIndex(), 7); + EXPECT_EQ(changelog.getNextEntryIndex(), 8); + EXPECT_EQ(changelog.getLastEntry()->get_term(), 60); + /// And we able to read it + DB::Changelog changelog_reader("./logs", 5); + changelog_reader.readChangelogAndInitWriter(7); + EXPECT_EQ(changelog_reader.size(), 1); + EXPECT_EQ(changelog_reader.getStartIndex(), 7); + EXPECT_EQ(changelog_reader.getNextEntryIndex(), 8); + EXPECT_EQ(changelog_reader.getLastEntry()->get_term(), 60); +} + +TEST(CoordinationTest, ChangelogTestBatchOperations) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 100); + changelog.readChangelogAndInitWriter(1); + for (size_t i = 0; i < 10; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + + EXPECT_EQ(changelog.size(), 10); + + auto entries = changelog.serializeEntriesToBuffer(1, 5); + + DB::Changelog apply_changelog("./logs", 100); + apply_changelog.readChangelogAndInitWriter(1); + + for (size_t i = 0; i < 10; ++i) + { + EXPECT_EQ(apply_changelog.entryAt(i + 1)->get_term(), i * 10); + } + EXPECT_EQ(apply_changelog.size(), 10); + + apply_changelog.applyEntriesFromBuffer(8, *entries); + + EXPECT_EQ(apply_changelog.size(), 12); + EXPECT_EQ(apply_changelog.getStartIndex(), 1); + EXPECT_EQ(apply_changelog.getNextEntryIndex(), 13); + + for (size_t i = 0; i < 7; ++i) + { + EXPECT_EQ(apply_changelog.entryAt(i + 1)->get_term(), i * 10); + } + + EXPECT_EQ(apply_changelog.entryAt(8)->get_term(), 0); + EXPECT_EQ(apply_changelog.entryAt(9)->get_term(), 10); + EXPECT_EQ(apply_changelog.entryAt(10)->get_term(), 20); + EXPECT_EQ(apply_changelog.entryAt(11)->get_term(), 30); + EXPECT_EQ(apply_changelog.entryAt(12)->get_term(), 40); +} + +TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 100); + changelog.readChangelogAndInitWriter(1); + for (size_t i = 0; i < 10; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + + EXPECT_EQ(changelog.size(), 10); + + auto entries = changelog.serializeEntriesToBuffer(5, 5); + + ChangelogDirTest test1("./logs1"); + DB::Changelog changelog_new("./logs1", 100); + changelog_new.readChangelogAndInitWriter(1); + EXPECT_EQ(changelog_new.size(), 0); + + changelog_new.applyEntriesFromBuffer(5, *entries); + + EXPECT_EQ(changelog_new.size(), 5); + EXPECT_EQ(changelog_new.getStartIndex(), 5); + EXPECT_EQ(changelog_new.getNextEntryIndex(), 10); + + for (size_t i = 4; i < 9; ++i) + EXPECT_EQ(changelog_new.entryAt(i + 1)->get_term(), i * 10); + + changelog_new.appendEntry(changelog_new.getNextEntryIndex(), getLogEntry("hello_world", 110)); + EXPECT_EQ(changelog_new.size(), 6); + EXPECT_EQ(changelog_new.getStartIndex(), 5); + EXPECT_EQ(changelog_new.getNextEntryIndex(), 11); + + DB::Changelog changelog_reader("./logs1", 100); + changelog_reader.readChangelogAndInitWriter(5); +} + + +TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 5); + changelog.readChangelogAndInitWriter(1); + + for (size_t i = 0; i < 33; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + EXPECT_EQ(changelog.size(), 33); + + changelog.writeAt(7, getLogEntry("helloworld", 5555)); + EXPECT_EQ(changelog.size(), 7); + EXPECT_EQ(changelog.getStartIndex(), 1); + EXPECT_EQ(changelog.getNextEntryIndex(), 8); + EXPECT_EQ(changelog.getLastEntry()->get_term(), 5555); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + + EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_11_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); + + DB::Changelog changelog_read("./logs", 5); + changelog_read.readChangelogAndInitWriter(1); + EXPECT_EQ(changelog_read.size(), 7); + EXPECT_EQ(changelog_read.getStartIndex(), 1); + EXPECT_EQ(changelog_read.getNextEntryIndex(), 8); + EXPECT_EQ(changelog_read.getLastEntry()->get_term(), 5555); } #endif From b029f3e5cf4b03df444ee2da007040756cb46570 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Feb 2021 20:32:35 +0300 Subject: [PATCH 174/510] Border test --- src/Coordination/tests/gtest_for_build.cpp | 36 ++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 6335df4b940..f6139ea5de3 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -668,4 +668,40 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_EQ(changelog_read.getLastEntry()->get_term(), 5555); } +TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) +{ + ChangelogDirTest test("./logs"); + DB::Changelog changelog("./logs", 5); + changelog.readChangelogAndInitWriter(1); + + for (size_t i = 0; i < 33; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.appendEntry(changelog.getNextEntryIndex(), entry); + } + EXPECT_EQ(changelog.size(), 33); + + changelog.writeAt(11, getLogEntry("helloworld", 5555)); + EXPECT_EQ(changelog.size(), 11); + EXPECT_EQ(changelog.getStartIndex(), 1); + EXPECT_EQ(changelog.getNextEntryIndex(), 12); + EXPECT_EQ(changelog.getLastEntry()->get_term(), 5555); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_11_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); + + DB::Changelog changelog_read("./logs", 5); + changelog_read.readChangelogAndInitWriter(1); + EXPECT_EQ(changelog_read.size(), 11); + EXPECT_EQ(changelog_read.getStartIndex(), 1); + EXPECT_EQ(changelog_read.getNextEntryIndex(), 12); + EXPECT_EQ(changelog_read.getLastEntry()->get_term(), 5555); +} + #endif From b76b8013ba88b081362ab9f31c103a3b6c77bc27 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Feb 2021 20:47:12 +0300 Subject: [PATCH 175/510] Fix tests --- src/Coordination/Changelog.cpp | 1 - src/Coordination/tests/gtest_for_build.cpp | 22 ++++++++++++++++++++-- 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 6fa3e0e9e03..5198382e731 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -344,7 +344,6 @@ void Changelog::compact(size_t up_to_log_idx) ChangelogName parsed_name = getChangelogName(itr->second); if (parsed_name.to_log_idx <= up_to_log_idx) { - for (size_t idx = parsed_name.from_log_idx; idx <= parsed_name.to_log_idx; ++idx) { auto index_pos = index_to_start_pos.find(idx); diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index f6139ea5de3..fa8ae8f8b82 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -643,6 +643,15 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); changelog.appendEntry(changelog.getNextEntryIndex(), entry); } + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin")); + EXPECT_EQ(changelog.size(), 33); changelog.writeAt(7, getLogEntry("helloworld", 5555)); @@ -656,7 +665,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); - EXPECT_FALSE(fs::exists("./logs/changelog_11_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); @@ -679,6 +688,15 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); changelog.appendEntry(changelog.getNextEntryIndex(), entry); } + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin")); + EXPECT_EQ(changelog.size(), 33); changelog.writeAt(11, getLogEntry("helloworld", 5555)); @@ -692,7 +710,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); - EXPECT_FALSE(fs::exists("./logs/changelog_11_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); From e93e1911ee0b11278e13a2deb8022bbb456ef15d Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 16 Feb 2021 21:01:36 +0300 Subject: [PATCH 176/510] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../functions/type-conversion-functions.md | 14 +- .../functions/type-conversion-functions.md | 172 ++++++++++++++++++ 2 files changed, 177 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 81b5649db32..6795b31bd33 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -701,21 +701,19 @@ parseDateTimeBestEffortUSOrNull(time_string[, time_zone]) **Parameters** -- `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`). [String](../../sql-reference/data-types/string.md). +- `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`, etc). [String](../../sql-reference/data-types/string.md). - `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). **Supported non-standard formats** - A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). - A string with a date and a time components: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. -- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. +- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY`, etc. - A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted with `2000-01`. - A string that includes date and time along with timezone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. **Returned values** -Possible values: - - `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type. - `NULL` if the input string cannot be converted to the `DateTime` data type. @@ -789,23 +787,21 @@ parseDateTimeBestEffortUSOrZero(time_string[, time_zone]) **Parameters** -- `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`). [String](../../sql-reference/data-types/string.md). +- `time_string` — String containing a date or date with time to convert. The date must be in the US date format (`MM/DD/YYYY`, etc). [String](../../sql-reference/data-types/string.md). - `time_zone` — [Timezone](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). The function parses `time_string` according to the timezone. Optional. [String](../../sql-reference/data-types/string.md). **Supported non-standard formats** - A string containing 9..10 digit [unix timestamp](https://en.wikipedia.org/wiki/Unix_time). - A string with a date and a time components: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss`, etc. -- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` etc. +- A string with a date, but no time component: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY`, etc. - A string with a day and time: `DD`, `DD hh`, `DD hh:mm`. In this case, `YYYY-MM` are substituted with `2000-01`. - A string that includes date and time along with timezone offset information: `YYYY-MM-DD hh:mm:ss ±h:mm`, etc. For example, `2020-12-12 17:36:00 -5:00`. **Returned values** -Possible values: - - `time_string` converted to the [DateTime](../../sql-reference/data-types/datetime.md) data type. -- `zero date time`. +- Zero date or zero date with time if the input string cannot be converted to the `DateTime` data type. **Examples** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 4a314bd22d8..92e674242df 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -658,6 +658,178 @@ AS parseDateTimeBestEffortUS; └─────────────────────────——┘ ``` +## parseDateTimeBestEffortUSOrNull {#parsedatetimebesteffortusornull} + +Похожа на функцию [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но разница состоит в том, что возвращает `NULL`, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). + +**Синтаксис** + +``` sql +parseDateTimeBestEffortUSOrNull(time_string[, time_zone]) +``` + +**Параметры** + +- `time_string` — строка, содержащая дату или дату со временем для преобразования. Дата должна быть в американском формате (`MM/DD/YYYY` и т.д.). [String](../../sql-reference/data-types/string.md). +- `time_zone` — [часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). Функция анализирует `time_string` в соответствии с заданным часовым поясом. Опциональный параметр. [String](../../sql-reference/data-types/string.md). + +**Поддерживаемые нестандартные форматы** + +- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 символов. +- Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss` и т.д. +- Строка, содержащая дату без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` и т.д. +- Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. +- Строка, содержащая дату и время, а также информацию о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm` и т.д. Например, `2020-12-12 17:36:00 -5:00`. + +**Возвращаемые значения** + +- `time_string`, преобразованная в тип данных `DateTime`. +- `NULL`, если входная строка не может быть преобразована в тип данных `DateTime`. + +**Примеры** + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02/10/2021 21:12:57') AS parseDateTimeBestEffortUSOrNull; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02-10-2021 21:12:57 GMT', 'Europe/Moscow') AS parseDateTimeBestEffortUSOrNull; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-11 00:12:57 │ +└─────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('02.10.2021') AS parseDateTimeBestEffortUSOrNull; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ 2021-02-10 00:00:00 │ +└─────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrNull('10.2021') AS parseDateTimeBestEffortUSOrNull; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrNull─┐ +│ ᴺᵁᴸᴸ │ +└─────────────────────────────────┘ +``` + +## parseDateTimeBestEffortUSOrZero {#parsedatetimebesteffortusorzero} + +Похожа на функцию [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но разница в том, что возвращает нулевую дату или нулевую дату со временем, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). + +**Синтаксис** + +``` sql +parseDateTimeBestEffortUSOrZero(time_string[, time_zone]) +``` + +**Параметры** + +- `time_string` — строка, содержащая дату или дату со временем для преобразования. Дата должна быть в американском формате (`MM/DD/YYYY` и т.д.). [String](../../sql-reference/data-types/string.md). +- `time_zone` — [часовой пояс](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone). Функция анализирует `time_string` в соответствии с заданным часовым поясом. Опциональный параметр. [String](../../sql-reference/data-types/string.md). + +**Поддерживаемые нестандартные форматы** + +- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 символов. +- Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss` и т.д. +- Строка, содержащая дату без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` и т.д. +- Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. +- Строка, содержащая дату и время, а также информацию о часовом поясе: `YYYY-MM-DD hh:mm:ss ±h:mm` и т.д. Например, `2020-12-12 17:36:00 -5:00`. + +**Возвращаемые значения** + +- `time_string`, преобразованная в тип данных `DateTime`. +- Нулевая дата или нулевая дата со временем, если входная строка не может быть преобразована в тип данных `DateTime`. + +**Примеры** + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02/10/2021 21:12:57') AS parseDateTimeBestEffortUSOrZero; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 2021-02-10 21:12:57 │ +└─────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02-10-2021 21:12:57 GMT', 'Europe/Moscow') AS parseDateTimeBestEffortUSOrZero; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 2021-02-11 00:12:57 │ +└─────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02.10.2021') AS parseDateTimeBestEffortUSOrZero; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 2021-02-10 00:00:00 │ +└─────────────────────────────────┘ +``` + +Запрос: + +``` sql +SELECT parseDateTimeBestEffortUSOrZero('02.2021') AS parseDateTimeBestEffortUSOrZero; +``` + +Результат: + +``` text +┌─parseDateTimeBestEffortUSOrZero─┐ +│ 1970-01-01 00:00:00 │ +└─────────────────────────────────┘ +``` + ## toUnixTimestamp64Milli ## toUnixTimestamp64Micro ## toUnixTimestamp64Nano From d3e87701d478c2f779eae5b892c040b1132d8b6c Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Feb 2021 22:02:18 +0300 Subject: [PATCH 177/510] Persistent storage --- src/Coordination/Changelog.cpp | 10 ++-- src/Coordination/Changelog.h | 2 - src/Coordination/CoordinationSettings.h | 3 +- src/Coordination/InMemoryStateManager.cpp | 21 ++++--- src/Coordination/InMemoryStateManager.h | 13 +++-- src/Coordination/NuKeeperServer.cpp | 12 +++- src/Coordination/tests/gtest_for_build.cpp | 67 +++++++++++----------- tests/config/config.d/test_keeper_port.xml | 1 + 8 files changed, 77 insertions(+), 52 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 5198382e731..e4d8b13ec37 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -16,10 +16,8 @@ namespace ErrorCodes extern const int CORRUPTED_DATA; extern const int UNKNOWN_FORMAT_VERSION; extern const int LOGICAL_ERROR; - extern const int NOT_IMPLEMENTED; } - std::string toString(const ChangelogVersion & version) { if (version == ChangelogVersion::V0) @@ -147,7 +145,6 @@ private: size_t start_index; }; - class ChangelogReader { public: @@ -202,7 +199,10 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval , rotate_interval(rotate_interval_) { namespace fs = std::filesystem; - for(const auto & p : fs::directory_iterator(changelogs_dir)) + if (!fs::exists(changelogs_dir)) + fs::create_directories(changelogs_dir); + + for (const auto & p : fs::directory_iterator(changelogs_dir)) { auto name = getChangelogName(p.path()); existing_changelogs[name.from_log_idx] = p.path(); @@ -233,7 +233,7 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) } else { - rotate(from_log_idx); + rotate(start_index); } } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 97669d1aa19..7c352e7a91b 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -45,8 +45,6 @@ struct ChangelogRecord nuraft::ptr blob; }; - - class ChangelogWriter; class Changelog diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 441e1a5936f..0f1afb3fffe 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -28,7 +28,8 @@ struct Settings; M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \ M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \ - M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) + M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ + M(UInt64, rotate_log_storage_interval, 500000, "How many records will be stored in one log storage file", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp index 69e93578cc1..6c4e95b993a 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/InMemoryStateManager.cpp @@ -9,10 +9,10 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port) +InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) : my_server_id(server_id_) , my_port(port) - , log_store(nuraft::cs_new()) + , log_store(nuraft::cs_new(logs_path, 5000)) , cluster_config(nuraft::cs_new()) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); @@ -22,17 +22,19 @@ InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & h InMemoryStateManager::InMemoryStateManager( int my_server_id_, const std::string & config_prefix, - const Poco::Util::AbstractConfiguration & config) + const Poco::Util::AbstractConfiguration & config, + const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) - , log_store(nuraft::cs_new()) + , log_store(nuraft::cs_new(config.getString(config_prefix + ".log_storage_path"), coordination_settings->rotate_log_storage_interval)) , cluster_config(nuraft::cs_new()) { + Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix, keys); + config.keys(config_prefix + ".raft_configuration", keys); for (const auto & server_key : keys) { - std::string full_prefix = config_prefix + "." + server_key; + std::string full_prefix = config_prefix + ".raft_configuration." + server_key; int server_id = config.getInt(full_prefix + ".id"); std::string hostname = config.getString(full_prefix + ".hostname"); int port = config.getInt(full_prefix + ".port"); @@ -53,12 +55,17 @@ InMemoryStateManager::InMemoryStateManager( cluster_config->get_servers().push_back(peer_config); } if (!my_server_config) - throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section"); + throw Exception(ErrorCodes::RAFT_ERROR, "Our server id {} not found in raft_configuration section", my_server_id); if (start_as_follower_servers.size() == cluster_config->get_servers().size()) throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); } +void InMemoryStateManager::loadLogStore(size_t start_log_index) +{ + log_store->init(start_log_index); +} + void InMemoryStateManager::save_config(const nuraft::cluster_config & config) { // Just keep in memory in this example. diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h index 2a5c2f00dba..8a7be7d0129 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/InMemoryStateManager.h @@ -2,7 +2,8 @@ #include #include -#include +#include +#include #include // Y_IGNORE #include @@ -15,12 +16,16 @@ public: InMemoryStateManager( int server_id_, const std::string & config_prefix, - const Poco::Util::AbstractConfiguration & config); + const Poco::Util::AbstractConfiguration & config, + const CoordinationSettingsPtr & coordination_settings); InMemoryStateManager( int server_id_, const std::string & host, - int port); + int port, + const std::string & logs_path); + + void loadLogStore(size_t start_log_index); nuraft::ptr load_config() override { return cluster_config; } @@ -49,7 +54,7 @@ private: int my_server_id; int my_port; std::unordered_set start_as_follower_servers; - nuraft::ptr log_store; + nuraft::ptr log_store; nuraft::ptr my_server_config; nuraft::ptr cluster_config; nuraft::ptr server_state; diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 7464a06e86f..a4582a5fbb8 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -26,13 +26,16 @@ NuKeeperServer::NuKeeperServer( : server_id(server_id_) , coordination_settings(coordination_settings_) , state_machine(nuraft::cs_new(responses_queue_, coordination_settings)) - , state_manager(nuraft::cs_new(server_id, "test_keeper_server.raft_configuration", config)) + , state_manager(nuraft::cs_new(server_id, "test_keeper_server", config, coordination_settings)) , responses_queue(responses_queue_) { } void NuKeeperServer::startup() { + + state_manager->loadLogStore(state_machine->last_commit_index()); + nuraft::raft_params params; params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); @@ -172,6 +175,13 @@ void NuKeeperServer::waitInit() int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds(); if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag; })) throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization"); + + /// TODO FIXME somehow + while (isLeader() && raft_instance->get_committed_log_idx() != raft_instance->get_last_log_idx()) + { + LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Loading from log store {}/{}", raft_instance->get_committed_log_idx(), raft_instance->get_last_log_idx()); + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + } } std::unordered_set NuKeeperServer::getDeadSessions() diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index fa8ae8f8b82..6d91ba95111 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -26,6 +26,26 @@ #include #include +namespace fs = std::filesystem; +struct ChangelogDirTest +{ + std::string path; + bool drop; + ChangelogDirTest(std::string path_, bool drop_ = true) + : path(path_) + , drop(drop_) + { + if (fs::exists(path)) + EXPECT_TRUE(false) << "Path " << path << " already exists, remove it to run test"; + fs::create_directory(path); + } + + ~ChangelogDirTest() + { + if (fs::exists(path) && drop) + fs::remove_all(path); + } +}; TEST(CoordinationTest, BuildTest) { @@ -70,14 +90,15 @@ TEST(CoordinationTest, BufferSerde) template struct SimpliestRaftServer { - SimpliestRaftServer(int server_id_, const std::string & hostname_, int port_) + SimpliestRaftServer(int server_id_, const std::string & hostname_, int port_, const std::string & logs_path) : server_id(server_id_) , hostname(hostname_) , port(port_) , endpoint(hostname + ":" + std::to_string(port)) , state_machine(nuraft::cs_new()) - , state_manager(nuraft::cs_new(server_id, hostname, port)) + , state_manager(nuraft::cs_new(server_id, hostname, port, logs_path)) { + state_manager->loadLogStore(1); nuraft::raft_params params; params.heart_beat_interval_ = 100; params.election_timeout_lower_bound_ = 200; @@ -126,7 +147,7 @@ struct SimpliestRaftServer nuraft::ptr state_machine; // State manager. - nuraft::ptr state_manager; + nuraft::ptr state_manager; // Raft launcher. nuraft::raft_launcher launcher; @@ -141,7 +162,6 @@ nuraft::ptr getBuffer(int64_t number) { nuraft::ptr ret = nuraft::buffer::alloc(sizeof(number)); nuraft::buffer_serializer bs(ret); - // WARNING: We don't consider endian-safety in this example. bs.put_raw(&number, sizeof(number)); return ret; } @@ -149,7 +169,8 @@ nuraft::ptr getBuffer(int64_t number) TEST(CoordinationTest, TestSummingRaft1) { - SummingRaftServer s1(1, "localhost", 44444); + ChangelogDirTest test("./logs"); + SummingRaftServer s1(1, "localhost", 44444, "./logs"); /// Single node is leader EXPECT_EQ(s1.raft_instance->get_leader(), 1); @@ -172,9 +193,12 @@ TEST(CoordinationTest, TestSummingRaft1) TEST(CoordinationTest, TestSummingRaft3) { - SummingRaftServer s1(1, "localhost", 44444); - SummingRaftServer s2(2, "localhost", 44445); - SummingRaftServer s3(3, "localhost", 44446); + ChangelogDirTest test1("./logs1"); + SummingRaftServer s1(1, "localhost", 44444, "./logs1"); + ChangelogDirTest test2("./logs2"); + SummingRaftServer s2(2, "localhost", 44445, "./logs2"); + ChangelogDirTest test3("./logs3"); + SummingRaftServer s3(3, "localhost", 44446, "./logs3"); nuraft::srv_config first_config(1, "localhost:44444"); auto ret1 = s2.raft_instance->add_srv(first_config); @@ -343,27 +367,6 @@ DB::LogEntryPtr getLogEntry(const std::string & s, size_t term) return nuraft::cs_new(term, bufwriter.getBuffer()); } -namespace fs = std::filesystem; -struct ChangelogDirTest -{ - std::string path; - bool drop; - ChangelogDirTest(std::string path_, bool drop_ = true) - : path(path_) - , drop(drop_) - { - if (fs::exists(path)) - EXPECT_TRUE(false) << "Path " << path << " already exists, remove it to run test"; - fs::create_directory(path); - } - - ~ChangelogDirTest() - { - if (fs::exists(path) && drop) - fs::remove_all(path); - } -}; - TEST(CoordinationTest, ChangelogTestSimple) { ChangelogDirTest test("./logs"); @@ -386,7 +389,7 @@ TEST(CoordinationTest, ChangelogTestFile) auto entry = getLogEntry("hello world", 77); changelog.appendEntry(1, entry); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); - for(const auto & p : fs::directory_iterator("./logs")) + for (const auto & p : fs::directory_iterator("./logs")) EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin"); changelog.appendEntry(2, entry); @@ -484,7 +487,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); size_t logs_count = 0; - for(const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + for (const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) logs_count++; EXPECT_EQ(logs_count, 2); @@ -497,7 +500,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); logs_count = 0; - for(const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) + for (const auto & _ [[maybe_unused]]: fs::directory_iterator("./logs")) logs_count++; EXPECT_EQ(logs_count, 3); diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index 97c6d7c2e33..44123ffe9c1 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -2,6 +2,7 @@ 9181 1 + /var/lib/clickhouse/coordination/log 10000 From a1cd07b9a00ff0ea4bc4e98d03af9b5046e6854f Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:24:49 +0300 Subject: [PATCH 178/510] Update docs/ru/sql-reference/aggregate-functions/parametric-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md index 2c367882714..d96f7a13bcc 100644 --- a/docs/ru/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/ru/sql-reference/aggregate-functions/parametric-functions.md @@ -239,7 +239,7 @@ windowFunnel(window, [mode])(timestamp, cond1, cond2, ..., condN) **Параметры** -- `window` — ширина скользящего окна по времени. Единица измерения зависит от `timestamp` и может варьироваться. Определяется выражением `timestamp от cond2 <= timestamp от cond1 + window`. +- `window` — ширина скользящего окна по времени. Единица измерения зависит от `timestamp` и может варьироваться. Должно соблюдаться условие `timestamp события cond2 <= timestamp события cond1 + window`. - `mode` - необязательный параметр. Если установлено значение `'strict'`, то функция `windowFunnel()` применяет условия только для уникальных значений. - `timestamp` — имя столбца, содержащего временные отметки. [Date](../../sql-reference/aggregate-functions/parametric-functions.md), [DateTime](../../sql-reference/aggregate-functions/parametric-functions.md#data_type-datetime) и другие параметры с типом `Integer`. В случае хранения меток времени в столбцах с типом `UInt64`, максимально допустимое значение соответствует ограничению для типа `Int64`, т.е. равно `2^63-1`. - `cond` — условия или данные, описывающие цепочку событий. [UInt8](../../sql-reference/aggregate-functions/parametric-functions.md). From 8717dbd0e222536e6daf709820c3bee1ef395c05 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Feb 2021 22:29:09 +0300 Subject: [PATCH 179/510] Missed configs --- .../test_testkeeper_back_to_back/configs/enable_test_keeper.xml | 1 + .../configs/enable_test_keeper1.xml | 1 + .../configs/enable_test_keeper2.xml | 1 + .../configs/enable_test_keeper3.xml | 1 + .../configs/enable_test_keeper1.xml | 1 + .../configs/enable_test_keeper2.xml | 1 + .../configs/enable_test_keeper3.xml | 1 + 7 files changed, 7 insertions(+) diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index 1a441909998..a8b8991f959 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -2,6 +2,7 @@ 9181 1 + /var/lib/clickhouse/coordination/log 5000 diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper1.xml index 4ad76889d1e..a47e5eae09a 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper1.xml @@ -2,6 +2,7 @@ 9181 1 + /var/lib/clickhouse/coordination/log 5000 diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper2.xml index a1954a1e639..18681f0dc95 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper2.xml @@ -2,6 +2,7 @@ 9181 2 + /var/lib/clickhouse/coordination/log 5000 diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper3.xml index 88d2358138f..184d3724219 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/configs/enable_test_keeper3.xml @@ -2,6 +2,7 @@ 9181 3 + /var/lib/clickhouse/coordination/log 5000 diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml index 4ad76889d1e..a47e5eae09a 100644 --- a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml +++ b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper1.xml @@ -2,6 +2,7 @@ 9181 1 + /var/lib/clickhouse/coordination/log 5000 diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml index a1954a1e639..18681f0dc95 100644 --- a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml +++ b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper2.xml @@ -2,6 +2,7 @@ 9181 2 + /var/lib/clickhouse/coordination/log 5000 diff --git a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml index 88d2358138f..184d3724219 100644 --- a/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml +++ b/tests/integration/test_testkeeper_multinode_simple/configs/enable_test_keeper3.xml @@ -2,6 +2,7 @@ 9181 3 + /var/lib/clickhouse/coordination/log 5000 From b8be90cdf9c8505714cfaeb94ac6ffa296a0778d Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:32:59 +0300 Subject: [PATCH 180/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 91b26a2415d..adf084a6b21 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -110,7 +110,7 @@ SELECT comment, hex(secret) FROM encryption_test WHERE comment LIKE '%gcm%'; Совместима с шифрованием myqsl, результат может быть расшифрован функцией [AES_DECRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-decrypt). -При одинаковых вводных зашифрованный текст будет совпадать с результатом `encrypt`. Однако, когда `key` или `iv` длиннее, чем должны быть, `aes_encrypt_mysql` будет работать аналогично MySQL `aes_encrypt`: свернет ключ и проигнорирует лишнюю часть `iv`. +При одинаковых входящих значениях зашифрованный текст будет совпадать с результатом, возвращаемым функцией `encrypt`. Однако если `key` или `iv` длиннее, чем должны быть, `aes_encrypt_mysql` будет работать аналогично функции `aes_encrypt` в MySQL: свернет ключ и проигнорирует лишнюю часть `iv`. Функция поддерживает шифрофание данных следующими режимами: From a642dbce46f1734b1893f6528ad591641edbdc70 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:33:19 +0300 Subject: [PATCH 181/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index adf084a6b21..0e8e7d2a33a 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -140,7 +140,7 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv]) **Примеры** -При одинаковых вводных результаты шифрования `encrypt` и `aes_encrypt_mysql` будут совпадать. +При одинаковых входящих значениях результаты шифрования у функций `encrypt` и `aes_encrypt_mysql` совпадают. Запрос: From 22ab639287ea47b9a2dba80982170e15c9edd3a0 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:33:32 +0300 Subject: [PATCH 182/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 0e8e7d2a33a..a72866121c4 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -156,7 +156,7 @@ SELECT encrypt('aes-256-cfb128', 'Secret', '12345678910121314151617181920212', ' └───────────────────┘ ``` -Но `encrypt` генерирует исключение, когда `key` или `iv` длиннее, чем нужно: +Функция `encrypt` генерирует исключение, если `key` или `iv` длиннее чем нужно: Запрос: From d213039fe58fa8efe4340fdd4e3b14564139c71f Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:33:57 +0300 Subject: [PATCH 183/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index a72866121c4..90aa3268922 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -171,7 +171,7 @@ Received exception from server (version 21.1.2): Code: 36. DB::Exception: Received from localhost:9000. DB::Exception: Invalid key size: 33 expected 32: While processing encrypt('aes-256-cfb128', 'Secret', '123456789101213141516171819202122', 'iviviviviviviviv123'). ``` -Тогда как `aes_encrypt_mysql` возвращает совместимый с MySQL вывод: +Однако функция `aes_encrypt_mysql` в аналогичном случае возвращает результат, который может быть обработан MySQL: Запрос: From 66d6b7a3a088be7e72cab7ced29b1c7fa5c4f418 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:34:33 +0300 Subject: [PATCH 184/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 90aa3268922..f75e7bcc1a3 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -220,7 +220,7 @@ mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviv ## decrypt {#decrypt} -Функция расшифровывает зашифрованный текст в обычный следующими режимами: +Функция расшифровывает зашифрованный текст и может работать в следующих режимах: - aes-128-ecb, aes-192-ecb, aes-256-ecb - aes-128-cbc, aes-192-cbc, aes-256-cbc From 5edba428658e60f9ee0be3681e17b638e8f2d254 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:34:43 +0300 Subject: [PATCH 185/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index f75e7bcc1a3..c4e0968d6f9 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -203,7 +203,7 @@ SELECT hex(aes_encrypt_mysql('aes-256-cfb128', 'Secret', '1234567891012131415161 └──────────────┘ ``` -Это совпадает с тем, что выводит MySQL с такими же вводными: +Это совпадает с результатом, возвращаемым MySQL при таких же входящих значениях: ``` sql mysql> SET block_encryption_mode='aes-256-cfb128'; From a26f2b77cb84e5d5629a706f42bd5a0c8214c694 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:35:07 +0300 Subject: [PATCH 186/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index c4e0968d6f9..92e8d62faca 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -250,7 +250,7 @@ decrypt('mode', 'ciphertext', 'key' [, iv, aad]) **Примеры** -Используется таблица из [encrypt](#encrypt). +Рассмотрим таблицу из примера для функции [encrypt](#encrypt). Запрос: From 7a910d38a10c92f1aae4d13e5de34a73e10e978e Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:35:12 +0300 Subject: [PATCH 187/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 92e8d62faca..faddf314fe7 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -278,7 +278,7 @@ SELECT comment, hex(secret) FROM encryption_test; Запрос: ``` sql -SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920212') as plaintext FROM encryption_test +SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920212') as plaintext FROM encryption_test; ``` Результат: From 07795335cecc9352b7d4164bbd6c63599d19bda1 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:35:23 +0300 Subject: [PATCH 188/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index faddf314fe7..0f46f3c1fd5 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -293,7 +293,7 @@ SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920 └─────────────────────────────────────┴───────────┘ ``` -Обратите внимание, что только часть данных была расшифрована, а остальное является бессмыслицей, как как `mode`, `key`, или `iv` были другими во время шифрования. +Обратите внимание, что только часть данных была расшифрована верно. Оставшаяся часть расшифрована некорректно, так как при шифровании использовались другие значения `mode`, `key`, или `iv`. ## aes_decrypt_mysql {#aes_decrypt_mysql} From 579f8a95bcaa804b4264e8047d68474af5ef3ec6 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:35:43 +0300 Subject: [PATCH 189/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 0f46f3c1fd5..6cf5b520f23 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -299,7 +299,7 @@ SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920 Совместима с шифрованием myqsl и может расшифровать данные, зашифрованные функцией [AES_ENCRYPT](https://dev.mysql.com/doc/refman/8.0/en/encryption-functions.html#function_aes-encrypt). -При одинаковых вводных расшифрованный текст будет совпадать с результатом `decrypt`. Однако, когда `key` или `iv` длиннее, чем должны быть, `aes_decrypt_mysql` будет работать аналогично MySQL `aes_decrypt`: свернет ключ и проигнорирует лишнюю часть `iv`. +При одинаковых входящих значениях расшифрованный текст будет совпадать с результатом, возвращаемым функцией `decrypt`. Однако если `key` или `iv` длиннее, чем должны быть, `aes_decrypt_mysql` будет работать аналогично функции `aes_decrypt` в MySQL: свернет ключ и проигнорирует лишнюю часть `iv`. Функция поддерживает расшифровку данных следующими режимами: From b82e564076203733a292d53ebcf843ad0289ace9 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:35:48 +0300 Subject: [PATCH 190/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 6cf5b520f23..04a74fe8107 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -301,7 +301,7 @@ SELECT comment, decrypt('aes-256-cfb128', secret, '12345678910121314151617181920 При одинаковых входящих значениях расшифрованный текст будет совпадать с результатом, возвращаемым функцией `decrypt`. Однако если `key` или `iv` длиннее, чем должны быть, `aes_decrypt_mysql` будет работать аналогично функции `aes_decrypt` в MySQL: свернет ключ и проигнорирует лишнюю часть `iv`. -Функция поддерживает расшифровку данных следующими режимами: +Функция поддерживает расшифровку данных в следующих режимах: - aes-128-ecb, aes-192-ecb, aes-256-ecb - aes-128-cbc, aes-192-cbc, aes-256-cbc From c10485d21a29ab7e1ec405ef19fad35ca306185a Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:35:55 +0300 Subject: [PATCH 191/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 04a74fe8107..3c2f9e3e682 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -348,7 +348,7 @@ mysql> SELECT aes_encrypt('Secret', '123456789101213141516171819202122', 'iviviv Запрос: ``` sql -SELECT aes_decrypt_mysql('aes-256-cfb128', unhex('24E9E4966469'), '123456789101213141516171819202122', 'iviviviviviviviv123456') AS plaintext +SELECT aes_decrypt_mysql('aes-256-cfb128', unhex('24E9E4966469'), '123456789101213141516171819202122', 'iviviviviviviviv123456') AS plaintext; ``` Результат: From 236b9cfeff06a9ac5115736041586a9ae119d761 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:36:01 +0300 Subject: [PATCH 192/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 3c2f9e3e682..5406112624f 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -329,7 +329,7 @@ aes_decrypt_mysql('mode', 'ciphertext', 'key' [, iv]) **Примеры** -Расшифруем данные, которые до этого зашифровали с помощью MySQL: +Расшифруем данные, которые до этого были зашифрованы в MySQL: ``` sql From f2c7c38c18b817bf101769d4d69e1ab78075778e Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 16 Feb 2021 22:38:20 +0300 Subject: [PATCH 193/510] Update docs/ru/sql-reference/functions/encryption-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 5406112624f..e2c5560e4f6 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -11,7 +11,7 @@ toc_title: "\u0424\u0443\u043d\u043a\u0446\u0438\u0438 \u0434\u043b\u044f \u0448 Длина инициализирующего вектора всегда 16 байт (лишнии байты игнорируются). -Обратите внимание, что до версии Clickhouse 21.1 эти функции работают медленно. +Обратите внимание, что до версии Clickhouse 21.1 эти функции работали медленно. ## encrypt {#encrypt} From 2858151d09b70b018a9626a2c4efda6d1535ec8b Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Wed, 17 Feb 2021 00:25:34 +0300 Subject: [PATCH 194/510] Update kafka.md --- docs/ru/engines/table-engines/integrations/kafka.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/engines/table-engines/integrations/kafka.md b/docs/ru/engines/table-engines/integrations/kafka.md index 2b9dfcd49da..a1528edfd1d 100644 --- a/docs/ru/engines/table-engines/integrations/kafka.md +++ b/docs/ru/engines/table-engines/integrations/kafka.md @@ -47,7 +47,9 @@ SETTINGS - `kafka_row_delimiter` — символ-разделитель записей (строк), которым завершается сообщение. - `kafka_schema` — опциональный параметр, необходимый, если используется формат, требующий определения схемы. Например, [Cap’n Proto](https://capnproto.org/) требует путь к файлу со схемой и название корневого объекта `schema.capnp:Message`. - `kafka_num_consumers` — количество потребителей (consumer) на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. Общее число потребителей не должно превышать количество партиций в топике, так как на одну партицию может быть назначено не более одного потребителя. +- `kafka_max_block_size` — максимальный размер пачек (в сообщениях) для poll (по умолчанию `max_block_size`). - `kafka_skip_broken_messages` — максимальное количество некорректных сообщений в блоке. Если `kafka_skip_broken_messages = N`, то движок отбрасывает `N` сообщений Кафки, которые не получилось обработать. Одно сообщение в точности соответствует одной записи (строке). Значение по умолчанию – 0. +- `kafka_commit_every_batch` — фиксирует каждый обработанный и потребленный пакет вместо отдельной фиксации после записи целого блока (по умолчанию `0`). - `kafka_thread_per_consumer` — снабжает каждого потребителя независимым потоком (по умолчанию `0`). При включенном состоянии каждый потребитель сбрасывает данные независимо и параллельно (иначе — строки от нескольких потребителей склеиваются в один блок). Примеры From 23754e46e8a8c54ff00537546908fa629f8ece71 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Wed, 17 Feb 2021 01:41:47 +0300 Subject: [PATCH 195/510] Update docs/ru/engines/table-engines/integrations/kafka.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/engines/table-engines/integrations/kafka.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/integrations/kafka.md b/docs/ru/engines/table-engines/integrations/kafka.md index a1528edfd1d..5a6971b1ae6 100644 --- a/docs/ru/engines/table-engines/integrations/kafka.md +++ b/docs/ru/engines/table-engines/integrations/kafka.md @@ -49,8 +49,8 @@ SETTINGS - `kafka_num_consumers` — количество потребителей (consumer) на таблицу. По умолчанию: `1`. Укажите больше потребителей, если пропускная способность одного потребителя недостаточна. Общее число потребителей не должно превышать количество партиций в топике, так как на одну партицию может быть назначено не более одного потребителя. - `kafka_max_block_size` — максимальный размер пачек (в сообщениях) для poll (по умолчанию `max_block_size`). - `kafka_skip_broken_messages` — максимальное количество некорректных сообщений в блоке. Если `kafka_skip_broken_messages = N`, то движок отбрасывает `N` сообщений Кафки, которые не получилось обработать. Одно сообщение в точности соответствует одной записи (строке). Значение по умолчанию – 0. -- `kafka_commit_every_batch` — фиксирует каждый обработанный и потребленный пакет вместо отдельной фиксации после записи целого блока (по умолчанию `0`). -- `kafka_thread_per_consumer` — снабжает каждого потребителя независимым потоком (по умолчанию `0`). При включенном состоянии каждый потребитель сбрасывает данные независимо и параллельно (иначе — строки от нескольких потребителей склеиваются в один блок). +- `kafka_commit_every_batch` — включает или отключает режим записи каждой принятой и обработанной пачки по отдельности вместо единой записи целого блока (по умолчанию `0`). +- `kafka_thread_per_consumer` — включает или отключает предоставление отдельного потока каждому потребителю (по умолчанию `0`). При включенном режиме каждый потребитель сбрасывает данные независимо и параллельно, при отключённом — строки с данными от нескольких потребителей собираются в один блок. Примеры From dfaa79b88ed8bd5e67df1e510d1a91cb1644a6a5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 10:10:46 +0300 Subject: [PATCH 196/510] Add missed file --- src/Coordination/NuKeeperLogStore.cpp | 97 +++++++++++++++++++++++++++ 1 file changed, 97 insertions(+) create mode 100644 src/Coordination/NuKeeperLogStore.cpp diff --git a/src/Coordination/NuKeeperLogStore.cpp b/src/Coordination/NuKeeperLogStore.cpp new file mode 100644 index 00000000000..fa0631e14ad --- /dev/null +++ b/src/Coordination/NuKeeperLogStore.cpp @@ -0,0 +1,97 @@ +#include + +namespace DB +{ + +NuKeeperLogStore::NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_) + : changelog(changelogs_path, rotate_interval_) +{ +} + +size_t NuKeeperLogStore::start_index() const +{ + std::lock_guard lock(changelog_lock); + return changelog.getStartIndex(); +} + +void NuKeeperLogStore::init(size_t from_log_idx) +{ + std::lock_guard lock(changelog_lock); + changelog.readChangelogAndInitWriter(from_log_idx); +} + +size_t NuKeeperLogStore::next_slot() const +{ + std::lock_guard lock(changelog_lock); + return changelog.getNextEntryIndex(); +} + +nuraft::ptr NuKeeperLogStore::last_entry() const +{ + std::lock_guard lock(changelog_lock); + return changelog.getLastEntry(); +} + +size_t NuKeeperLogStore::append(nuraft::ptr & entry) +{ + std::lock_guard lock(changelog_lock); + size_t idx = changelog.getNextEntryIndex(); + changelog.appendEntry(idx, entry); + return idx; +} + + +void NuKeeperLogStore::write_at(size_t index, nuraft::ptr & entry) +{ + std::lock_guard lock(changelog_lock); + changelog.writeAt(index, entry); +} + +nuraft::ptr>> NuKeeperLogStore::log_entries(size_t start, size_t end) +{ + std::lock_guard lock(changelog_lock); + return changelog.getLogEntriesBetween(start, end); +} + +nuraft::ptr NuKeeperLogStore::entry_at(size_t index) +{ + std::lock_guard lock(changelog_lock); + return changelog.entryAt(index); +} + +size_t NuKeeperLogStore::term_at(size_t index) +{ + std::lock_guard lock(changelog_lock); + auto entry = changelog.entryAt(index); + if (entry) + return entry->get_term(); + return 0; +} + +nuraft::ptr NuKeeperLogStore::pack(size_t index, int32_t cnt) +{ + std::lock_guard lock(changelog_lock); + return changelog.serializeEntriesToBuffer(index, cnt); +} + +bool NuKeeperLogStore::compact(size_t last_log_index) +{ + std::lock_guard lock(changelog_lock); + changelog.compact(last_log_index); + return true; +} + +bool NuKeeperLogStore::flush() +{ + std::lock_guard lock(changelog_lock); + changelog.flush(); + return true; +} + +void NuKeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack) +{ + std::lock_guard lock(changelog_lock); + changelog.applyEntriesFromBuffer(index, pack); +} + +} From af95db2fcf8ac6c974e9a3d546392419b1ba6a5f Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 11:00:17 +0300 Subject: [PATCH 197/510] Test log storage instead of changelog --- src/Coordination/Changelog.cpp | 7 +- src/Coordination/NuKeeperLogStore.cpp | 6 + src/Coordination/NuKeeperLogStore.h | 2 + src/Coordination/tests/gtest_for_build.cpp | 327 +++++++++++++-------- 4 files changed, 218 insertions(+), 124 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index e4d8b13ec37..4f095974836 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -212,6 +212,8 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval void Changelog::readChangelogAndInitWriter(size_t from_log_idx) { size_t read_from_last = 0; + start_index = from_log_idx == 0 ? 1 : from_log_idx; + size_t total_read = 0; for (const auto & [start_id, changelog_file] : existing_changelogs) { ChangelogName parsed_name = getChangelogName(changelog_file); @@ -219,11 +221,10 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) { ChangelogReader reader(changelog_file); read_from_last = reader.readChangelog(logs, from_log_idx, index_to_start_pos); + total_read += read_from_last; } } - start_index = from_log_idx == 0 ? 1 : from_log_idx; - if (existing_changelogs.size() > 0 && read_from_last < rotate_interval) { auto str_name = existing_changelogs.rbegin()->second; @@ -233,7 +234,7 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) } else { - rotate(start_index); + rotate(start_index + total_read); } } diff --git a/src/Coordination/NuKeeperLogStore.cpp b/src/Coordination/NuKeeperLogStore.cpp index fa0631e14ad..fa8d6d6c299 100644 --- a/src/Coordination/NuKeeperLogStore.cpp +++ b/src/Coordination/NuKeeperLogStore.cpp @@ -94,4 +94,10 @@ void NuKeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack) changelog.applyEntriesFromBuffer(index, pack); } +size_t NuKeeperLogStore::size() const +{ + std::lock_guard lock(changelog_lock); + return changelog.size(); +} + } diff --git a/src/Coordination/NuKeeperLogStore.h b/src/Coordination/NuKeeperLogStore.h index 981dc3f24e7..49d5dbfdf7c 100644 --- a/src/Coordination/NuKeeperLogStore.h +++ b/src/Coordination/NuKeeperLogStore.h @@ -39,6 +39,8 @@ public: bool flush() override; + size_t size() const; + private: mutable std::mutex changelog_lock; Changelog changelog; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 6d91ba95111..8328d93d9cf 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -114,10 +114,10 @@ struct SimpliestRaftServer if (!raft_instance) { - std::cerr << "Failed to initialize launcher (see the message " - "in the log file)." << std::endl; + std::cerr << "Failed to initialize launcher" << std::endl; exit(-1); } + std::cout << "init Raft instance " << server_id; for (size_t ii = 0; ii < 20; ++ii) { @@ -370,33 +370,33 @@ DB::LogEntryPtr getLogEntry(const std::string & s, size_t term) TEST(CoordinationTest, ChangelogTestSimple) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 5); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); auto entry = getLogEntry("hello world", 77); - changelog.appendEntry(1, entry); - EXPECT_EQ(changelog.getNextEntryIndex(), 2); - EXPECT_EQ(changelog.getStartIndex(), 1); - EXPECT_EQ(changelog.getLastEntry()->get_term(), 77); - EXPECT_EQ(changelog.entryAt(1)->get_term(), 77); - EXPECT_EQ(changelog.getLogEntriesBetween(1, 2)->size(), 1); + changelog.append(entry); + EXPECT_EQ(changelog.next_slot(), 2); + EXPECT_EQ(changelog.start_index(), 1); + EXPECT_EQ(changelog.last_entry()->get_term(), 77); + EXPECT_EQ(changelog.entry_at(1)->get_term(), 77); + EXPECT_EQ(changelog.log_entries(1, 2)->size(), 1); } TEST(CoordinationTest, ChangelogTestFile) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 5); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); auto entry = getLogEntry("hello world", 77); - changelog.appendEntry(1, entry); + changelog.append(entry); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); for (const auto & p : fs::directory_iterator("./logs")) EXPECT_EQ(p.path(), "./logs/changelog_1_5.bin"); - changelog.appendEntry(2, entry); - changelog.appendEntry(3, entry); - changelog.appendEntry(4, entry); - changelog.appendEntry(5, entry); - changelog.appendEntry(6, entry); + changelog.append(entry); + changelog.append(entry); + changelog.append(entry); + changelog.append(entry); + changelog.append(entry); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); @@ -405,26 +405,26 @@ TEST(CoordinationTest, ChangelogTestFile) TEST(CoordinationTest, ChangelogReadWrite) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 1000); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 1000); + changelog.init(1); for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_EQ(changelog.size(), 10); - DB::Changelog changelog_reader("./logs", 1000); - changelog_reader.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog_reader("./logs", 1000); + changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), 10); - EXPECT_EQ(changelog_reader.getLastEntry()->get_term(), changelog.getLastEntry()->get_term()); - EXPECT_EQ(changelog_reader.getStartIndex(), changelog.getStartIndex()); - EXPECT_EQ(changelog_reader.getNextEntryIndex(), changelog.getNextEntryIndex()); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); + EXPECT_EQ(changelog_reader.start_index(), changelog.start_index()); + EXPECT_EQ(changelog_reader.next_slot(), changelog.next_slot()); for (size_t i = 0; i < 10; ++i) - EXPECT_EQ(changelog_reader.entryAt(i + 1)->get_term(), changelog.entryAt(i + 1)->get_term()); + EXPECT_EQ(changelog_reader.entry_at(i + 1)->get_term(), changelog.entry_at(i + 1)->get_term()); - auto entries_from_range_read = changelog_reader.getLogEntriesBetween(1, 11); - auto entries_from_range = changelog.getLogEntriesBetween(1, 11); + auto entries_from_range_read = changelog_reader.log_entries(1, 11); + auto entries_from_range = changelog.log_entries(1, 11); EXPECT_EQ(entries_from_range_read->size(), entries_from_range->size()); EXPECT_EQ(10, entries_from_range->size()); } @@ -432,55 +432,55 @@ TEST(CoordinationTest, ChangelogReadWrite) TEST(CoordinationTest, ChangelogWriteAt) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 1000); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 1000); + changelog.init(1); for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_EQ(changelog.size(), 10); auto entry = getLogEntry("writer", 77); - changelog.writeAt(7, entry); + changelog.write_at(7, entry); EXPECT_EQ(changelog.size(), 7); - EXPECT_EQ(changelog.getLastEntry()->get_term(), 77); - EXPECT_EQ(changelog.entryAt(7)->get_term(), 77); - EXPECT_EQ(changelog.getNextEntryIndex(), 8); + EXPECT_EQ(changelog.last_entry()->get_term(), 77); + EXPECT_EQ(changelog.entry_at(7)->get_term(), 77); + EXPECT_EQ(changelog.next_slot(), 8); - DB::Changelog changelog_reader("./logs", 1000); - changelog_reader.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog_reader("./logs", 1000); + changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), changelog.size()); - EXPECT_EQ(changelog_reader.getLastEntry()->get_term(), changelog.getLastEntry()->get_term()); - EXPECT_EQ(changelog_reader.getStartIndex(), changelog.getStartIndex()); - EXPECT_EQ(changelog_reader.getNextEntryIndex(), changelog.getNextEntryIndex()); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); + EXPECT_EQ(changelog_reader.start_index(), changelog.start_index()); + EXPECT_EQ(changelog_reader.next_slot(), changelog.next_slot()); } TEST(CoordinationTest, ChangelogTestAppendAfterRead) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 5); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); for (size_t i = 0; i < 7; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_EQ(changelog.size(), 7); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); - DB::Changelog changelog_reader("./logs", 5); - changelog_reader.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog_reader("./logs", 5); + changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), 7); for (size_t i = 7; i < 10; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog_reader.appendEntry(changelog_reader.getNextEntryIndex(), entry); + changelog_reader.append(entry); } EXPECT_EQ(changelog_reader.size(), 10); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); @@ -493,7 +493,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_EQ(logs_count, 2); auto entry = getLogEntry("someentry", 77); - changelog_reader.appendEntry(changelog_reader.getNextEntryIndex(), entry); + changelog_reader.append(entry); EXPECT_EQ(changelog_reader.size(), 11); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); @@ -509,13 +509,13 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead) TEST(CoordinationTest, ChangelogTestCompaction) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 5); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); for (size_t i = 0; i < 3; ++i) { auto entry = getLogEntry("hello world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_EQ(changelog.size(), 3); @@ -523,15 +523,19 @@ TEST(CoordinationTest, ChangelogTestCompaction) changelog.compact(2); EXPECT_EQ(changelog.size(), 1); - EXPECT_EQ(changelog.getStartIndex(), 3); - EXPECT_EQ(changelog.getNextEntryIndex(), 4); - EXPECT_EQ(changelog.getLastEntry()->get_term(), 20); + EXPECT_EQ(changelog.start_index(), 3); + EXPECT_EQ(changelog.next_slot(), 4); + EXPECT_EQ(changelog.last_entry()->get_term(), 20); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); - changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 30)); - changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 40)); - changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 50)); - changelog.appendEntry(changelog.getNextEntryIndex(), getLogEntry("hello world", 60)); + auto e1 = getLogEntry("hello world", 30); + changelog.append(e1); + auto e2 = getLogEntry("hello world", 40); + changelog.append(e2); + auto e3 = getLogEntry("hello world", 50); + changelog.append(e3); + auto e4 = getLogEntry("hello world", 60); + changelog.append(e4); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); @@ -542,109 +546,110 @@ TEST(CoordinationTest, ChangelogTestCompaction) EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); EXPECT_EQ(changelog.size(), 1); - EXPECT_EQ(changelog.getStartIndex(), 7); - EXPECT_EQ(changelog.getNextEntryIndex(), 8); - EXPECT_EQ(changelog.getLastEntry()->get_term(), 60); + EXPECT_EQ(changelog.start_index(), 7); + EXPECT_EQ(changelog.next_slot(), 8); + EXPECT_EQ(changelog.last_entry()->get_term(), 60); /// And we able to read it - DB::Changelog changelog_reader("./logs", 5); - changelog_reader.readChangelogAndInitWriter(7); + DB::NuKeeperLogStore changelog_reader("./logs", 5); + changelog_reader.init(7); EXPECT_EQ(changelog_reader.size(), 1); - EXPECT_EQ(changelog_reader.getStartIndex(), 7); - EXPECT_EQ(changelog_reader.getNextEntryIndex(), 8); - EXPECT_EQ(changelog_reader.getLastEntry()->get_term(), 60); + EXPECT_EQ(changelog_reader.start_index(), 7); + EXPECT_EQ(changelog_reader.next_slot(), 8); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), 60); } TEST(CoordinationTest, ChangelogTestBatchOperations) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 100); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 100); + changelog.init(1); for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_EQ(changelog.size(), 10); - auto entries = changelog.serializeEntriesToBuffer(1, 5); + auto entries = changelog.pack(1, 5); - DB::Changelog apply_changelog("./logs", 100); - apply_changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore apply_changelog("./logs", 100); + apply_changelog.init(1); for (size_t i = 0; i < 10; ++i) { - EXPECT_EQ(apply_changelog.entryAt(i + 1)->get_term(), i * 10); + EXPECT_EQ(apply_changelog.entry_at(i + 1)->get_term(), i * 10); } EXPECT_EQ(apply_changelog.size(), 10); - apply_changelog.applyEntriesFromBuffer(8, *entries); + apply_changelog.apply_pack(8, *entries); EXPECT_EQ(apply_changelog.size(), 12); - EXPECT_EQ(apply_changelog.getStartIndex(), 1); - EXPECT_EQ(apply_changelog.getNextEntryIndex(), 13); + EXPECT_EQ(apply_changelog.start_index(), 1); + EXPECT_EQ(apply_changelog.next_slot(), 13); for (size_t i = 0; i < 7; ++i) { - EXPECT_EQ(apply_changelog.entryAt(i + 1)->get_term(), i * 10); + EXPECT_EQ(apply_changelog.entry_at(i + 1)->get_term(), i * 10); } - EXPECT_EQ(apply_changelog.entryAt(8)->get_term(), 0); - EXPECT_EQ(apply_changelog.entryAt(9)->get_term(), 10); - EXPECT_EQ(apply_changelog.entryAt(10)->get_term(), 20); - EXPECT_EQ(apply_changelog.entryAt(11)->get_term(), 30); - EXPECT_EQ(apply_changelog.entryAt(12)->get_term(), 40); + EXPECT_EQ(apply_changelog.entry_at(8)->get_term(), 0); + EXPECT_EQ(apply_changelog.entry_at(9)->get_term(), 10); + EXPECT_EQ(apply_changelog.entry_at(10)->get_term(), 20); + EXPECT_EQ(apply_changelog.entry_at(11)->get_term(), 30); + EXPECT_EQ(apply_changelog.entry_at(12)->get_term(), 40); } TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 100); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 100); + changelog.init(1); for (size_t i = 0; i < 10; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_EQ(changelog.size(), 10); - auto entries = changelog.serializeEntriesToBuffer(5, 5); + auto entries = changelog.pack(5, 5); ChangelogDirTest test1("./logs1"); - DB::Changelog changelog_new("./logs1", 100); - changelog_new.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog_new("./logs1", 100); + changelog_new.init(1); EXPECT_EQ(changelog_new.size(), 0); - changelog_new.applyEntriesFromBuffer(5, *entries); + changelog_new.apply_pack(5, *entries); EXPECT_EQ(changelog_new.size(), 5); - EXPECT_EQ(changelog_new.getStartIndex(), 5); - EXPECT_EQ(changelog_new.getNextEntryIndex(), 10); + EXPECT_EQ(changelog_new.start_index(), 5); + EXPECT_EQ(changelog_new.next_slot(), 10); for (size_t i = 4; i < 9; ++i) - EXPECT_EQ(changelog_new.entryAt(i + 1)->get_term(), i * 10); + EXPECT_EQ(changelog_new.entry_at(i + 1)->get_term(), i * 10); - changelog_new.appendEntry(changelog_new.getNextEntryIndex(), getLogEntry("hello_world", 110)); + auto e = getLogEntry("hello_world", 110); + changelog_new.append(e); EXPECT_EQ(changelog_new.size(), 6); - EXPECT_EQ(changelog_new.getStartIndex(), 5); - EXPECT_EQ(changelog_new.getNextEntryIndex(), 11); + EXPECT_EQ(changelog_new.start_index(), 5); + EXPECT_EQ(changelog_new.next_slot(), 11); - DB::Changelog changelog_reader("./logs1", 100); - changelog_reader.readChangelogAndInitWriter(5); + DB::NuKeeperLogStore changelog_reader("./logs1", 100); + changelog_reader.init(5); } TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 5); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); @@ -657,11 +662,12 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_EQ(changelog.size(), 33); - changelog.writeAt(7, getLogEntry("helloworld", 5555)); + auto e1 = getLogEntry("helloworld", 5555); + changelog.write_at(7, e1); EXPECT_EQ(changelog.size(), 7); - EXPECT_EQ(changelog.getStartIndex(), 1); - EXPECT_EQ(changelog.getNextEntryIndex(), 8); - EXPECT_EQ(changelog.getLastEntry()->get_term(), 5555); + EXPECT_EQ(changelog.start_index(), 1); + EXPECT_EQ(changelog.next_slot(), 8); + EXPECT_EQ(changelog.last_entry()->get_term(), 5555); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); @@ -672,24 +678,24 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); - DB::Changelog changelog_read("./logs", 5); - changelog_read.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog_read("./logs", 5); + changelog_read.init(1); EXPECT_EQ(changelog_read.size(), 7); - EXPECT_EQ(changelog_read.getStartIndex(), 1); - EXPECT_EQ(changelog_read.getNextEntryIndex(), 8); - EXPECT_EQ(changelog_read.getLastEntry()->get_term(), 5555); + EXPECT_EQ(changelog_read.start_index(), 1); + EXPECT_EQ(changelog_read.next_slot(), 8); + EXPECT_EQ(changelog_read.last_entry()->get_term(), 5555); } TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) { ChangelogDirTest test("./logs"); - DB::Changelog changelog("./logs", 5); - changelog.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); for (size_t i = 0; i < 33; ++i) { auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); - changelog.appendEntry(changelog.getNextEntryIndex(), entry); + changelog.append(entry); } EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); @@ -702,11 +708,12 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_EQ(changelog.size(), 33); - changelog.writeAt(11, getLogEntry("helloworld", 5555)); + auto e1 = getLogEntry("helloworld", 5555); + changelog.write_at(11, e1); EXPECT_EQ(changelog.size(), 11); - EXPECT_EQ(changelog.getStartIndex(), 1); - EXPECT_EQ(changelog.getNextEntryIndex(), 12); - EXPECT_EQ(changelog.getLastEntry()->get_term(), 5555); + EXPECT_EQ(changelog.start_index(), 1); + EXPECT_EQ(changelog.next_slot(), 12); + EXPECT_EQ(changelog.last_entry()->get_term(), 5555); EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); @@ -717,12 +724,90 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); - DB::Changelog changelog_read("./logs", 5); - changelog_read.readChangelogAndInitWriter(1); + DB::NuKeeperLogStore changelog_read("./logs", 5); + changelog_read.init(1); EXPECT_EQ(changelog_read.size(), 11); - EXPECT_EQ(changelog_read.getStartIndex(), 1); - EXPECT_EQ(changelog_read.getNextEntryIndex(), 12); - EXPECT_EQ(changelog_read.getLastEntry()->get_term(), 5555); + EXPECT_EQ(changelog_read.start_index(), 1); + EXPECT_EQ(changelog_read.next_slot(), 12); + EXPECT_EQ(changelog_read.last_entry()->get_term(), 5555); +} + +TEST(CoordinationTest, ChangelogTestWriteAtAllFiles) +{ + ChangelogDirTest test("./logs"); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); + + for (size_t i = 0; i < 33; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.append(entry); + } + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin")); + + EXPECT_EQ(changelog.size(), 33); + + auto e1 = getLogEntry("helloworld", 5555); + changelog.write_at(1, e1); + EXPECT_EQ(changelog.size(), 1); + EXPECT_EQ(changelog.start_index(), 1); + EXPECT_EQ(changelog.next_slot(), 2); + EXPECT_EQ(changelog.last_entry()->get_term(), 5555); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + + EXPECT_FALSE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); +} + +TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead) +{ + ChangelogDirTest test("./logs"); + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); + + for (size_t i = 0; i < 35; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.append(entry); + } + EXPECT_EQ(changelog.size(), 35); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin")); + + + DB::NuKeeperLogStore changelog_reader("./logs", 5); + changelog_reader.init(1); + + auto entry = getLogEntry("36_hello_world", 360); + changelog_reader.append(entry); + + EXPECT_EQ(changelog_reader.size(), 36); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_36_40.bin")); } #endif From acf843a01a9ff7677188dfabbebd4a861a2a7d5a Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 12:00:12 +0300 Subject: [PATCH 198/510] Slightly more optimal --- src/Coordination/Changelog.cpp | 88 ++++++++++++---------- src/Coordination/Changelog.h | 11 ++- src/Coordination/tests/gtest_for_build.cpp | 57 ++++++++++++++ 3 files changed, 116 insertions(+), 40 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 4f095974836..9e1ed557430 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -39,21 +39,15 @@ namespace static constexpr auto DEFAULT_PREFIX = "changelog"; -struct ChangelogName -{ - std::string prefix; - size_t from_log_idx; - size_t to_log_idx; -}; - -std::string formatChangelogPath(const std::string & prefix, const ChangelogName & name) +std::string formatChangelogPath(const std::string & prefix, const ChangelogFileDescription & name) { std::filesystem::path path(prefix); path /= std::filesystem::path(name.prefix + "_" + std::to_string(name.from_log_idx) + "_" + std::to_string(name.to_log_idx) + ".bin"); return path; } -ChangelogName getChangelogName(const std::string & path_str) + +ChangelogFileDescription getChangelogFileDescription(const std::string & path_str) { std::filesystem::path path(path_str); std::string filename = path.stem(); @@ -62,10 +56,11 @@ ChangelogName getChangelogName(const std::string & path_str) if (filename_parts.size() < 3) throw Exception(ErrorCodes::CORRUPTED_DATA, "Invalid changelog {}", path_str); - ChangelogName result; + ChangelogFileDescription result; result.prefix = filename_parts[0]; result.from_log_idx = parse(filename_parts[1]); result.to_log_idx = parse(filename_parts[2]); + result.path = path_str; return result; } @@ -204,8 +199,8 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval for (const auto & p : fs::directory_iterator(changelogs_dir)) { - auto name = getChangelogName(p.path()); - existing_changelogs[name.from_log_idx] = p.path(); + auto file_description = getChangelogFileDescription(p.path()); + existing_changelogs[file_description.from_log_idx] = file_description; } } @@ -214,22 +209,40 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) size_t read_from_last = 0; start_index = from_log_idx == 0 ? 1 : from_log_idx; size_t total_read = 0; - for (const auto & [start_id, changelog_file] : existing_changelogs) + size_t entries_in_last = 0; + size_t incomplete_log_idx = 0; + for (const auto & [start_idx, changelog_description] : existing_changelogs) { - ChangelogName parsed_name = getChangelogName(changelog_file); - if (parsed_name.to_log_idx >= from_log_idx) + entries_in_last = changelog_description.to_log_idx - changelog_description.from_log_idx + 1; + + if (changelog_description.to_log_idx >= from_log_idx) { - ChangelogReader reader(changelog_file); + ChangelogReader reader(changelog_description.path); read_from_last = reader.readChangelog(logs, from_log_idx, index_to_start_pos); total_read += read_from_last; + + /// May happen after truncate and crash + if (read_from_last < entries_in_last) + { + incomplete_log_idx = start_idx; + break; + } } } - if (existing_changelogs.size() > 0 && read_from_last < rotate_interval) + if (incomplete_log_idx != 0) { - auto str_name = existing_changelogs.rbegin()->second; - auto parsed_name = getChangelogName(str_name); - current_writer = std::make_unique(str_name, WriteMode::Append, parsed_name.from_log_idx); + for (auto itr = existing_changelogs.upper_bound(incomplete_log_idx); itr != existing_changelogs.end();) + { + std::filesystem::remove(itr->second.path); + itr = existing_changelogs.erase(itr); + } + } + + if (existing_changelogs.size() > 0 && read_from_last < entries_in_last) + { + auto description = existing_changelogs.rbegin()->second; + current_writer = std::make_unique(description.path, WriteMode::Append, description.from_log_idx); current_writer->setEntriesWritten(read_from_last); } else @@ -243,14 +256,14 @@ void Changelog::rotate(size_t new_start_log_idx) if (current_writer) current_writer->flush(); - ChangelogName new_name; - new_name.prefix = DEFAULT_PREFIX; - new_name.from_log_idx = new_start_log_idx; - new_name.to_log_idx = new_start_log_idx + rotate_interval - 1; + ChangelogFileDescription new_description; + new_description.prefix = DEFAULT_PREFIX; + new_description.from_log_idx = new_start_log_idx; + new_description.to_log_idx = new_start_log_idx + rotate_interval - 1; - auto new_log_path = formatChangelogPath(changelogs_dir, new_name); - existing_changelogs[new_start_log_idx] = new_log_path; - current_writer = std::make_unique(new_log_path, WriteMode::Rewrite, new_start_log_idx); + new_description.path = formatChangelogPath(changelogs_dir, new_description); + existing_changelogs[new_start_log_idx] = new_description; + current_writer = std::make_unique(new_description.path, WriteMode::Rewrite, new_start_log_idx); } ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) const @@ -301,15 +314,14 @@ void Changelog::writeAt(size_t index, nuraft::ptr log_entry) if (need_rollback) { auto index_changelog = existing_changelogs.lower_bound(index); - std::string fname; + ChangelogFileDescription description; if (index_changelog->first == index) - fname = index_changelog->second; + description = index_changelog->second; else - fname = std::prev(index_changelog)->second; + description = std::prev(index_changelog)->second; - current_writer = std::make_unique(fname, WriteMode::Append, index_changelog->first); - auto formated_name = getChangelogName(fname); - current_writer->setEntriesWritten(formated_name.to_log_idx - formated_name.from_log_idx + 1); + current_writer = std::make_unique(description.path, WriteMode::Append, index_changelog->first); + current_writer->setEntriesWritten(description.to_log_idx - description.from_log_idx + 1); } auto entries_written = current_writer->getEntriesWritten(); @@ -320,7 +332,7 @@ void Changelog::writeAt(size_t index, nuraft::ptr log_entry) auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { - std::filesystem::remove(itr->second); + std::filesystem::remove(itr->second.path); itr = existing_changelogs.erase(itr); } } @@ -342,17 +354,16 @@ void Changelog::compact(size_t up_to_log_idx) { for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();) { - ChangelogName parsed_name = getChangelogName(itr->second); - if (parsed_name.to_log_idx <= up_to_log_idx) + if (itr->second.to_log_idx <= up_to_log_idx) { - for (size_t idx = parsed_name.from_log_idx; idx <= parsed_name.to_log_idx; ++idx) + for (size_t idx = itr->second.from_log_idx; idx <= itr->second.to_log_idx; ++idx) { auto index_pos = index_to_start_pos.find(idx); if (index_pos == index_to_start_pos.end()) break; index_to_start_pos.erase(index_pos); } - std::filesystem::remove(itr->second); + std::filesystem::remove(itr->second.path); itr = existing_changelogs.erase(itr); } else @@ -366,7 +377,6 @@ void Changelog::compact(size_t up_to_log_idx) LogEntryPtr Changelog::getLastEntry() const { - static LogEntryPtr fake_entry = nuraft::cs_new(0, nuraft::buffer::alloc(sizeof(size_t))); size_t next_idx = getNextEntryIndex() - 1; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 7c352e7a91b..e154c1c70c6 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -45,6 +45,15 @@ struct ChangelogRecord nuraft::ptr blob; }; +struct ChangelogFileDescription +{ + std::string prefix; + size_t from_log_idx; + size_t to_log_idx; + + std::string path; +}; + class ChangelogWriter; class Changelog @@ -98,7 +107,7 @@ private: private: std::string changelogs_dir; - std::map existing_changelogs; + std::map existing_changelogs; std::unique_ptr current_writer; IndexToOffset index_to_start_pos; const size_t rotate_interval; diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 8328d93d9cf..76dd08a6d33 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -36,7 +36,9 @@ struct ChangelogDirTest , drop(drop_) { if (fs::exists(path)) + { EXPECT_TRUE(false) << "Path " << path << " already exists, remove it to run test"; + } fs::create_directory(path); } @@ -810,4 +812,59 @@ TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_36_40.bin")); } + +TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) +{ + ChangelogDirTest test("./logs"); + + DB::NuKeeperLogStore changelog("./logs", 5); + changelog.init(1); + + for (size_t i = 0; i < 35; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", i * 10); + changelog.append(entry); + } + EXPECT_EQ(changelog.size(), 35); + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_31_35.bin")); + + DB::WriteBufferFromFile plain_buf("./logs/changelog_11_15.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + plain_buf.truncate(0); + + DB::NuKeeperLogStore changelog_reader("./logs", 5); + changelog_reader.init(1); + + EXPECT_EQ(changelog_reader.size(), 10); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), 90); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); + + auto entry = getLogEntry("h", 7777); + changelog_reader.append(entry); + EXPECT_EQ(changelog_reader.size(), 11); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); + + EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_11_15.bin")); + + EXPECT_FALSE(fs::exists("./logs/changelog_16_20.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); +} + #endif From e5cef576e589f4307f35074cf45e8dbb08801c65 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 17 Feb 2021 12:39:40 +0300 Subject: [PATCH 199/510] Update subqueries.xml --- tests/performance/subqueries.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/performance/subqueries.xml b/tests/performance/subqueries.xml index f1481a78c7e..0d41099841b 100644 --- a/tests/performance/subqueries.xml +++ b/tests/performance/subqueries.xml @@ -1,7 +1,7 @@ - create table tab (a UInt32, b UInt32) engine = MergeTree order by (a, b) + create table tab (a UInt32, b UInt32) engine = MergeTree order by (a, b) insert into tab values (1, 1) select a, b from tab where (a, b) in (select toUInt32(number) as x, toUInt32(sleep(0.1) + 1) from numbers_mt(16)) settings max_threads = 2, max_block_size = 4 select a, b from tab where (1, 1) = (select min(toUInt32(number + 1)) as x, min(toUInt32(sleep(0.1) + 1)) from numbers_mt(16)) settings max_threads = 2, max_block_size = 4 DROP TABLE tab - \ No newline at end of file + From a8647096ed96fb348aea73edf54b5e7bedea4284 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Feb 2021 13:27:47 +0300 Subject: [PATCH 200/510] Try fix tests. --- src/Interpreters/ActionsDAG.cpp | 20 +++++++++++++------ .../Optimizations/filterPushDown.cpp | 4 ++-- .../QueryPlan/Optimizations/optimizeTree.cpp | 8 ++++++++ 3 files changed, 24 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index e9e9d1628a8..691905bed27 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1245,14 +1245,14 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, { struct Frame { - const Node * node; + Node * node; bool is_predicate = false; size_t next_child_to_visit = 0; size_t num_allowed_children = 0; }; std::stack stack; - std::unordered_set visited_nodes; + std::unordered_set visited_nodes; stack.push(Frame{.node = *it, .is_predicate = true}); visited_nodes.insert(*it); @@ -1290,12 +1290,12 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, else if (is_conjunction) { for (auto * child : cur.node->children) - { if (allowed_nodes.count(child)) selected_predicates.insert(child); - else - other_predicates.insert(child); - } + } + else if (cur.is_predicate) + { + other_predicates.insert(cur.node); } stack.pop(); @@ -1311,6 +1311,14 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, return nullptr; } + // std::cerr << "************* Selectecd predicates\n"; + // for (const auto * p : selected_predicates) + // std::cerr << p->result_name << std::endl; + + // std::cerr << "............. Other predicates\n"; + // for (const auto * p : other_predicates) + // std::cerr << p->result_name << std::endl; + auto actions = cloneEmpty(); actions->settings.project_input = false; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 98e923249f3..39f24a32b45 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -117,8 +117,8 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (keys.count(column.name) == 0) allowed_inputs.push_back(column.name); - for (const auto & name : allowed_inputs) - std::cerr << name << std::endl; + // for (const auto & name : allowed_inputs) + // std::cerr << name << std::endl; if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) return updated_steps; diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index e5ccc173ed8..cc81a7f39fc 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -23,6 +23,9 @@ void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes) std::stack stack; stack.push(Frame{.node = &root}); + size_t max_optimizations_to_apply = 0; + size_t total_applied_optimizations = 0; + while (!stack.empty()) { auto & frame = stack.top(); @@ -54,8 +57,13 @@ void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes) if (!optimization.apply) continue; + if (max_optimizations_to_apply && max_optimizations_to_apply < total_applied_optimizations) + continue; + /// Try to apply optimization. auto update_depth = optimization.apply(frame.node, nodes); + if (update_depth) + ++total_applied_optimizations; max_update_depth = std::max(max_update_depth, update_depth); } From 9396bae2e2051e2d50faa0d8c1005465171db481 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 14:53:47 +0300 Subject: [PATCH 201/510] More reliable test keeper tests --- src/Coordination/tests/gtest_for_build.cpp | 2 +- .../test_testkeeper_back_to_back/test.py | 536 +++++++++--------- .../__init__.py | 1 + .../configs/enable_test_keeper.xml | 21 + .../configs/logs_conf.xml | 12 + .../configs/use_test_keeper.xml | 8 + .../test_testkeeper_persistent_log/test.py | 124 ++++ 7 files changed, 444 insertions(+), 260 deletions(-) create mode 100644 tests/integration/test_testkeeper_persistent_log/__init__.py create mode 100644 tests/integration/test_testkeeper_persistent_log/configs/enable_test_keeper.xml create mode 100644 tests/integration/test_testkeeper_persistent_log/configs/logs_conf.xml create mode 100644 tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml create mode 100644 tests/integration/test_testkeeper_persistent_log/test.py diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 76dd08a6d33..81e1751c08c 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -31,7 +31,7 @@ struct ChangelogDirTest { std::string path; bool drop; - ChangelogDirTest(std::string path_, bool drop_ = true) + explicit ChangelogDirTest(std::string path_, bool drop_ = true) : path(path_) , drop(drop_) { diff --git a/tests/integration/test_testkeeper_back_to_back/test.py b/tests/integration/test_testkeeper_back_to_back/test.py index 8ec54f1a883..dd4e1f98cfd 100644 --- a/tests/integration/test_testkeeper_back_to_back/test.py +++ b/tests/integration/test_testkeeper_back_to_back/test.py @@ -8,32 +8,23 @@ from multiprocessing.dummy import Pool cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=['configs/enable_test_keeper.xml', 'configs/logs_conf.xml'], with_zookeeper=True) -from kazoo.client import KazooClient, KazooState - -_genuine_zk_instance = None -_fake_zk_instance = None +from kazoo.client import KazooClient, KazooState, KeeperState def get_genuine_zk(): - global _genuine_zk_instance - if not _genuine_zk_instance: - print("Zoo1", cluster.get_instance_ip("zoo1")) - _genuine_zk_instance = cluster.get_kazoo_client('zoo1') - return _genuine_zk_instance - + print("Zoo1", cluster.get_instance_ip("zoo1")) + return cluster.get_kazoo_client('zoo1') def get_fake_zk(): - global _fake_zk_instance - if not _fake_zk_instance: - print("node", cluster.get_instance_ip("node")) - _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181", timeout=30.0) - def reset_last_zxid_listener(state): - print("Fake zk callback called for state", state) - global _fake_zk_instance - if state != KazooState.CONNECTED: - _fake_zk_instance._reset() + print("node", cluster.get_instance_ip("node")) + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip("node") + ":9181", timeout=30.0) + def reset_last_zxid_listener(state): + print("Fake zk callback called for state", state) + nonlocal _fake_zk_instance + if state != KazooState.CONNECTED: + _fake_zk_instance._reset() - _fake_zk_instance.add_listener(reset_last_zxid_listener) - _fake_zk_instance.start() + _fake_zk_instance.add_listener(reset_last_zxid_listener) + _fake_zk_instance.start() return _fake_zk_instance def random_string(length): @@ -44,6 +35,15 @@ def create_random_path(prefix="", depth=1): return prefix return create_random_path(os.path.join(prefix, random_string(3)), depth - 1) +def stop_zk(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + + @pytest.fixture(scope="module") def started_cluster(): try: @@ -53,44 +53,46 @@ def started_cluster(): finally: cluster.shutdown() - if _genuine_zk_instance: - _genuine_zk_instance.stop() - _genuine_zk_instance.close() - if _fake_zk_instance: - _fake_zk_instance.stop() - _fake_zk_instance.close() def test_simple_commands(started_cluster): - genuine_zk = get_genuine_zk() - fake_zk = get_fake_zk() + try: + genuine_zk = get_genuine_zk() + fake_zk = get_fake_zk() - for zk in [genuine_zk, fake_zk]: - zk.create("/test_simple_commands", b"") - zk.create("/test_simple_commands/somenode1", b"hello") - zk.set("/test_simple_commands/somenode1", b"world") + for zk in [genuine_zk, fake_zk]: + zk.create("/test_simple_commands", b"") + zk.create("/test_simple_commands/somenode1", b"hello") + zk.set("/test_simple_commands/somenode1", b"world") - for zk in [genuine_zk, fake_zk]: - assert zk.exists("/test_simple_commands") - assert zk.exists("/test_simple_commands/somenode1") - print(zk.get("/test_simple_commands/somenode1")) - assert zk.get("/test_simple_commands/somenode1")[0] == b"world" + for zk in [genuine_zk, fake_zk]: + assert zk.exists("/test_simple_commands") + assert zk.exists("/test_simple_commands/somenode1") + print(zk.get("/test_simple_commands/somenode1")) + assert zk.get("/test_simple_commands/somenode1")[0] == b"world" + finally: + for zk in [genuine_zk, fake_zk]: + stop_zk(zk) def test_sequential_nodes(started_cluster): - genuine_zk = get_genuine_zk() - fake_zk = get_fake_zk() - genuine_zk.create("/test_sequential_nodes") - fake_zk.create("/test_sequential_nodes") - for i in range(1, 11): - genuine_zk.create("/test_sequential_nodes/" + ("a" * i) + "-", sequence=True) - genuine_zk.create("/test_sequential_nodes/" + ("b" * i)) - fake_zk.create("/test_sequential_nodes/" + ("a" * i) + "-", sequence=True) - fake_zk.create("/test_sequential_nodes/" + ("b" * i)) + try: + genuine_zk = get_genuine_zk() + fake_zk = get_fake_zk() + genuine_zk.create("/test_sequential_nodes") + fake_zk.create("/test_sequential_nodes") + for i in range(1, 11): + genuine_zk.create("/test_sequential_nodes/" + ("a" * i) + "-", sequence=True) + genuine_zk.create("/test_sequential_nodes/" + ("b" * i)) + fake_zk.create("/test_sequential_nodes/" + ("a" * i) + "-", sequence=True) + fake_zk.create("/test_sequential_nodes/" + ("b" * i)) - genuine_childs = list(sorted(genuine_zk.get_children("/test_sequential_nodes"))) - fake_childs = list(sorted(fake_zk.get_children("/test_sequential_nodes"))) - assert genuine_childs == fake_childs + genuine_childs = list(sorted(genuine_zk.get_children("/test_sequential_nodes"))) + fake_childs = list(sorted(fake_zk.get_children("/test_sequential_nodes"))) + assert genuine_childs == fake_childs + finally: + for zk in [genuine_zk, fake_zk]: + stop_zk(zk) def assert_eq_stats(stat1, stat2): @@ -102,130 +104,141 @@ def assert_eq_stats(stat1, stat2): assert stat1.numChildren == stat2.numChildren def test_stats(started_cluster): - genuine_zk = get_genuine_zk() - fake_zk = get_fake_zk() - genuine_zk.create("/test_stats_nodes") - fake_zk.create("/test_stats_nodes") - genuine_stats = genuine_zk.exists("/test_stats_nodes") - fake_stats = fake_zk.exists("/test_stats_nodes") - assert_eq_stats(genuine_stats, fake_stats) - for i in range(1, 11): - genuine_zk.create("/test_stats_nodes/" + ("a" * i) + "-", sequence=True) - genuine_zk.create("/test_stats_nodes/" + ("b" * i)) - fake_zk.create("/test_stats_nodes/" + ("a" * i) + "-", sequence=True) - fake_zk.create("/test_stats_nodes/" + ("b" * i)) + try: + genuine_zk = get_genuine_zk() + fake_zk = get_fake_zk() + genuine_zk.create("/test_stats_nodes") + fake_zk.create("/test_stats_nodes") + genuine_stats = genuine_zk.exists("/test_stats_nodes") + fake_stats = fake_zk.exists("/test_stats_nodes") + assert_eq_stats(genuine_stats, fake_stats) + for i in range(1, 11): + genuine_zk.create("/test_stats_nodes/" + ("a" * i) + "-", sequence=True) + genuine_zk.create("/test_stats_nodes/" + ("b" * i)) + fake_zk.create("/test_stats_nodes/" + ("a" * i) + "-", sequence=True) + fake_zk.create("/test_stats_nodes/" + ("b" * i)) - genuine_stats = genuine_zk.exists("/test_stats_nodes") - fake_stats = fake_zk.exists("/test_stats_nodes") - assert_eq_stats(genuine_stats, fake_stats) - for i in range(1, 11): - print("/test_stats_nodes/" + ("a" * i) + "-" + "{:010d}".format((i - 1) * 2)) - genuine_zk.delete("/test_stats_nodes/" + ("a" * i) + "-" + "{:010d}".format((i - 1) * 2)) - genuine_zk.delete("/test_stats_nodes/" + ("b" * i)) - fake_zk.delete("/test_stats_nodes/" + ("a" * i) + "-" + "{:010d}".format((i - 1) * 2)) - fake_zk.delete("/test_stats_nodes/" + ("b" * i)) + genuine_stats = genuine_zk.exists("/test_stats_nodes") + fake_stats = fake_zk.exists("/test_stats_nodes") + assert_eq_stats(genuine_stats, fake_stats) + for i in range(1, 11): + print("/test_stats_nodes/" + ("a" * i) + "-" + "{:010d}".format((i - 1) * 2)) + genuine_zk.delete("/test_stats_nodes/" + ("a" * i) + "-" + "{:010d}".format((i - 1) * 2)) + genuine_zk.delete("/test_stats_nodes/" + ("b" * i)) + fake_zk.delete("/test_stats_nodes/" + ("a" * i) + "-" + "{:010d}".format((i - 1) * 2)) + fake_zk.delete("/test_stats_nodes/" + ("b" * i)) - genuine_stats = genuine_zk.exists("/test_stats_nodes") - fake_stats = fake_zk.exists("/test_stats_nodes") - print(genuine_stats) - print(fake_stats) - assert_eq_stats(genuine_stats, fake_stats) - for i in range(100): - genuine_zk.set("/test_stats_nodes", ("q" * i).encode()) - fake_zk.set("/test_stats_nodes", ("q" * i).encode()) + genuine_stats = genuine_zk.exists("/test_stats_nodes") + fake_stats = fake_zk.exists("/test_stats_nodes") + print(genuine_stats) + print(fake_stats) + assert_eq_stats(genuine_stats, fake_stats) + for i in range(100): + genuine_zk.set("/test_stats_nodes", ("q" * i).encode()) + fake_zk.set("/test_stats_nodes", ("q" * i).encode()) - genuine_stats = genuine_zk.exists("/test_stats_nodes") - fake_stats = fake_zk.exists("/test_stats_nodes") - print(genuine_stats) - print(fake_stats) - assert_eq_stats(genuine_stats, fake_stats) + genuine_stats = genuine_zk.exists("/test_stats_nodes") + fake_stats = fake_zk.exists("/test_stats_nodes") + print(genuine_stats) + print(fake_stats) + assert_eq_stats(genuine_stats, fake_stats) + finally: + for zk in [genuine_zk, fake_zk]: + stop_zk(zk) def test_watchers(started_cluster): - genuine_zk = get_genuine_zk() - fake_zk = get_fake_zk() - genuine_zk.create("/test_data_watches") - fake_zk.create("/test_data_watches") - genuine_data_watch_data = None + try: + genuine_zk = get_genuine_zk() + fake_zk = get_fake_zk() + genuine_zk.create("/test_data_watches") + fake_zk.create("/test_data_watches") + genuine_data_watch_data = None - def genuine_callback(event): - print("Genuine data watch called") - nonlocal genuine_data_watch_data - genuine_data_watch_data = event + def genuine_callback(event): + print("Genuine data watch called") + nonlocal genuine_data_watch_data + genuine_data_watch_data = event - fake_data_watch_data = None - def fake_callback(event): - print("Fake data watch called") - nonlocal fake_data_watch_data - fake_data_watch_data = event + fake_data_watch_data = None + def fake_callback(event): + print("Fake data watch called") + nonlocal fake_data_watch_data + fake_data_watch_data = event - genuine_zk.get("/test_data_watches", watch=genuine_callback) - fake_zk.get("/test_data_watches", watch=fake_callback) + genuine_zk.get("/test_data_watches", watch=genuine_callback) + fake_zk.get("/test_data_watches", watch=fake_callback) - print("Calling set genuine") - genuine_zk.set("/test_data_watches", b"a") - print("Calling set fake") - fake_zk.set("/test_data_watches", b"a") - time.sleep(3) + print("Calling set genuine") + genuine_zk.set("/test_data_watches", b"a") + print("Calling set fake") + fake_zk.set("/test_data_watches", b"a") + time.sleep(3) - print("Genuine data", genuine_data_watch_data) - print("Fake data", fake_data_watch_data) - assert genuine_data_watch_data == fake_data_watch_data + print("Genuine data", genuine_data_watch_data) + print("Fake data", fake_data_watch_data) + assert genuine_data_watch_data == fake_data_watch_data - genuine_children = None - def genuine_child_callback(event): - print("Genuine child watch called") - nonlocal genuine_children - genuine_children = event + genuine_children = None + def genuine_child_callback(event): + print("Genuine child watch called") + nonlocal genuine_children + genuine_children = event - fake_children = None - def fake_child_callback(event): - print("Fake child watch called") - nonlocal fake_children - fake_children = event + fake_children = None + def fake_child_callback(event): + print("Fake child watch called") + nonlocal fake_children + fake_children = event - genuine_zk.get_children("/test_data_watches", watch=genuine_child_callback) - fake_zk.get_children("/test_data_watches", watch=fake_child_callback) + genuine_zk.get_children("/test_data_watches", watch=genuine_child_callback) + fake_zk.get_children("/test_data_watches", watch=fake_child_callback) - print("Calling genuine child") - genuine_zk.create("/test_data_watches/child", b"b") - print("Calling fake child") - fake_zk.create("/test_data_watches/child", b"b") + print("Calling genuine child") + genuine_zk.create("/test_data_watches/child", b"b") + print("Calling fake child") + fake_zk.create("/test_data_watches/child", b"b") - time.sleep(3) + time.sleep(3) - print("Genuine children", genuine_children) - print("Fake children", fake_children) - assert genuine_children == fake_children + print("Genuine children", genuine_children) + print("Fake children", fake_children) + assert genuine_children == fake_children + finally: + for zk in [genuine_zk, fake_zk]: + stop_zk(zk) def test_multitransactions(started_cluster): - genuine_zk = get_genuine_zk() - fake_zk = get_fake_zk() - for zk in [genuine_zk, fake_zk]: - zk.create('/test_multitransactions') - t = zk.transaction() - t.create('/test_multitransactions/freddy') - t.create('/test_multitransactions/fred', ephemeral=True) - t.create('/test_multitransactions/smith', sequence=True) - results = t.commit() - assert len(results) == 3 - assert results[0] == '/test_multitransactions/freddy' - assert results[2].startswith('/test_multitransactions/smith0') is True - - from kazoo.exceptions import RolledBackError, NoNodeError - for i, zk in enumerate([genuine_zk, fake_zk]): - print("Processing ZK", i) - t = zk.transaction() - t.create('/test_multitransactions/q') - t.delete('/test_multitransactions/a') - t.create('/test_multitransactions/x') - results = t.commit() - print("Results", results) - assert results[0].__class__ == RolledBackError - assert results[1].__class__ == NoNodeError - assert zk.exists('/test_multitransactions/q') is None - assert zk.exists('/test_multitransactions/a') is None - assert zk.exists('/test_multitransactions/x') is None + try: + genuine_zk = get_genuine_zk() + fake_zk = get_fake_zk() + for zk in [genuine_zk, fake_zk]: + zk.create('/test_multitransactions') + t = zk.transaction() + t.create('/test_multitransactions/freddy') + t.create('/test_multitransactions/fred', ephemeral=True) + t.create('/test_multitransactions/smith', sequence=True) + results = t.commit() + assert len(results) == 3 + assert results[0] == '/test_multitransactions/freddy' + assert results[2].startswith('/test_multitransactions/smith0') is True + from kazoo.exceptions import RolledBackError, NoNodeError + for i, zk in enumerate([genuine_zk, fake_zk]): + print("Processing ZK", i) + t = zk.transaction() + t.create('/test_multitransactions/q') + t.delete('/test_multitransactions/a') + t.create('/test_multitransactions/x') + results = t.commit() + print("Results", results) + assert results[0].__class__ == RolledBackError + assert results[1].__class__ == NoNodeError + assert zk.exists('/test_multitransactions/q') is None + assert zk.exists('/test_multitransactions/a') is None + assert zk.exists('/test_multitransactions/x') is None + finally: + for zk in [genuine_zk, fake_zk]: + stop_zk(zk) def exists(zk, path): result = zk.exists(path) @@ -278,13 +291,13 @@ class Request(object): arg_str = ', '.join([str(k) + "=" + str(v) for k, v in self.arguments.items()]) return "ZKRequest name {} with arguments {}".format(self.name, arg_str) -def generate_requests(iters=1): +def generate_requests(prefix="/", iters=1): requests = [] existing_paths = [] for i in range(iters): for _ in range(100): rand_length = random.randint(0, 10) - path = "/" + path = prefix for j in range(1, rand_length): path = create_random_path(path, 1) existing_paths.append(path) @@ -322,31 +335,43 @@ def generate_requests(iters=1): def test_random_requests(started_cluster): - requests = generate_requests(10) - genuine_zk = get_genuine_zk() - fake_zk = get_fake_zk() - for i, request in enumerate(requests): - genuine_throw = False - fake_throw = False - fake_result = None - genuine_result = None - try: - genuine_result = request.callback(genuine_zk) - except Exception as ex: - genuine_throw = True + try: + requests = generate_requests("/test_random_requests", 10) + print("Generated", len(requests), "requests") + genuine_zk = get_genuine_zk() + fake_zk = get_fake_zk() + genuine_zk.create("/test_random_requests") + fake_zk.create("/test_random_requests") + for i, request in enumerate(requests): + genuine_throw = False + fake_throw = False + fake_result = None + genuine_result = None + try: + genuine_result = request.callback(genuine_zk) + except Exception as ex: + print("i", i, "request", request) + print("Genuine exception", str(ex)) + genuine_throw = True - try: - fake_result = request.callback(fake_zk) - except Exception as ex: - fake_throw = True + try: + fake_result = request.callback(fake_zk) + except Exception as ex: + print("i", i, "request", request) + print("Fake exception", str(ex)) + fake_throw = True - assert fake_throw == genuine_throw, "Fake throw genuine not or vise versa" - assert fake_result == genuine_result, "Zookeeper results differ" - root_children_genuine = [elem for elem in list(sorted(genuine_zk.get_children("/"))) if elem not in ('clickhouse', 'zookeeper')] - root_children_fake = [elem for elem in list(sorted(fake_zk.get_children("/"))) if elem not in ('clickhouse', 'zookeeper')] - assert root_children_fake == root_children_genuine + assert fake_throw == genuine_throw, "Fake throw genuine not or vise versa request {}" + assert fake_result == genuine_result, "Zookeeper results differ" + root_children_genuine = [elem for elem in list(sorted(genuine_zk.get_children("/test_random_requests"))) if elem not in ('clickhouse', 'zookeeper')] + root_children_fake = [elem for elem in list(sorted(fake_zk.get_children("/test_random_requests"))) if elem not in ('clickhouse', 'zookeeper')] + assert root_children_fake == root_children_genuine + finally: + for zk in [genuine_zk, fake_zk]: + stop_zk(zk) def test_end_of_session(started_cluster): + fake_zk1 = None fake_zk2 = None genuine_zk1 = None @@ -401,13 +426,8 @@ def test_end_of_session(started_cluster): assert fake_ephemeral_event == genuine_ephemeral_event finally: - try: - for zk in [fake_zk1, fake_zk2, genuine_zk1, genuine_zk2]: - if zk: - zk.stop() - zk.close() - except: - pass + for zk in [fake_zk1, fake_zk2, genuine_zk1, genuine_zk2]: + stop_zk(zk) def test_end_of_watches_session(started_cluster): fake_zk1 = None @@ -442,91 +462,89 @@ def test_end_of_watches_session(started_cluster): assert dummy_set == 2 finally: - try: - for zk in [fake_zk1, fake_zk2]: - if zk: - zk.stop() - zk.close() - except: - pass + for zk in [fake_zk1, fake_zk2]: + stop_zk(zk) def test_concurrent_watches(started_cluster): - fake_zk = get_fake_zk() - fake_zk.restart() - global_path = "/test_concurrent_watches_0" - fake_zk.create(global_path) + try: + fake_zk = get_fake_zk() + fake_zk.restart() + global_path = "/test_concurrent_watches_0" + fake_zk.create(global_path) - dumb_watch_triggered_counter = 0 - all_paths_triggered = [] + dumb_watch_triggered_counter = 0 + all_paths_triggered = [] - existing_path = [] - all_paths_created = [] - watches_created = 0 - def create_path_and_watch(i): - nonlocal watches_created - nonlocal all_paths_created - fake_zk.ensure_path(global_path + "/" + str(i)) - # new function each time - def dumb_watch(event): - nonlocal dumb_watch_triggered_counter - dumb_watch_triggered_counter += 1 - nonlocal all_paths_triggered - all_paths_triggered.append(event.path) + existing_path = [] + all_paths_created = [] + watches_created = 0 + def create_path_and_watch(i): + nonlocal watches_created + nonlocal all_paths_created + fake_zk.ensure_path(global_path + "/" + str(i)) + # new function each time + def dumb_watch(event): + nonlocal dumb_watch_triggered_counter + dumb_watch_triggered_counter += 1 + nonlocal all_paths_triggered + all_paths_triggered.append(event.path) - fake_zk.get(global_path + "/" + str(i), watch=dumb_watch) - all_paths_created.append(global_path + "/" + str(i)) - watches_created += 1 - existing_path.append(i) + fake_zk.get(global_path + "/" + str(i), watch=dumb_watch) + all_paths_created.append(global_path + "/" + str(i)) + watches_created += 1 + existing_path.append(i) - trigger_called = 0 - def trigger_watch(i): - nonlocal trigger_called - trigger_called += 1 - fake_zk.set(global_path + "/" + str(i), b"somevalue") - try: - existing_path.remove(i) - except: - pass - - def call(total): - for i in range(total): - create_path_and_watch(random.randint(0, 1000)) - time.sleep(random.random() % 0.5) + trigger_called = 0 + def trigger_watch(i): + nonlocal trigger_called + trigger_called += 1 + fake_zk.set(global_path + "/" + str(i), b"somevalue") try: - rand_num = random.choice(existing_path) - trigger_watch(rand_num) - except: - pass - while existing_path: - try: - rand_num = random.choice(existing_path) - trigger_watch(rand_num) + existing_path.remove(i) except: pass - p = Pool(10) - arguments = [100] * 10 - watches_must_be_created = sum(arguments) - watches_trigger_must_be_called = sum(arguments) - watches_must_be_triggered = sum(arguments) - p.map(call, arguments) - p.close() + def call(total): + for i in range(total): + create_path_and_watch(random.randint(0, 1000)) + time.sleep(random.random() % 0.5) + try: + rand_num = random.choice(existing_path) + trigger_watch(rand_num) + except: + pass + while existing_path: + try: + rand_num = random.choice(existing_path) + trigger_watch(rand_num) + except: + pass - # waiting for late watches - for i in range(50): - if dumb_watch_triggered_counter == watches_must_be_triggered: - break + p = Pool(10) + arguments = [100] * 10 + watches_must_be_created = sum(arguments) + watches_trigger_must_be_called = sum(arguments) + watches_must_be_triggered = sum(arguments) + p.map(call, arguments) + p.close() - time.sleep(0.1) + # waiting for late watches + for i in range(50): + if dumb_watch_triggered_counter == watches_must_be_triggered: + break - assert watches_created == watches_must_be_created - assert trigger_called >= watches_trigger_must_be_called - assert len(existing_path) == 0 - if dumb_watch_triggered_counter != watches_must_be_triggered: - print("All created paths", all_paths_created) - print("All triggerred paths", all_paths_triggered) - print("All paths len", len(all_paths_created)) - print("All triggered len", len(all_paths_triggered)) - print("Diff", list(set(all_paths_created) - set(all_paths_triggered))) + time.sleep(0.1) - assert dumb_watch_triggered_counter == watches_must_be_triggered + assert watches_created == watches_must_be_created + assert trigger_called >= watches_trigger_must_be_called + assert len(existing_path) == 0 + if dumb_watch_triggered_counter != watches_must_be_triggered: + print("All created paths", all_paths_created) + print("All triggerred paths", all_paths_triggered) + print("All paths len", len(all_paths_created)) + print("All triggered len", len(all_paths_triggered)) + print("Diff", list(set(all_paths_created) - set(all_paths_triggered))) + + assert dumb_watch_triggered_counter == watches_must_be_triggered + finally: + stop_zk(fake_zk) diff --git a/tests/integration/test_testkeeper_persistent_log/__init__.py b/tests/integration/test_testkeeper_persistent_log/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_testkeeper_persistent_log/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_persistent_log/configs/enable_test_keeper.xml new file mode 100644 index 00000000000..a8b8991f959 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log/configs/enable_test_keeper.xml @@ -0,0 +1,21 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + + + 5000 + 10000 + trace + + + + + 1 + localhost + 44444 + + + + diff --git a/tests/integration/test_testkeeper_persistent_log/configs/logs_conf.xml b/tests/integration/test_testkeeper_persistent_log/configs/logs_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log/configs/logs_conf.xml @@ -0,0 +1,12 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml new file mode 100644 index 00000000000..12dc7fd9447 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml @@ -0,0 +1,8 @@ + + + + node1 + 9181 + + + diff --git a/tests/integration/test_testkeeper_persistent_log/test.py b/tests/integration/test_testkeeper_persistent_log/test.py new file mode 100644 index 00000000000..71fee94088f --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log/test.py @@ -0,0 +1,124 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time +from kazoo.client import KazooClient, KazooState + + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', main_configs=['configs/enable_test_keeper.xml', 'configs/logs_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) + + +def random_string(length): + return ''.join(random.choices(string.ascii_lowercase + string.digits, k=length)) + +def create_random_path(prefix="", depth=1): + if depth == 0: + return prefix + return create_random_path(os.path.join(prefix, random_string(3)), depth - 1) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def get_connection_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + def reset_listener(state): + nonlocal _fake_zk_instance + print("Fake zk callback called for state", state) + if state != KazooState.CONNECTED: + _fake_zk_instance._reset() + + _fake_zk_instance.add_listener(reset_listener) + _fake_zk_instance.start() + return _fake_zk_instance + +def test_state_after_restart(started_cluster): + try: + node_zk = None + node_zk2 = None + node_zk = get_connection_zk("node") + + node_zk.create("/test_state_after_restart", b"somevalue") + strs = [] + for i in range(100): + strs.append(random_string(123).encode()) + node_zk.create("/test_state_after_restart/node" + str(i), strs[i]) + + for i in range(100): + if i % 7 == 0: + node_zk.delete("/test_state_after_restart/node" + str(i)) + + node.restart_clickhouse(kill=True) + + node_zk2 = get_connection_zk("node") + + assert node_zk2.get("/test_state_after_restart")[0] == b"somevalue" + for i in range(100): + if i % 7 == 0: + assert node_zk2.exists("/test_state_after_restart/node" + str(i)) is None + else: + assert len(node_zk2.get("/test_state_after_restart/node" + str(i))[0]) == 123 + assert node_zk2.get("/test_state_after_restart/node" + str(i))[0] == strs[i] + finally: + try: + if node_zk is not None: + node_zk.stop() + node_zk.close() + + if node_zk2 is not None: + node_zk2.stop() + node_zk2.close() + except: + pass + + +# http://zookeeper-user.578899.n2.nabble.com/Why-are-ephemeral-nodes-written-to-disk-tp7583403p7583418.html +def test_ephemeral_after_restart(started_cluster): + try: + node_zk = None + node_zk2 = None + node_zk = get_connection_zk("node") + + node_zk.create("/test_ephemeral_after_restart", b"somevalue") + strs = [] + for i in range(100): + strs.append(random_string(123).encode()) + node_zk.create("/test_ephemeral_after_restart/node" + str(i), strs[i], ephemeral=True) + + for i in range(100): + if i % 7 == 0: + node_zk.delete("/test_ephemeral_after_restart/node" + str(i)) + + node.restart_clickhouse(kill=True) + + node_zk2 = get_connection_zk("node") + + assert node_zk2.get("/test_ephemeral_after_restart")[0] == b"somevalue" + for i in range(100): + if i % 7 == 0: + assert node_zk2.exists("/test_ephemeral_after_restart/node" + str(i)) is None + else: + assert len(node_zk2.get("/test_ephemeral_after_restart/node" + str(i))[0]) == 123 + assert node_zk2.get("/test_ephemeral_after_restart/node" + str(i))[0] == strs[i] + finally: + try: + if node_zk is not None: + node_zk.stop() + node_zk.close() + + if node_zk2 is not None: + node_zk2.stop() + node_zk2.close() + except: + pass From e82bd824d7818279db000f2019f5d2c82fefbb38 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 16:07:59 +0300 Subject: [PATCH 202/510] Fix restart replica in test --- .../test.py | 36 ++++++++++++++----- 1 file changed, 28 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py index 3b2867ef3c7..a1fd066ab83 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py @@ -87,7 +87,7 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node2.query("SYSTEM RESTART REPLICA ordinary.t1") + restart_replica_for_sure(node2, "ordinary.t1", "/clickhouse/t1/replicas/2") node2.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") break except Exception as ex: @@ -104,7 +104,7 @@ def test_blocade_leader(started_cluster): for i in range(100): try: - node3.query("SYSTEM RESTART REPLICA ordinary.t1") + restart_replica_for_sure(node3, "ordinary.t1", "/clickhouse/t1/replicas/3") node3.query("INSERT INTO ordinary.t1 SELECT rand() FROM numbers(100)") break except Exception as ex: @@ -122,7 +122,7 @@ def test_blocade_leader(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA ordinary.t1") + restart_replica_for_sure(node, "ordinary.t1", "/clickhouse/t1/replicas/{}".format(n + 1)) break except Exception as ex: try: @@ -150,7 +150,7 @@ def test_blocade_leader(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA ordinary.t1") + restart_replica_for_sure(node, "ordinary.t1", "/clickhouse/t1/replicas/{}".format(n + 1)) node.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) break except Exception as ex: @@ -188,6 +188,25 @@ def dump_zk(node, zk_path, replica_path): print("Parts") print(node.query("SELECT name FROM system.zookeeper WHERE path = '{}/parts' FORMAT Vertical".format(replica_path))) +def restart_replica_for_sure(node, table_name, zk_replica_path): + fake_zk = None + try: + node.query("DETACH TABLE {}".format(table_name)) + fake_zk = get_fake_zk(node.name) + if fake_zk.exists(zk_replica_path + "/is_active") is not None: + fake_zk.delete(zk_replica_path + "/is_active") + + node.query("ATTACH TABLE {}".format(table_name)) + except Exception as ex: + print("Exception", ex) + raise ex + finally: + if fake_zk: + fake_zk.stop() + fake_zk.close() + + + # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader_twice(started_cluster): @@ -211,7 +230,7 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: - node2.query("SYSTEM RESTART REPLICA ordinary.t2") + restart_replica_for_sure(node2, "ordinary.t2", "/clickhouse/t2/replicas/2") node2.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") break except Exception as ex: @@ -228,7 +247,8 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: - node3.query("SYSTEM RESTART REPLICA ordinary.t2") + + restart_replica_for_sure(node3, "ordinary.t2", "/clickhouse/t2/replicas/3") node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") break except Exception as ex: @@ -265,7 +285,7 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA ordinary.t2") + restart_replica_for_sure(node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1)) break except Exception as ex: try: @@ -296,7 +316,7 @@ def test_blocade_leader_twice(started_cluster): for n, node in enumerate([node1, node2, node3]): for i in range(100): try: - node.query("SYSTEM RESTART REPLICA ordinary.t2") + restart_replica_for_sure(node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1)) node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) break except Exception as ex: From ee4d3f7aa485f851831b9ce96c8d1b4b78f90589 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Wed, 17 Feb 2021 16:23:10 +0300 Subject: [PATCH 203/510] edited ; in queries, edited after review --- docs/en/sql-reference/functions/array-functions.md | 12 ++++++------ .../example-datasets/brown-benchmark.md | 6 +++--- docs/ru/sql-reference/functions/array-functions.md | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 48c5176f0e1..528d81b0a0b 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1315,7 +1315,7 @@ Type: [Int](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-ref Query: ``` sql -SELECT arrayMin([1, 2, 4]) AS res +SELECT arrayMin([1, 2, 4]) AS res; ``` Result: @@ -1329,7 +1329,7 @@ Result: Query: ``` sql -SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res +SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res; ``` Result: @@ -1367,7 +1367,7 @@ Type: [Int](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-ref Query: ```sql -SELECT arrayMax([1, 2, 4]) AS res +SELECT arrayMax([1, 2, 4]) AS res; ``` Result: @@ -1381,7 +1381,7 @@ Result: Query: ``` sql -SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res +SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res; ``` Result: @@ -1419,7 +1419,7 @@ Type: [Int](../../sql-reference/data-types/int-uint.md) or [Float](../../sql-ref Query: ```sql -SELECT arraySum([2,3]) AS res +SELECT arraySum([2,3]) AS res; ``` Result: @@ -1433,7 +1433,7 @@ Result: Query: ``` sql -SELECT arraySum(x -> x*x, [2, 3]) AS res +SELECT arraySum(x -> x*x, [2, 3]) AS res; ``` Result: diff --git a/docs/ru/getting-started/example-datasets/brown-benchmark.md b/docs/ru/getting-started/example-datasets/brown-benchmark.md index e4fe00ace93..23702e07fcd 100644 --- a/docs/ru/getting-started/example-datasets/brown-benchmark.md +++ b/docs/ru/getting-started/example-datasets/brown-benchmark.md @@ -5,7 +5,7 @@ toc_title: Brown University Benchmark # Brown University Benchmark -`MgBench` — это новый аналитический бенчмарк для данных журнала событий, сгенерированных машиной. Бенчмарк разработан [Andrew Crotty](http://cs.brown.edu/people/acrotty/). +`MgBench` — это аналитический тест производительности для данных журнала событий, сгенерированных машиной. Бенчмарк разработан [Andrew Crotty](http://cs.brown.edu/people/acrotty/). Скачать данные: ``` @@ -74,7 +74,7 @@ ENGINE = MergeTree() ORDER BY (event_type, log_time); ``` -Insert data: +Вставка данных: ``` clickhouse-client --query "INSERT INTO mgbench.logs1 FORMAT CSVWithNames" < mgbench1.csv @@ -82,7 +82,7 @@ clickhouse-client --query "INSERT INTO mgbench.logs2 FORMAT CSVWithNames" < mgbe clickhouse-client --query "INSERT INTO mgbench.logs3 FORMAT CSVWithNames" < mgbench3.csv ``` -Run benchmark queries: +Запуск тестов производительности: ``` -- Q1.1: What is the CPU/network utilization for each web server since midnight? diff --git a/docs/ru/sql-reference/functions/array-functions.md b/docs/ru/sql-reference/functions/array-functions.md index 7afd9da471e..9702ab13d5e 100644 --- a/docs/ru/sql-reference/functions/array-functions.md +++ b/docs/ru/sql-reference/functions/array-functions.md @@ -1162,7 +1162,7 @@ arrayMin(arr) Запрос: ``` sql -SELECT arrayMin([1, 2, 4]) AS res +SELECT arrayMin([1, 2, 4]) AS res; ``` Результат: @@ -1176,7 +1176,7 @@ SELECT arrayMin([1, 2, 4]) AS res Запрос: ``` sql -SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res +SELECT arrayMin(x -> (-x), [1, 2, 4]) AS res; ``` Результат: @@ -1214,7 +1214,7 @@ arrayMax(arr) Запрос: ```sql -SELECT arrayMax([1, 2, 4]) AS res +SELECT arrayMax([1, 2, 4]) AS res; ``` Результат: @@ -1228,7 +1228,7 @@ SELECT arrayMax([1, 2, 4]) AS res Запрос: ``` sql -SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res +SELECT arrayMax(x -> (-x), [1, 2, 4]) AS res; ``` Результат: @@ -1266,7 +1266,7 @@ arraySum(arr) Запрос: ```sql -SELECT arraySum([2,3]) AS res +SELECT arraySum([2,3]) AS res; ``` Результат: @@ -1280,7 +1280,7 @@ SELECT arraySum([2,3]) AS res Запрос: ``` sql -SELECT arraySum(x -> x*x, [2, 3]) AS res +SELECT arraySum(x -> x*x, [2, 3]) AS res; ``` Результат: From 499c100b12233e3a6fbd31066a4bac3914a650e1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 16:41:43 +0300 Subject: [PATCH 204/510] Better test --- .../test.py | 37 +++++++++++-------- 1 file changed, 22 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py index a1fd066ab83..49d86ab9fe8 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py @@ -55,7 +55,6 @@ def get_fake_zk(nodename, timeout=30.0): _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) def reset_listener(state): nonlocal _fake_zk_instance - print("Fake zk callback called for state", state) if state != KazooState.CONNECTED: _fake_zk_instance._reset() @@ -247,8 +246,8 @@ def test_blocade_leader_twice(started_cluster): for i in range(100): try: - restart_replica_for_sure(node3, "ordinary.t2", "/clickhouse/t2/replicas/3") + node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) node3.query("INSERT INTO ordinary.t2 SELECT rand() FROM numbers(100)") break except Exception as ex: @@ -263,6 +262,10 @@ def test_blocade_leader_twice(started_cluster): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node3" + node2.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) + + assert node2.query("SELECT COUNT() FROM ordinary.t2") == "210\n" + assert node3.query("SELECT COUNT() FROM ordinary.t2") == "210\n" # Total network partition pm.partition_instances(node3, node2) @@ -281,7 +284,6 @@ def test_blocade_leader_twice(started_cluster): except Exception as ex: time.sleep(0.5) - for n, node in enumerate([node1, node2, node3]): for i in range(100): try: @@ -313,24 +315,29 @@ def test_blocade_leader_twice(started_cluster): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect for node{}".format(n + 1) - for n, node in enumerate([node1, node2, node3]): for i in range(100): - try: - restart_replica_for_sure(node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1)) - node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - break - except Exception as ex: + all_done = True + for n, node in enumerate([node1, node2, node3]): try: - node.query("ATTACH TABLE ordinary.t2") - except Exception as attach_ex: - print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) + restart_replica_for_sure(node, "ordinary.t2", "/clickhouse/t2/replicas/{}".format(n + 1)) + node.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) + break + except Exception as ex: + all_done = False + try: + node.query("ATTACH TABLE ordinary.t2") + except Exception as attach_ex: + print("Got exception node{}".format(n + 1), smaller_exception(attach_ex)) - print("Got exception node{}".format(n + 1), smaller_exception(ex)) - time.sleep(0.5) + print("Got exception node{}".format(n + 1), smaller_exception(ex)) + time.sleep(0.5) + + if all_done: + break else: for num, node in enumerate([node1, node2, node3]): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) - assert False, "Cannot reconnect for node{}".format(n + 1) + assert False, "Cannot reconnect in i {} retries".format(i) assert node1.query("SELECT COUNT() FROM ordinary.t2") == "510\n" if node2.query("SELECT COUNT() FROM ordinary.t2") != "510\n": From 8cecb533ca53038fe70a55fc4aa46e7ab2b0bef9 Mon Sep 17 00:00:00 2001 From: Marvin Taschenberger <45663148+Taschenbergerm@users.noreply.github.com> Date: Wed, 17 Feb 2021 15:03:09 +0100 Subject: [PATCH 205/510] Update argmax.md --- .../aggregate-functions/reference/argmax.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 7639117042f..1af188ad026 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -17,12 +17,12 @@ argMax(arg, val) or ``` sql -argMax(tuple(arg, val)) +argMax(tuple(arg1, arg2), val) ``` **Arguments** -- `arg` — Argument. +- `arg{i}` — Argument. - `val` — Value. **Returned value** @@ -33,7 +33,7 @@ Type: matches `arg` type. For tuple in the input: -- Tuple `(arg, val)`, where `val` is the maximum value and `arg` is a corresponding value. +- Tuple `(arg1, arg2)`, where `arg1` and `arg2` are the corresponding values. Type: [Tuple](../../../sql-reference/data-types/tuple.md). @@ -52,13 +52,13 @@ Input table: Query: ``` sql -SELECT argMax(user, salary), argMax(tuple(user, salary)) FROM salary; +SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; ``` Result: ``` text -┌─argMax(user, salary)─┬─argMax(tuple(user, salary))─┐ +┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┐ │ director │ ('director',5000) │ └──────────────────────┴─────────────────────────────┘ ``` From bb4ced05f9da997c987c7f520f423fd3892bb7d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 17:52:32 +0300 Subject: [PATCH 206/510] Fix fast test --- docker/test/fasttest/run.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 90663102f17..202e2f12a1a 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -361,7 +361,7 @@ function run_tests stop_server ||: # Clean the data so that there is no interference from the previous test run. - rm -rf "$FASTTEST_DATA"/{{meta,}data,user_files} ||: + rm -rf "$FASTTEST_DATA"/{{meta,}data,user_files,coordination} ||: start_server From 6522bfc402260b2b4edfd4c2f0ab55a662296e63 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Feb 2021 19:54:11 +0300 Subject: [PATCH 207/510] Support for DIstinct, sorting steps. --- src/Interpreters/ActionsDAG.cpp | 2 +- src/Processors/QueryPlan/CreatingSetsStep.h | 2 +- src/Processors/QueryPlan/CubeStep.cpp | 5 ++ src/Processors/QueryPlan/CubeStep.h | 2 + src/Processors/QueryPlan/FillingStep.h | 2 + .../Optimizations/filterPushDown.cpp | 68 +++++++++++++++++-- 6 files changed, 74 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 691905bed27..8b6013a4365 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1389,7 +1389,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, for (const auto * predicate : selected_predicates) args.emplace_back(nodes_mapping[predicate]); - result_predicate = &actions->addFunction(func_builder_and, args, {}, true); + result_predicate = &actions->addFunction(func_builder_and, args, {}, true, false); } actions->index.insert(result_predicate); diff --git a/src/Processors/QueryPlan/CreatingSetsStep.h b/src/Processors/QueryPlan/CreatingSetsStep.h index ec13ab2052e..97821cb63d3 100644 --- a/src/Processors/QueryPlan/CreatingSetsStep.h +++ b/src/Processors/QueryPlan/CreatingSetsStep.h @@ -34,7 +34,7 @@ private: class CreatingSetsStep : public IQueryPlanStep { public: - CreatingSetsStep(DataStreams input_streams_); + explicit CreatingSetsStep(DataStreams input_streams_); String getName() const override { return "CreatingSets"; } diff --git a/src/Processors/QueryPlan/CubeStep.cpp b/src/Processors/QueryPlan/CubeStep.cpp index de8bb2b3d43..6a0ec33402b 100644 --- a/src/Processors/QueryPlan/CubeStep.cpp +++ b/src/Processors/QueryPlan/CubeStep.cpp @@ -43,4 +43,9 @@ void CubeStep::transformPipeline(QueryPipeline & pipeline) }); } +const Aggregator::Params & CubeStep::getParams() const +{ + return params->params; +} + } diff --git a/src/Processors/QueryPlan/CubeStep.h b/src/Processors/QueryPlan/CubeStep.h index 707f62ce7d6..f67a03dc7e2 100644 --- a/src/Processors/QueryPlan/CubeStep.h +++ b/src/Processors/QueryPlan/CubeStep.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include namespace DB { @@ -18,6 +19,7 @@ public: void transformPipeline(QueryPipeline & pipeline) override; + const Aggregator::Params & getParams() const; private: AggregatingTransformParamsPtr params; }; diff --git a/src/Processors/QueryPlan/FillingStep.h b/src/Processors/QueryPlan/FillingStep.h index 85736464a6c..c8d1f74c6ca 100644 --- a/src/Processors/QueryPlan/FillingStep.h +++ b/src/Processors/QueryPlan/FillingStep.h @@ -17,6 +17,8 @@ public: void describeActions(FormatSettings & settings) const override; + const SortDescription & getSortDescription() const { return sort_description; } + private: SortDescription sort_description; }; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 39f24a32b45..74c4fa6f329 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -4,9 +4,15 @@ #include #include #include +#include #include #include #include +#include "Processors/QueryPlan/FinishSortingStep.h" +#include "Processors/QueryPlan/MergeSortingStep.h" +#include "Processors/QueryPlan/MergingSortedStep.h" +#include "Processors/QueryPlan/PartialSortingStep.h" +#include #include namespace DB::ErrorCodes @@ -79,6 +85,30 @@ static size_t tryAddNewFilterStep( return 3; } +static Names getAggregatinKeys(const Aggregator::Params & params) +{ + Names keys; + keys.reserve(params.keys.size()); + for (auto pos : params.keys) + keys.push_back(params.src_header.getByPosition(pos).name); + + return keys; +} + +// static NameSet getColumnNamesFromSortDescription(const SortDescription & sort_desc, const Block & header) +// { +// NameSet names; +// for (const auto & column : sort_desc) +// { +// if (!column.column_name.empty()) +// names.insert(column.column_name); +// else +// names.insert(header.safeGetByPosition(column.column_number).name); +// } + +// return names; +// } + size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) { if (parent_node->children.size() != 1) @@ -96,11 +126,7 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (auto * aggregating = typeid_cast(child.get())) { const auto & params = aggregating->getParams(); - - Names keys; - keys.reserve(params.keys.size()); - for (auto pos : params.keys) - keys.push_back(params.src_header.getByPosition(pos).name); + Names keys = getAggregatinKeys(params); if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, keys)) return updated_steps; @@ -124,6 +150,38 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return updated_steps; } + if (auto * distinct = typeid_cast(child.get())) + { + Names allowed_inputs = distinct->getOutputStream().header.getNames(); + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) + return updated_steps; + } + + /// TODO. + /// We can filter earlier if expression does not depend on WITH FILL columns. + /// But we cannot just push down condition, because other column may be filled with defaults. + /// + /// It is possible to filter columns before and after WITH FILL, but such change is not idempotent. + /// So, appliying this to pair (Filter -> Filling) several times will create several similar filters. + // if (auto * filling = typeid_cast(child.get())) + // { + // } + + /// Same reason for Cube + // if (auto * cube = typeid_cast(child.get())) + // { + // } + + if (typeid_cast(child.get()) + || typeid_cast(child.get()) + || typeid_cast(child.get()) + || typeid_cast(child.get())) + { + Names allowed_inputs = child->getOutputStream().header.getNames(); + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, allowed_inputs)) + return updated_steps; + } + return 0; } From e5b9c42860cce08b0b94f7863dbeb6f38b066d83 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Feb 2021 19:54:37 +0300 Subject: [PATCH 208/510] Update test. --- .../01655_plan_optimizations.reference | 70 +++++++++++++++ .../0_stateless/01655_plan_optimizations.sh | 85 +++++++++++++++++-- 2 files changed, 149 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 1e638829c74..7bc75dc0bf6 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -13,32 +13,102 @@ Limit 10 > filter should be pushed down after aggregating Aggregating Filter +0 1 +1 2 +2 3 +3 4 +4 5 +5 6 +6 7 +7 8 +8 9 +9 10 > filter should be pushed down after aggregating, column after aggregation is const COLUMN Const(UInt8) -> notEquals(y, 0) Aggregating Filter Filter +0 1 1 +1 2 1 +2 3 1 +3 4 1 +4 5 1 +5 6 1 +6 7 1 +7 8 1 +8 9 1 +9 10 1 > one condition of filter should be pushed down after aggregating, other condition is aliased Filter column ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4)) Aggregating Filter column: notEquals(y, 0) +0 1 +1 2 +2 3 +3 4 +5 6 +6 7 +7 8 +8 9 +9 10 > one condition of filter should be pushed down after aggregating, other condition is casted Filter column FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) +0 1 +1 2 +2 3 +3 4 +5 6 +6 7 +7 8 +8 9 +9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column FUNCTION and(minus(s, 4) :: 2, minus(s, 8) :: 1) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) +0 1 +1 2 +2 3 +3 4 +5 6 +6 7 +7 8 +9 10 > two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased Filter column ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating Filter column: and(minus(y, 4), notEquals(y, 0)) +0 1 +1 2 +2 3 +4 5 +5 6 +6 7 +7 8 +9 10 > filter is split, one part is filtered before ARRAY JOIN Filter column: and(notEquals(y, 2), notEquals(x, 0)) ARRAY JOIN x Filter column: notEquals(y, 2) +1 3 +> filter is pushed down before Distinct +Distinct +Distinct +Filter column: notEquals(y, 2) +0 0 +0 1 +1 0 +1 1 +> filter is pushed down before sorting steps +MergingSorted +MergeSorting +PartialSorting +Filter column: and(notEquals(x, 0), notEquals(y, 0)) +1 2 +1 1 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index ccd331df45e..f770643fc41 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -16,49 +16,122 @@ $CLICKHOUSE_CLIENT -q " select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter" +$CLICKHOUSE_CLIENT -q " + select s, y from (select sum(x) as s, y from ( + select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 order by s, y + settings enable_optimize_predicate_expression=0" echo "> filter should be pushed down after aggregating, column after aggregation is const" $CLICKHOUSE_CLIENT -q " - explain actions = 1 select *, y != 0 from (select sum(x), y from ( + explain actions = 1 select s, y, y != 0 from (select sum(x) as s, y from ( select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter\|COLUMN Const(UInt8) -> notEquals(y, 0)" +$CLICKHOUSE_CLIENT -q " + select s, y, y != 0 from (select sum(x) as s, y from ( + select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 order by s, y, y != 0 + settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other condition is aliased" $CLICKHOUSE_CLIENT -q " - explain actions = 1 select * from ( + explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|ALIAS notEquals(s, 4) :: 1 -> and(notEquals(y, 0), notEquals(s, 4))" +$CLICKHOUSE_CLIENT -q " + select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s != 4 order by s, y + settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other condition is casted" $CLICKHOUSE_CLIENT -q " - explain actions = 1 select * from ( + explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION CAST(minus(s, 4) :: 1, UInt8 :: 3) -> and(notEquals(y, 0), minus(s, 4))" +$CLICKHOUSE_CLIENT -q " + select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s - 4 order by s, y + settings enable_optimize_predicate_expression=0" echo "> one condition of filter should be pushed down after aggregating, other two conditions are ANDed" $CLICKHOUSE_CLIENT -q " - explain actions = 1 select * from ( + explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 2, minus(s, 8) :: 1) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" +$CLICKHOUSE_CLIENT -q " + select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s - 8 and s - 4 order by s, y + settings enable_optimize_predicate_expression=0" echo "> two conditions of filter should be pushed down after aggregating and ANDed, one condition is aliased" $CLICKHOUSE_CLIENT -q " - explain optimize = 1, actions = 1 select * from ( + explain actions = 1 select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | grep -o "Aggregating\|Filter column\|Filter column: and(minus(y, 4), notEquals(y, 0))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" +$CLICKHOUSE_CLIENT -q " + select s, y from ( + select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y + ) where y != 0 and s != 8 and y - 4 order by s, y + settings enable_optimize_predicate_expression=0" echo "> filter is split, one part is filtered before ARRAY JOIN" $CLICKHOUSE_CLIENT -q " explain actions = 1 select x, y from ( select range(number) as x, number + 1 as y from numbers(3) ) array join x where y != 2 and x != 0" | - grep -o "Filter column: and(notEquals(y, 2), notEquals(x, 0))\|ARRAY JOIN x\|Filter column: notEquals(y, 2)" \ No newline at end of file + grep -o "Filter column: and(notEquals(y, 2), notEquals(x, 0))\|ARRAY JOIN x\|Filter column: notEquals(y, 2)" +$CLICKHOUSE_CLIENT -q " + select x, y from ( + select range(number) as x, number + 1 as y from numbers(3) + ) array join x where y != 2 and x != 0 order by x, y" + +# echo "> filter is split, one part is filtered before Aggregating and Cube" +# $CLICKHOUSE_CLIENT -q " +# explain actions = 1 select * from ( +# select sum(x) as s, x, y from (select number as x, number + 1 as y from numbers(10)) group by x, y with cube +# ) where y != 0 and s != 4 +# settings enable_optimize_predicate_expression=0" | +# grep -o "Cube\|Aggregating\|Filter column: notEquals(y, 0)" +# $CLICKHOUSE_CLIENT -q " +# select s, x, y from ( +# select sum(x) as s, x, y from (select number as x, number + 1 as y from numbers(10)) group by x, y with cube +# ) where y != 0 and s != 4 order by s, x, y +# settings enable_optimize_predicate_expression=0" + +echo "> filter is pushed down before Distinct" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select x, y from ( + select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) + ) where y != 2 + settings enable_optimize_predicate_expression=0" | + grep -o "Distinct\|Filter column: notEquals(y, 2)" +$CLICKHOUSE_CLIENT -q " + select x, y from ( + select distinct x, y from (select number % 2 as x, number % 3 as y from numbers(10)) + ) where y != 2 order by x, y + settings enable_optimize_predicate_expression=0" + +echo "> filter is pushed down before sorting steps" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select x, y from ( + select number % 2 as x, number % 3 as y from numbers(6) order by y desc + ) where x != 0 and y != 0 + settings enable_optimize_predicate_expression = 0" | + grep -o "MergingSorted\|MergeSorting\|PartialSorting\|Filter column: and(notEquals(x, 0), notEquals(y, 0))" +$CLICKHOUSE_CLIENT -q " + select x, y from ( + select number % 2 as x, number % 3 as y from numbers(6) order by y desc + ) where x != 0 and y != 0 + settings enable_optimize_predicate_expression = 0" From f6278ed429dc2231d68aa5179e63b3bb635d081a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Feb 2021 19:56:17 +0300 Subject: [PATCH 209/510] Support for DIstinct, sorting steps. --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 74c4fa6f329..02e1914504d 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -60,12 +60,12 @@ static size_t tryAddNewFilterStep( "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", filter_column_name, expression->dumpDAG()); - std::cerr << "replacing to expr because filter " << filter_column_name << " was removed\n"; + // std::cerr << "replacing to expr because filter " << filter_column_name << " was removed\n"; parent = std::make_unique(child->getOutputStream(), expression); } else if ((*it)->column && isColumnConst(*(*it)->column)) { - std::cerr << "replacing to expr because filter is const\n"; + // std::cerr << "replacing to expr because filter is const\n"; parent = std::make_unique(child->getOutputStream(), expression); } From 1b78de2142062edcdd0f8f084758a54e3f03867d Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 17 Feb 2021 20:34:52 +0300 Subject: [PATCH 210/510] Use fibers in HedgedRequests --- src/Client/Connection.cpp | 258 +++++++---------- src/Client/Connection.h | 41 ++- src/Client/ConnectionEstablisher.cpp | 233 ++++++++++++++++ src/Client/ConnectionEstablisher.h | 116 ++++++++ src/Client/ConnectionPoolWithFailover.cpp | 213 +------------- src/Client/ConnectionPoolWithFailover.h | 45 --- src/Client/HedgedConnections.cpp | 147 +++++----- src/Client/HedgedConnections.h | 25 +- src/Client/HedgedConnectionsFactory.cpp | 261 ++++++------------ src/Client/HedgedConnectionsFactory.h | 55 ++-- src/Client/MultiplexedConnections.cpp | 6 +- src/Client/PacketReceiver.h | 143 ++++++++++ src/Client/ya.make | 1 + src/Common/Epoll.cpp | 8 +- src/Core/Defines.h | 3 +- src/Core/Settings.h | 8 +- src/IO/ConnectionTimeouts.h | 19 +- src/IO/ConnectionTimeoutsContext.h | 3 +- src/Server/TCPHandler.cpp | 35 +-- .../test_distributed_load_balancing/test.py | 4 + .../integration/test_hedged_requests/test.py | 169 +++--------- .../test_hedged_requests_parallel/test.py | 92 +++--- 22 files changed, 931 insertions(+), 954 deletions(-) create mode 100644 src/Client/ConnectionEstablisher.cpp create mode 100644 src/Client/ConnectionEstablisher.h create mode 100644 src/Client/PacketReceiver.h diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index a68ab6df34e..d30a6555da5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -59,43 +59,17 @@ namespace ErrorCodes void Connection::connect(const ConnectionTimeouts & timeouts) -{ - if (connected) - disconnect(); - - prepare(timeouts); - sendHello(); - receiveHello(); - - LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", - server_name, server_version_major, server_version_minor, server_version_patch); -} - - -void Connection::disconnect() -{ - LOG_DEBUG(log, "disconnect"); - maybe_compressed_out = nullptr; - in = nullptr; - last_input_packet_type.reset(); - out = nullptr; // can write to socket - if (socket) - socket->close(); - socket = nullptr; - connected = false; -} - -void Connection::prepare(const ConnectionTimeouts & timeouts) { try { - LOG_TRACE( - log_wrapper.get(), - "Connecting. Database: {}. User: {}{}{}", - default_database.empty() ? "(not specified)" : default_database, - user, - static_cast(secure) ? ". Secure" : "", - static_cast(compression) ? "" : ". Uncompressed"); + if (connected) + disconnect(); + + LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", + default_database.empty() ? "(not specified)" : default_database, + user, + static_cast(secure) ? ". Secure" : "", + static_cast(compression) ? "" : ". Uncompressed"); if (static_cast(secure)) { @@ -105,7 +79,7 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) /// we resolve the ip when we open SecureStreamSocket, so to make Server Name Indication (SNI) /// work we need to pass host name separately. It will be send into TLS Hello packet to let /// the server know which host we want to talk with (single IP can process requests for multiple hosts using SNI). - static_cast(socket.get())->setPeerHostName(host); + static_cast(socket.get())->setPeerHostName(host); #else throw Exception{"tcp_secure protocol is disabled because poco library was built without NetSSL support.", ErrorCodes::SUPPORT_IS_DISABLED}; #endif @@ -125,21 +99,28 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) if (timeouts.tcp_keep_alive_timeout.totalSeconds()) { socket->setKeepAlive(true); - socket->setOption( - IPPROTO_TCP, + socket->setOption(IPPROTO_TCP, #if defined(TCP_KEEPALIVE) TCP_KEEPALIVE #else - TCP_KEEPIDLE // __APPLE__ + TCP_KEEPIDLE // __APPLE__ #endif - , - timeouts.tcp_keep_alive_timeout); + , timeouts.tcp_keep_alive_timeout); } in = std::make_shared(*socket); + if (async_callback) + in->setAsyncCallback(std::move(async_callback)); + out = std::make_shared(*socket); connected = true; + + sendHello(); + receiveHello(); + + LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", + server_name, server_version_major, server_version_minor, server_version_patch); } catch (Poco::Net::NetException & e) { @@ -158,12 +139,21 @@ void Connection::prepare(const ConnectionTimeouts & timeouts) } +void Connection::disconnect() +{ + maybe_compressed_out = nullptr; + in = nullptr; + last_input_packet_type.reset(); + out = nullptr; // can write to socket + if (socket) + socket->close(); + socket = nullptr; + connected = false; +} + void Connection::sendHello() { - LOG_DEBUG(log_wrapper.get(), "sendHello"); - try - { - /** Disallow control characters in user controlled parameters + /** Disallow control characters in user controlled parameters * to mitigate the possibility of SSRF. * The user may do server side requests with 'remote' table function. * Malicious user with full r/w access to ClickHouse @@ -172,119 +162,84 @@ void Connection::sendHello() * Limiting number of possible characters in user-controlled part of handshake * will mitigate this possibility but doesn't solve it completely. */ - auto has_control_character = [](const std::string & s) - { - for (auto c : s) - if (isControlASCII(c)) - return true; - return false; - }; + auto has_control_character = [](const std::string & s) + { + for (auto c : s) + if (isControlASCII(c)) + return true; + return false; + }; - if (has_control_character(default_database) || has_control_character(user) || has_control_character(password)) - throw Exception( - "Parameters 'default_database', 'user' and 'password' must not contain ASCII control characters", - ErrorCodes::BAD_ARGUMENTS); + if (has_control_character(default_database) + || has_control_character(user) + || has_control_character(password)) + throw Exception("Parameters 'default_database', 'user' and 'password' must not contain ASCII control characters", ErrorCodes::BAD_ARGUMENTS); - writeVarUInt(Protocol::Client::Hello, *out); - writeStringBinary((DBMS_NAME " ") + client_name, *out); - writeVarUInt(DBMS_VERSION_MAJOR, *out); - writeVarUInt(DBMS_VERSION_MINOR, *out); - // NOTE For backward compatibility of the protocol, client cannot send its version_patch. - writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); - writeStringBinary(default_database, *out); - /// If interserver-secret is used, one do not need password - /// (NOTE we do not check for DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET, since we cannot ignore inter-server secret if it was requested) - if (!cluster_secret.empty()) - { - writeStringBinary(USER_INTERSERVER_MARKER, *out); - writeStringBinary("" /* password */, *out); + writeVarUInt(Protocol::Client::Hello, *out); + writeStringBinary((DBMS_NAME " ") + client_name, *out); + writeVarUInt(DBMS_VERSION_MAJOR, *out); + writeVarUInt(DBMS_VERSION_MINOR, *out); + // NOTE For backward compatibility of the protocol, client cannot send its version_patch. + writeVarUInt(DBMS_TCP_PROTOCOL_VERSION, *out); + writeStringBinary(default_database, *out); + /// If interserver-secret is used, one do not need password + /// (NOTE we do not check for DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET, since we cannot ignore inter-server secret if it was requested) + if (!cluster_secret.empty()) + { + writeStringBinary(USER_INTERSERVER_MARKER, *out); + writeStringBinary("" /* password */, *out); #if USE_SSL - sendClusterNameAndSalt(); + sendClusterNameAndSalt(); #else - throw Exception( - "Inter-server secret support is disabled, because ClickHouse was built without SSL library", - ErrorCodes::SUPPORT_IS_DISABLED); + throw Exception( + "Inter-server secret support is disabled, because ClickHouse was built without SSL library", + ErrorCodes::SUPPORT_IS_DISABLED); #endif - } - else - { - writeStringBinary(user, *out); - writeStringBinary(password, *out); - } - - out->next(); } - catch (Poco::Net::NetException & e) + else { - disconnect(); - - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::NETWORK_ERROR); + writeStringBinary(user, *out); + writeStringBinary(password, *out); } - catch (Poco::TimeoutException & e) - { - disconnect(); - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); - } + out->next(); } - void Connection::receiveHello() { - LOG_DEBUG(log_wrapper.get(), "receiveHello"); + /// Receive hello packet. + UInt64 packet_type = 0; - try + /// Prevent read after eof in readVarUInt in case of reset connection + /// (Poco should throw such exception while reading from socket but + /// sometimes it doesn't for unknown reason) + if (in->eof()) + throw Poco::Net::NetException("Connection reset by peer"); + + readVarUInt(packet_type, *in); + if (packet_type == Protocol::Server::Hello) { - /// Receive hello packet. - UInt64 packet_type = 0; - - /// Prevent read after eof in readVarUInt in case of reset connection - /// (Poco should throw such exception while reading from socket but - /// sometimes it doesn't for unknown reason) - if (in->eof()) - throw Poco::Net::NetException("Connection reset by peer"); - - readVarUInt(packet_type, *in); - if (packet_type == Protocol::Server::Hello) - { - readStringBinary(server_name, *in); - readVarUInt(server_version_major, *in); - readVarUInt(server_version_minor, *in); - readVarUInt(server_revision, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) - readStringBinary(server_timezone, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) - readStringBinary(server_display_name, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) - readVarUInt(server_version_patch, *in); - else - server_version_patch = server_revision; - } - else if (packet_type == Protocol::Server::Exception) - receiveException()->rethrow(); + readStringBinary(server_name, *in); + readVarUInt(server_version_major, *in); + readVarUInt(server_version_minor, *in); + readVarUInt(server_revision, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + readStringBinary(server_timezone, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) + readStringBinary(server_display_name, *in); + if (server_revision >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) + readVarUInt(server_version_patch, *in); else - { - /// Close connection, to not stay in unsynchronised state. - disconnect(); - throwUnexpectedPacket(packet_type, "Hello or Exception"); - } + server_version_patch = server_revision; } - catch (Poco::Net::NetException & e) + else if (packet_type == Protocol::Server::Exception) + receiveException()->rethrow(); + else { + /// Close connection, to not stay in unsynchronised state. disconnect(); - - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::NETWORK_ERROR); - } - catch (Poco::TimeoutException & e) - { - disconnect(); - - /// Add server address to exception. Also Exception will remember stack trace. It's a pity that more precise exception type is lost. - throw NetException(e.displayText() + " (" + getDescription() + ")", ErrorCodes::SOCKET_TIMEOUT); + throwUnexpectedPacket(packet_type, "Hello or Exception"); } } @@ -425,24 +380,9 @@ TablesStatusResponse Connection::getTablesStatus(const ConnectionTimeouts & time TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); - sendTablesStatusRequest(request); - TablesStatusResponse response = receiveTablesStatusResponse(); - - return response; -} - -void Connection::sendTablesStatusRequest(const TablesStatusRequest & request) -{ - LOG_DEBUG(log_wrapper.get(), "sendTablesStatusRequest"); - writeVarUInt(Protocol::Client::TablesStatusRequest, *out); request.write(*out, server_revision); out->next(); -} - -TablesStatusResponse Connection::receiveTablesStatusResponse() -{ - LOG_DEBUG(log_wrapper.get(), "receiveTablesStatusResponse"); UInt64 response_type = 0; readVarUInt(response_type, *in); @@ -457,6 +397,7 @@ TablesStatusResponse Connection::receiveTablesStatusResponse() return response; } + void Connection::sendQuery( const ConnectionTimeouts & timeouts, const String & query, @@ -466,8 +407,6 @@ void Connection::sendQuery( const ClientInfo * client_info, bool with_pending_data) { - LOG_DEBUG(log_wrapper.get(), "sendQuery"); - if (!connected) connect(timeouts); @@ -565,8 +504,6 @@ void Connection::sendQuery( void Connection::sendCancel() { - LOG_DEBUG(log_wrapper.get(), "sendCancel"); - /// If we already disconnected. if (!out) return; @@ -815,13 +752,8 @@ std::optional Connection::checkPacket(size_t timeout_microseconds) } -Packet Connection::receivePacket(AsyncCallback async_callback) +Packet Connection::receivePacket() { - LOG_DEBUG(log_wrapper.get(), "receivePacket"); - - in->setAsyncCallback(std::move(async_callback)); - SCOPE_EXIT(in->setAsyncCallback({})); - try { Packet res; @@ -896,8 +828,6 @@ Packet Connection::receivePacket(AsyncCallback async_callback) Block Connection::receiveData() { - LOG_DEBUG(log_wrapper.get(), "receiveData"); - initBlockInput(); return receiveDataImpl(block_in); } diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 1546f42f382..dd501b5f6ef 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -175,8 +175,7 @@ public: std::optional checkPacket(size_t timeout_microseconds = 0); /// Receive packet from server. - /// Each time read blocks and async_callback is set, it will be called. You can poll socket inside it. - Packet receivePacket(AsyncCallback async_callback = {}); + Packet receivePacket(); /// If not connected yet, or if connection is broken - then connect. If cannot connect - throw an exception. void forceConnected(const ConnectionTimeouts & timeouts); @@ -195,19 +194,15 @@ public: size_t outBytesCount() const { return out ? out->count() : 0; } size_t inBytesCount() const { return in ? in->count() : 0; } - /// Make preparation before sending Hello in connect - void prepare(const ConnectionTimeouts & timeouts); - - void sendHello(); - - void receiveHello(); - - void sendTablesStatusRequest(const TablesStatusRequest & request); - - TablesStatusResponse receiveTablesStatusResponse(); - Poco::Net::Socket * getSocket() { return socket.get(); } + void setAsyncCallback(AsyncCallback async_callback_) + { + async_callback = std::move(async_callback_); + if (in) + in->setAsyncCallback(std::move(async_callback)); + } + private: String host; UInt16 port; @@ -295,7 +290,11 @@ private: LoggerWrapper log_wrapper; + AsyncCallback async_callback; + void connect(const ConnectionTimeouts & timeouts); + void sendHello(); + void receiveHello(); #if USE_SSL void sendClusterNameAndSalt(); @@ -318,4 +317,20 @@ private: [[noreturn]] void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const; }; +class AsyncCallbackSetter +{ +public: + AsyncCallbackSetter(Connection * connection_, AsyncCallback async_callback) : connection(connection_) + { + connection->setAsyncCallback(std::move(async_callback)); + } + + ~AsyncCallbackSetter() + { + connection->setAsyncCallback({}); + } +private: + Connection * connection; +}; + } diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp new file mode 100644 index 00000000000..e529d366fdc --- /dev/null +++ b/src/Client/ConnectionEstablisher.cpp @@ -0,0 +1,233 @@ +#include +#include +#include + +namespace ProfileEvents +{ + extern const Event DistributedConnectionMissingTable; + extern const Event DistributedConnectionStaleReplica; +} + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int NETWORK_ERROR; + extern const int SOCKET_TIMEOUT; +} + + +ConnectionEstablisher::ConnectionEstablisher( + IConnectionPool * pool_, + const ConnectionTimeouts * timeouts_, + const Settings * settings_, + const QualifiedTableName * table_to_check_) + : pool(pool_), timeouts(timeouts_), settings(settings_), table_to_check(table_to_check_), + stage(Stage::INITIAL), log(&Poco::Logger::get("ConnectionEstablisher")) +{ +#if defined(OS_LINUX) + epoll.add(receive_timeout.getDescriptor()); +#endif +} + +void ConnectionEstablisher::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &) +{ +#if defined(OS_LINUX) + if (connection_establisher.socket_fd != fd) + { + if (connection_establisher.socket_fd != -1) + connection_establisher.epoll.remove(connection_establisher.socket_fd); + + connection_establisher.epoll.add(fd); + connection_establisher.socket_fd = fd; + } + + connection_establisher.receive_timeout.setRelative(timeout); + fiber = std::move(fiber).resume(); + connection_establisher.receive_timeout.reset(); +#endif +} + +Fiber ConnectionEstablisher::Routine::operator()(Fiber && sink) +{ + try + { + connection_establisher.establishConnection(ReadCallback{connection_establisher, sink}); + } + catch (const boost::context::detail::forced_unwind &) + { + /// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited + /// It should not be caught or it will segfault. + /// Other exceptions must be caught + throw; + } + catch (...) + { + connection_establisher.exception = std::current_exception(); + } + + return std::move(sink); +} + +void ConnectionEstablisher::resume() +{ + if (!fiber_created) + { + reset(); + fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); + fiber_created = true; + resumeFiber(); + return; + } + +#if defined(OS_LINUX) + bool is_socket_ready = false; + bool is_receive_timeout_alarmed = false; + + epoll_event events[2]; + events[0].data.fd = events[1].data.fd; + size_t ready_count = epoll.getManyReady(2, events, true); + for (size_t i = 0; i != ready_count; ++i) + { + if (events[i].data.fd == socket_fd) + is_socket_ready = true; + if (events[i].data.fd == receive_timeout.getDescriptor()) + is_receive_timeout_alarmed = true; + } + + if (is_receive_timeout_alarmed && !is_socket_ready) + processReceiveTimeout(); +#endif + + resumeFiber(); +} + +void ConnectionEstablisher::cancel() +{ + destroyFiber(); + reset(); +} + +void ConnectionEstablisher::processReceiveTimeout() +{ +#if defined(OS_LINUX) + destroyFiber(); + stage = Stage::FAILED; + fail_message = "Code: 209, e.displayText() = DB::NetException: Timeout exceeded while reading from socket (" + result.entry->getDescription() + ")"; + epoll.remove(socket_fd); + resetResult(); +#endif +} + +void ConnectionEstablisher::resetResult() +{ + if (!result.entry.isNull()) + { + result.entry->disconnect(); + result.reset(); + } +} + +void ConnectionEstablisher::reset() +{ + stage = Stage::INITIAL; + resetResult(); + fail_message.clear(); + socket_fd = -1; +} + +void ConnectionEstablisher::resumeFiber() +{ + fiber = std::move(fiber).resume(); + + if (exception) + std::rethrow_exception(std::move(exception)); + + if (stage == Stage::FAILED) + destroyFiber(); +} + +void ConnectionEstablisher::destroyFiber() +{ + Fiber to_destroy = std::move(fiber); + fiber_created = false; +} + +void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) +{ + try + { + stage = Stage::IN_PROCESS; + result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); + AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); + + UInt64 server_revision = 0; + if (table_to_check) + server_revision = result.entry->getServerRevision(*timeouts); + + if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) + { + result.entry->forceConnected(*timeouts); + result.is_usable = true; + result.is_up_to_date = true; + stage = Stage::FINISHED; + return; + } + + /// Only status of the remote table corresponding to the Distributed table is taken into account. + /// TODO: request status for joined tables also. + TablesStatusRequest status_request; + status_request.tables.emplace(*table_to_check); + + TablesStatusResponse status_response = result.entry->getTablesStatus(*timeouts, status_request); + auto table_status_it = status_response.table_states_by_id.find(*table_to_check); + if (table_status_it == status_response.table_states_by_id.end()) + { + const char * message_pattern = "There is no table {}.{} on server: {}"; + fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); + LOG_WARNING(log, fail_message); + ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); + + stage = Stage::FINISHED; + return; + } + + result.is_usable = true; + + UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; + if (!max_allowed_delay) + { + result.is_up_to_date = true; + stage = Stage::FINISHED; + return; + } + + UInt32 delay = table_status_it->second.absolute_delay; + + if (delay < max_allowed_delay) + result.is_up_to_date = true; + else + { + result.is_up_to_date = false; + result.staleness = delay; + + LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); + ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); + } + stage = Stage::FINISHED; + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT + && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throw; + + fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); + resetResult(); + stage = Stage::FAILED; + } +} + +} diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h new file mode 100644 index 00000000000..8d10126b3da --- /dev/null +++ b/src/Client/ConnectionEstablisher.h @@ -0,0 +1,116 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +/// Class for nonblocking establishing connection to the replica. +/// It runs establishing connection process in fiber and sets special +/// read callback which is called when reading from socket blocks. +/// When read callback is called, socket and receive timeout are added in epoll +/// and execution returns to the main program. +/// So, you can poll this epoll file descriptor to determine when to resume. +class ConnectionEstablisher +{ +public: + using TryResult = PoolWithFailoverBase::TryResult; + + ConnectionEstablisher(IConnectionPool * pool_, + const ConnectionTimeouts * timeouts_, + const Settings * settings_, + const QualifiedTableName * table_to_check = nullptr); + + /// Establish connection with replica, call async_callbeck when + /// reading from socket blocks. + void establishConnection(AsyncCallback async_callback = {}); + + /// In the first call create fiber with establishConnection function, + /// in the next - check timeout and resume fiber. + void resume(); + + /// Cancel establishing connections. Fiber will be destroyed, + /// class will be set in initial stage. + void cancel(); + + bool isInProcess() const { return stage == Stage::IN_PROCESS; } + + bool isFinished() const { return stage == Stage::FINISHED; } + + bool isFailed() const { return stage == Stage::FAILED; } + + int getFileDescriptor() const + { + int fd = -1; +#if defined(OS_LINUX) + fd = epoll.getFileDescriptor(); +#endif + return fd; + } + + const std::string & getFailMessage() const { return fail_message; } + + TryResult getResult() { return result; } + + Connection * getConnection() { return &*result.entry; } + + +private: + void processReceiveTimeout(); + + enum class Stage + { + INITIAL, + IN_PROCESS, + FINISHED, + FAILED, + }; + + struct Routine + { + ConnectionEstablisher & connection_establisher; + + struct ReadCallback + { + ConnectionEstablisher & connection_establisher; + Fiber & fiber; + + void operator()(int fd, const Poco::Timespan & timeout, const std::string &); + }; + + Fiber operator()(Fiber && sink); + }; + + void resetResult(); + + void reset(); + + void destroyFiber(); + + void resumeFiber(); + + IConnectionPool * pool; + const ConnectionTimeouts * timeouts; + std::string fail_message; + const Settings * settings; + const QualifiedTableName * table_to_check; + TryResult result; + Stage stage; + Poco::Logger * log; + Fiber fiber; + FiberStack fiber_stack; + std::exception_ptr exception; + int socket_fd = -1; + bool fiber_created = false; +#if defined(OS_LINUX) + TimerDescriptor receive_timeout; + Epoll epoll; +#endif +}; + +} diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index a027f7a186b..ec9215e3bc1 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -249,77 +250,10 @@ ConnectionPoolWithFailover::tryGetEntry( const Settings * settings, const QualifiedTableName * table_to_check) { - TryResult result; - try - { - result.entry = pool.get(timeouts, settings, /* force_connected = */ false); - - UInt64 server_revision = 0; - if (table_to_check) - server_revision = result.entry->getServerRevision(timeouts); - - if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) - { - result.entry->forceConnected(timeouts); - result.is_usable = true; - result.is_up_to_date = true; - return result; - } - - /// Only status of the remote table corresponding to the Distributed table is taken into account. - /// TODO: request status for joined tables also. - TablesStatusRequest status_request; - status_request.tables.emplace(*table_to_check); - - TablesStatusResponse status_response = result.entry->getTablesStatus(timeouts, status_request); - auto table_status_it = status_response.table_states_by_id.find(*table_to_check); - if (table_status_it == status_response.table_states_by_id.end()) - { - const char * message_pattern = "There is no table {}.{} on server: {}"; - fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); - LOG_WARNING(log, fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); - - return result; - } - - result.is_usable = true; - - UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; - if (!max_allowed_delay) - { - result.is_up_to_date = true; - return result; - } - - UInt32 delay = table_status_it->second.absolute_delay; - - if (delay < max_allowed_delay) - result.is_up_to_date = true; - else - { - result.is_up_to_date = false; - result.staleness = delay; - - LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); - ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - } - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT - && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - throw; - - fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); - - if (!result.entry.isNull()) - { - result.entry->disconnect(); - result.reset(); - } - } - return result; + ConnectionEstablisher connection_establisher(&pool, &timeouts, settings, table_to_check); + connection_establisher.establishConnection(); + fail_message = connection_establisher.getFailMessage(); + return connection_establisher.getResult(); } std::vector ConnectionPoolWithFailover::getShuffledPools(const Settings * settings) @@ -329,141 +263,4 @@ std::vector ConnectionPoolWithFa return Base::getShuffledPools(max_ignored_errors, get_priority); } -ConnectionEstablisher::ConnectionEstablisher( - IConnectionPool * pool_, - const ConnectionTimeouts * timeouts_, - const Settings * settings_, - const QualifiedTableName * table_to_check_, - Poco::Logger * log_) : - pool(pool_), timeouts(timeouts_), settings(settings_), - table_to_check(table_to_check_), log(log_), stage(Stage::CONNECT), socket_fd(-1) -{ -} - -void ConnectionEstablisher::reset() -{ - resetResult(); - stage = Stage::CONNECT; - action_before_disconnect = nullptr; - socket_fd = -1; - fail_message.clear(); -} - -void ConnectionEstablisher::resetResult() -{ - if (!result.entry.isNull()) - { - result.entry->disconnect(); - result.reset(); - } -} - -void ConnectionEstablisher::run() -{ - try - { - if (stage == Stage::CONNECT) - { - result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); - - if (!result.entry->isConnected()) - { - result.entry->prepare(*timeouts); - socket_fd = result.entry->getSocket()->impl()->sockfd(); - result.entry->sendHello(); - stage = Stage::RECEIVE_HELLO; - /// We are waiting for hello from replica. - return; - } - - socket_fd = result.entry->getSocket()->impl()->sockfd(); - stage = Stage::START_CHECK_TABLE; - } - - if (stage == Stage::RECEIVE_HELLO) - { - result.entry->receiveHello(); - stage = Stage::START_CHECK_TABLE; - } - - if (stage == Stage::START_CHECK_TABLE) - { - UInt64 server_revision = 0; - if (table_to_check) - server_revision = result.entry->getServerRevision(*timeouts); - - if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) - { - result.entry->forceConnected(*timeouts); - result.is_usable = true; - result.is_up_to_date = true; - stage = FINISHED; - return; - } - - TablesStatusRequest status_request; - status_request.tables.emplace(*table_to_check); - - result.entry->sendTablesStatusRequest(status_request); - stage = Stage::RECEIVE_TABLES_STATUS; - /// We are waiting for tables status response. - return; - } - - if (stage == Stage::RECEIVE_TABLES_STATUS) - { - TablesStatusResponse status_response = result.entry->receiveTablesStatusResponse(); - auto table_status_it = status_response.table_states_by_id.find(*table_to_check); - if (table_status_it == status_response.table_states_by_id.end()) - { - const char * message_pattern = "There is no table {}.{} on server: {}"; - fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); - LOG_WARNING(log, fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); - stage = Stage::FINISHED; - return; - } - - result.is_usable = true; - - UInt64 max_allowed_delay = settings ? UInt64(settings->max_replica_delay_for_distributed_queries) : 0; - if (!max_allowed_delay) - { - result.is_up_to_date = true; - stage = Stage::FINISHED; - return; - } - - UInt32 delay = table_status_it->second.absolute_delay; - - if (delay < max_allowed_delay) - result.is_up_to_date = true; - else - { - result.is_up_to_date = false; - result.staleness = delay; - - LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); - ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); - } - } - - stage = Stage::FINISHED; - } - catch (const Exception & e) - { - if (e.code() != ErrorCodes::NETWORK_ERROR && e.code() != ErrorCodes::SOCKET_TIMEOUT - && e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) - throw; - - if (action_before_disconnect) - action_before_disconnect(socket_fd); - - fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); - resetResult(); - socket_fd = -1; - stage = Stage::FAILED; - } -} - } diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index b25eee6e33d..2ecb0492747 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -31,51 +31,6 @@ enum class PoolMode GET_ALL }; -/// Class for establishing connection with replica without blocking using different stages. -class ConnectionEstablisher -{ -public: - enum Stage - { - CONNECT = 0, - RECEIVE_HELLO = 1, - START_CHECK_TABLE = 2, - RECEIVE_TABLES_STATUS = 3, - FINISHED = 4, - FAILED = 5, - }; - - using TryResult = PoolWithFailoverBase::TryResult; - - ConnectionEstablisher(IConnectionPool * pool_, - const ConnectionTimeouts * timeouts_, - const Settings * settings_, - const QualifiedTableName * table_to_check = nullptr, - Poco::Logger * log_ = nullptr); - - /// Continue connecting to replica from previous stage. Initial stage is CONNECT. - void run(); - - void resetResult(); - - /// Reset class to initial stage. - void reset(); - - /// If action_before_disconnect is set, action_before_disconnect(socket_fd) will be called before - /// disconnect. It may be useful for removing file descriptor from epoll. - void setActionBeforeDisconnect(std::function action) { action_before_disconnect = action; } - - IConnectionPool * pool; - const ConnectionTimeouts * timeouts; - std::string fail_message; - const Settings * settings; - const QualifiedTableName * table_to_check; - Poco::Logger * log; - TryResult result; - Stage stage; - int socket_fd; - std::function action_before_disconnect; -}; class ConnectionPoolWithFailover : public IConnectionPool, private PoolWithFailoverBase { diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 6d49c0f6749..61d6d317c6e 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -29,24 +29,26 @@ HedgedConnections::HedgedConnections( if (connections.empty()) return; + offset_states.reserve(connections.size()); for (size_t i = 0; i != connections.size(); ++i) { - ReplicaState replica; - replica.connection = connections[i]; - replica.connection->setThrottler(throttler_); - replica.epoll.add(replica.connection->getSocket()->impl()->sockfd()); - epoll.add(replica.epoll.getFileDescriptor()); - fd_to_replica_location[replica.epoll.getFileDescriptor()] = ReplicaLocation{i, 0}; offset_states.emplace_back(); - offset_states[i].replicas.emplace_back(std::move(replica)); + offset_states[i].replicas.emplace_back(connections[i]); offset_states[i].active_connection_count = 1; + + ReplicaState & replica = offset_states[i].replicas.back(); + replica.connection->setThrottler(throttler_); + + epoll.add(replica.packet_receiver.getFileDescriptor()); + fd_to_replica_location[replica.packet_receiver.getFileDescriptor()] = ReplicaLocation{i, 0}; + + epoll.add(replica.change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{i, 0}; } active_connection_count = connections.size(); offsets_with_received_first_data_packet = 0; pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); }); - - log = &Poco::Logger::get("HedgedConnections"); } void HedgedConnections::Pipeline::add(std::function send_function) @@ -155,11 +157,10 @@ void HedgedConnections::sendQuery( if (offset_states.size() > 1) { modified_settings.parallel_replicas_count = offset_states.size(); - modified_settings.parallel_replica_offset = fd_to_replica_location[replica.epoll.getFileDescriptor()].offset; + modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver.getFileDescriptor()].offset; } replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); - replica.receive_timeout.setRelative(timeouts.receive_timeout); replica.change_replica_timeout.setRelative(timeouts.receive_data_timeout); }; @@ -281,75 +282,60 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) if (epoll.empty()) throw Exception("No pending events in epoll.", ErrorCodes::LOGICAL_ERROR); - ReplicaLocation location = getReadyReplicaLocation(async_callback); - return receivePacketFromReplica(location, std::move(async_callback)); + ReplicaLocation location = getReadyReplicaLocation(std::move(async_callback)); + return receivePacketFromReplica(location); } HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback) { - LOG_DEBUG(log, "getReadyReplicaLocation"); int event_fd; while (true) { - /// Check connections for pending data. + /// Check connections for pending data in buffer. ReplicaLocation location; if (checkPendingData(location)) - return location; + { + ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; + + replica_state.packet_receiver.resume(); + if (replica_state.packet_receiver.isPacketReady()) + return location; + continue; + } /// Get ready file descriptor from epoll and process it. event_fd = getReadyFileDescriptor(async_callback); if (event_fd == hedged_connections_factory.getFileDescriptor()) - { tryGetNewReplica(false); - continue; - } - - if (!fd_to_replica_location.contains(event_fd)) - throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); - - location = fd_to_replica_location[event_fd]; - - /// Read all events from replica epoll. - /// If socket is ready and timeout is alarmed simultaneously, skip timeout. - bool is_socket_ready = false; - bool is_change_replica_timeout_alarmed = false; - bool is_receive_timeout_alarmed = false; - - epoll_event events[3]; - events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; - size_t ready_count = replica_state.epoll.getManyReady(3, events, true); - - for (size_t i = 0; i != ready_count; ++i) + else if (fd_to_replica_location.contains(event_fd)) { - if (events[i].data.fd == replica_state.connection->getSocket()->impl()->sockfd()) - is_socket_ready = true; - if (events[i].data.fd == replica_state.change_replica_timeout.getDescriptor()) - is_change_replica_timeout_alarmed = true; - if (events[i].data.fd == replica_state.receive_timeout.getDescriptor()) - is_receive_timeout_alarmed = true; + location = fd_to_replica_location[event_fd]; + ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; + replica_state.packet_receiver.resume(); + + if (replica_state.packet_receiver.isPacketReady()) + return location; + + if (replica_state.packet_receiver.isReceiveTimeoutExpired()) + { + finishProcessReplica(replica_state, true); + + /// Check if there is no more active connections with the same offset and there is no new replica in process. + if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process) + throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); + } } - - if (is_socket_ready) - return location; - - /// We reach this point only if there is an alarmed timeout. - - if (is_change_replica_timeout_alarmed) + else if (timeout_fd_to_replica_location.contains(event_fd)) { - replica_state.change_replica_timeout.reset(); + location = timeout_fd_to_replica_location[event_fd]; + offset_states[location.offset].replicas[location.index].change_replica_timeout.reset(); + offset_states[location.offset].next_replica_in_process = true; offsets_queue.push(location.offset); tryGetNewReplica(true); } - if (is_receive_timeout_alarmed) - { - finishProcessReplica(replica_state, true); - - /// Check if there is no more active connections with the same offset and there is no new replica in process. - if (offset_states[location.offset].active_connection_count == 0 && !next_replica_in_process) - throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); - } + else + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); } }; @@ -375,19 +361,15 @@ bool HedgedConnections::checkPendingData(ReplicaLocation & location_out) return false; } -Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & replica_location, AsyncCallback async_callback) +Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & replica_location) { - LOG_DEBUG(log, "receivePacketFromReplica"); - ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; - replica.receive_timeout.reset(); - Packet packet = replica.connection->receivePacket(std::move(async_callback)); + Packet packet = replica.packet_receiver.getPacket(); switch (packet.type) { case Protocol::Server::Data: if (!offset_states[replica_location.offset].first_packet_of_data_received) processReceivedFirstDataPacket(replica_location); - replica.receive_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_timeout); break; case Protocol::Server::PartUUIDs: case Protocol::Server::Progress: @@ -395,7 +377,6 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: - replica.receive_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_timeout); break; case Protocol::Server::EndOfStream: @@ -413,8 +394,6 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & replica_location) { - LOG_DEBUG(log, "processReceivedFirstDataPacket"); - /// When we receive first packet of data from replica, we stop working with replicas, that are /// responsible for the same offset. OffsetState & offset_state = offset_states[replica_location.offset]; @@ -445,8 +424,6 @@ void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & r void HedgedConnections::tryGetNewReplica(bool start_new_connection) { - LOG_DEBUG(log, "tryGetNewReplica"); - Connection * connection = nullptr; HedgedConnectionsFactory::State state = hedged_connections_factory.getNextConnection(start_new_connection, false, connection); @@ -461,14 +438,18 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) size_t offset = offsets_queue.front(); offsets_queue.pop(); - ReplicaState replica; - replica.connection = connection; - replica.epoll.add(replica.connection->getSocket()->impl()->sockfd()); - epoll.add(replica.epoll.getFileDescriptor()); - fd_to_replica_location[replica.epoll.getFileDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size()}; + offset_states[offset].replicas.emplace_back(connection); + ++offset_states[offset].active_connection_count; + offset_states[offset].next_replica_in_process = false; ++active_connection_count; + + ReplicaState & replica = offset_states[offset].replicas.back(); + epoll.add(replica.packet_receiver.getFileDescriptor()); + fd_to_replica_location[replica.packet_receiver.getFileDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1}; + epoll.add(replica.change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1}; + pipeline_for_new_replicas.run(replica); - offset_states[offset].replicas.push_back(std::move(replica)); } else if (state == HedgedConnectionsFactory::State::NOT_READY && !next_replica_in_process) { @@ -483,6 +464,7 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) { if (offset_states[offsets_queue.front()].active_connection_count == 0) throw Exception("Cannot find enough connections to replicas", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + offset_states[offsets_queue.front()].next_replica_in_process = false; offsets_queue.pop(); } } @@ -497,11 +479,16 @@ void HedgedConnections::tryGetNewReplica(bool start_new_connection) void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { - LOG_DEBUG(log, "finishProcessReplica"); + replica.packet_receiver.cancel(); + replica.change_replica_timeout.reset(); + + epoll.remove(replica.packet_receiver.getFileDescriptor()); + --offset_states[fd_to_replica_location[replica.packet_receiver.getFileDescriptor()].offset].active_connection_count; + fd_to_replica_location.erase(replica.packet_receiver.getFileDescriptor()); + + epoll.remove(replica.change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_location.erase(replica.change_replica_timeout.getDescriptor()); - epoll.remove(replica.epoll.getFileDescriptor()); - --offset_states[fd_to_replica_location[replica.epoll.getFileDescriptor()].offset].active_connection_count; - fd_to_replica_location.erase(replica.epoll.getFileDescriptor()); --active_connection_count; if (disconnect) diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 41c548de9ef..bfd5a36c500 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -5,6 +5,9 @@ #include #include #include +#include +#include +#include namespace DB { @@ -20,18 +23,13 @@ class HedgedConnections : public IConnections public: struct ReplicaState { - ReplicaState() + ReplicaState(Connection * connection_) : connection(connection_), packet_receiver(connection_) { - epoll.add(receive_timeout.getDescriptor()); - epoll.add(change_replica_timeout.getDescriptor()); } Connection * connection = nullptr; - TimerDescriptor receive_timeout; + PacketReceiver packet_receiver; TimerDescriptor change_replica_timeout; - /// We store socket and timeout descriptors in epoll - /// and use it's fd outside. - Epoll epoll; }; struct OffsetState @@ -43,6 +41,12 @@ public: /// other replicas when we receive first data packet from one of them) size_t active_connection_count = 0; bool first_packet_of_data_received = false; + + /// This flag is true when this offset is in queue for + /// new replicas. It's needed to process receive timeout + /// (throw an exception when receive timeout expired and there is no + /// new replica in process) + bool next_replica_in_process = false; }; /// We process events in epoll, so we need to determine replica by it's @@ -109,7 +113,7 @@ private: std::vector> pipeline; }; - Packet receivePacketFromReplica(const ReplicaLocation & replica_location, AsyncCallback async_callback = {}); + Packet receivePacketFromReplica(const ReplicaLocation & replica_location); ReplicaLocation getReadyReplicaLocation(AsyncCallback async_callback = {}); @@ -133,6 +137,9 @@ private: /// Map socket file descriptor to replica location (it's offset and index in OffsetState.replicas). std::unordered_map fd_to_replica_location; + /// Map receive data timeout file descriptor to replica location. + std::unordered_map timeout_fd_to_replica_location; + /// A queue of offsets for new replicas. When we get RECEIVE_DATA_TIMEOUT from /// the replica, we push it's offset to this queue and start trying to get /// new replica. @@ -163,8 +170,6 @@ private: bool cancelled = false; mutable std::mutex cancel_mutex; - - Poco::Logger * log; }; } diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index ba0e4ac7b22..3551814d603 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -22,10 +22,7 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( { shuffled_pools = pool->getShuffledPools(settings); for (size_t i = 0; i != shuffled_pools.size(); ++i) - { - ConnectionEstablisher establisher(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get(), log); - replicas.emplace_back(std::move(establisher)); - } + replicas.emplace_back(ConnectionEstablisher(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get())); max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); @@ -43,7 +40,6 @@ HedgedConnectionsFactory::~HedgedConnectionsFactory() std::vector HedgedConnectionsFactory::getManyConnections(PoolMode pool_mode) { - LOG_DEBUG(log, "getManyConnections"); size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; size_t max_entries; @@ -103,8 +99,6 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out) { - LOG_DEBUG(log, "getNextConnection"); - if (start_new_connection) { int index = startEstablishingNewConnection(connection_out); @@ -120,7 +114,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool /// We will try to use usable replica. /// Check if we are not allowed to use usable replicas or there is no even a free usable replica. - if (!fallback_to_stale_replicas || !canGetNewConnection()) + if (!fallback_to_stale_replicas) return State::CANNOT_CHOOSE; return setBestUsableReplica(connection_out); @@ -128,21 +122,24 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool void HedgedConnectionsFactory::stopChoosingReplicas() { - LOG_DEBUG(log, "stopChoosingReplicas"); - for (auto & [fd, replica_index] : fd_to_replica_index) + for (auto & [fd, index] : fd_to_replica_index) { - resetReplicaTimeouts(replica_index); epoll.remove(fd); - replicas[replica_index].connection_establisher.reset(); + replicas[index].connection_establisher.cancel(); + } + + for (auto & [fd, index] : timeout_fd_to_replica_index) + { + replicas[index].change_replica_timeout.reset(); + epoll.remove(fd); } fd_to_replica_index.clear(); + timeout_fd_to_replica_index.clear(); } int HedgedConnectionsFactory::getNextIndex() { - LOG_DEBUG(log, "getNextIndex"); - /// Check if there is no free replica. if (entries_count + replicas_in_process_count + failed_pools_count >= shuffled_pools.size()) return -1; @@ -161,8 +158,9 @@ int HedgedConnectionsFactory::getNextIndex() next_index = (next_index + 1) % shuffled_pools.size(); /// Check if we can try this replica. - if (!replicas[next_index].is_in_process && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries) - && replicas[next_index].connection_establisher.stage != ConnectionEstablisher::Stage::FINISHED) + if (!replicas[next_index].connection_establisher.isInProcess() + && !replicas[next_index].connection_establisher.isFinished() + && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries)) finish = true; /// If we made a complete round, there is no replica to connect. @@ -176,100 +174,80 @@ int HedgedConnectionsFactory::getNextIndex() int HedgedConnectionsFactory::startEstablishingNewConnection(Connection *& connection_out) { - LOG_DEBUG(log, "startEstablishingNewConnection"); - int index; do { - LOG_DEBUG(log, "startEstablishingNewConnection loop"); - index = getNextIndex(); if (index == -1) return -1; ReplicaStatus & replica = replicas[index]; - ++replicas_in_process_count; - replica.is_in_process = true; - replica.connection_establisher.reset(); - replica.connection_establisher.run(); + replica.connection_establisher.resume(); processConnectionEstablisherStage(index); - if (replica.is_in_process) + if (replica.connection_establisher.isInProcess()) { - replica.epoll.add(replica.connection_establisher.socket_fd); - replica.connection_establisher.setActionBeforeDisconnect([&](int fd){ replica.epoll.remove(fd); }); - addTimeouts(index); - epoll.add(replica.epoll.getFileDescriptor()); - fd_to_replica_index[replica.epoll.getFileDescriptor()] = index; + epoll.add(replica.connection_establisher.getFileDescriptor()); + fd_to_replica_index[replica.connection_establisher.getFileDescriptor()] = index; + + /// Add timeout for changing replica. + replica.change_replica_timeout.setRelative(timeouts.hedged_connection_timeout); + epoll.add(replica.change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_index[replica.change_replica_timeout.getDescriptor()] = index; } } - while (!replicas[index].is_ready && !replicas[index].is_in_process); + while (!replicas[index].connection_establisher.isInProcess() && !replicas[index].is_ready); if (replicas[index].is_ready) - connection_out = &*replicas[index].connection_establisher.result.entry; + connection_out = replicas[index].connection_establisher.getConnection(); return index; } -void HedgedConnectionsFactory::processConnectionEstablisherStage(int replica_index, bool remove_from_epoll) +void HedgedConnectionsFactory::processConnectionEstablisherStage(int index, bool remove_from_epoll) { - LOG_DEBUG(log, "processConnectionEstablisherStage"); + ReplicaStatus & replica = replicas[index]; - ReplicaStatus & replica = replicas[replica_index]; - - if (replica.connection_establisher.stage == ConnectionEstablisher::Stage::FINISHED) + if (replica.connection_establisher.isFinished()) { - replica.is_in_process = false; --replicas_in_process_count; ++entries_count; if (remove_from_epoll) - { - epoll.remove(replica.epoll.getFileDescriptor()); - fd_to_replica_index.erase(replica.epoll.getFileDescriptor()); - } + removeReplicaFromEpoll(index); - if (replica.connection_establisher.result.is_usable) + if (replica.connection_establisher.getResult().is_usable) { ++usable_count; - if (replica.connection_establisher.result.is_up_to_date) - { - LOG_DEBUG(log, "READY"); - ++ready_replicas_count; + if (replica.connection_establisher.getResult().is_up_to_date) replica.is_ready = true; - return; - } - } - else - { - std::string & fail_message = replica.connection_establisher.fail_message; - if (!fail_message.empty()) - fail_messages += fail_message + "\n"; + + return; } + + /// If replica is not usable, we need to save fail message. + if (!replica.connection_establisher.getFailMessage().empty()) + fail_messages += replica.connection_establisher.getFailMessage() + "\n"; } - else if (replica.connection_establisher.stage == ConnectionEstablisher::Stage::FAILED) - processFailedConnection(replica_index, remove_from_epoll); + else if (replica.connection_establisher.isFailed()) + processFailedConnection(index, remove_from_epoll); } -void HedgedConnectionsFactory::processFailedConnection(int replica_index, bool remove_from_epoll) +void HedgedConnectionsFactory::processFailedConnection(int index, bool remove_from_epoll) { - LOG_DEBUG(log, "processFailedConnection"); - + ConnectionEstablisher & connection_establisher = replicas[index].connection_establisher; + if (remove_from_epoll) - { - epoll.remove(replicas[replica_index].epoll.getFileDescriptor()); - fd_to_replica_index.erase(replicas[replica_index].epoll.getFileDescriptor()); - } + removeReplicaFromEpoll(index); - std::string & fail_message = replicas[replica_index].connection_establisher.fail_message; - if (!fail_message.empty()) - fail_messages += fail_message + "\n"; + if (!connection_establisher.getFailMessage().empty()) + fail_messages += connection_establisher.getFailMessage() + "\n"; - ShuffledPool & shuffled_pool = shuffled_pools[replica_index]; + ShuffledPool & shuffled_pool = shuffled_pools[index]; LOG_WARNING( - log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message); + log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), connection_establisher.getFailMessage()); ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); @@ -281,109 +259,39 @@ void HedgedConnectionsFactory::processFailedConnection(int replica_index, bool r } --replicas_in_process_count; - replicas[replica_index].is_in_process = false; -} - -void HedgedConnectionsFactory::addTimeouts(int replica_index) -{ - LOG_DEBUG(log, "addTimeouts"); - - auto stage = replicas[replica_index].connection_establisher.stage; - if (stage == ConnectionEstablisher::Stage::RECEIVE_HELLO) - { - replicas[replica_index].receive_timeout.setRelative(timeouts.receive_timeout); - replicas[replica_index].change_replica_timeout.setRelative(timeouts.receive_hello_timeout); - } - else if (stage == ConnectionEstablisher::Stage::RECEIVE_TABLES_STATUS) - { - replicas[replica_index].receive_timeout.setRelative(Poco::Timespan(DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC, 0)); - replicas[replica_index].change_replica_timeout.setRelative(timeouts.receive_tables_status_timeout); - } -} - -void HedgedConnectionsFactory::resetReplicaTimeouts(int replica_index) -{ - LOG_DEBUG(log, "resetReplicaTimeouts"); - - replicas[replica_index].receive_timeout.reset(); - replicas[replica_index].change_replica_timeout.reset(); } HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out) { - LOG_DEBUG(log, "processEpollEvents"); - int event_fd; while (!epoll.empty()) { - /// Firstly, check connections for pending data. - int replica_index = checkPendingData(); - if (replica_index != -1) - { - processSocketEvent(replica_index, connection_out); - /// Return only if replica is ready. - if (replicas[replica_index].is_ready) - return State::READY; - - continue; - } - - /// Get ready descriptor from epoll. event_fd = getReadyFileDescriptor(blocking); - /// Check if there is no events. if (event_fd == -1) return State::NOT_READY; - if (!fd_to_replica_index.contains(event_fd)) - throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); - - replica_index = fd_to_replica_index[event_fd]; - - /// Read all events from replica epoll. - /// If socket is ready and timeout is alarmed simultaneously, skip timeout. - bool is_socket_ready = false; - bool is_receive_timeout_alarmed = false; - bool is_change_replica_timeout_alarmed = false; - - epoll_event events[3]; - events[0].data.fd = events[1].data.fd = events[2].data.fd = -1; - size_t ready_count = replicas[replica_index].epoll.getManyReady(3, events, true); - for (size_t i = 0; i != ready_count; ++i) + if (fd_to_replica_index.contains(event_fd)) { - if (events[i].data.fd == replicas[replica_index].connection_establisher.socket_fd) - is_socket_ready = true; - if (events[i].data.fd == replicas[replica_index].receive_timeout.getDescriptor()) - is_receive_timeout_alarmed = true; - if (events[i].data.fd == replicas[replica_index].change_replica_timeout.getDescriptor()) - is_change_replica_timeout_alarmed = true; - } + int index = fd_to_replica_index[event_fd]; + processConnectionEstablisherEvent(index, connection_out); - if (is_socket_ready) - { - processSocketEvent(replica_index, connection_out); - /// Return only if replica is ready. - if (replicas[replica_index].is_ready) + if (replicas[index].is_ready) return State::READY; - if (replicas[replica_index].is_in_process) + if (replicas[index].connection_establisher.isInProcess()) continue; } + else if (timeout_fd_to_replica_index.contains(event_fd)) + replicas[timeout_fd_to_replica_index[event_fd]].change_replica_timeout.reset(); else - { - if (is_receive_timeout_alarmed) - processReceiveTimeout(replica_index); + throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); - if (is_change_replica_timeout_alarmed) - { - LOG_DEBUG(log, "change_replica_timeout"); - replicas[replica_index].change_replica_timeout.reset(); - } - } + /// We reach this point only if we need to start new connection + /// (Special timeout expired or one of the previous connections failed). + int index = startEstablishingNewConnection(connection_out); - /// We reach this point only if we need to start new connection. - replica_index = startEstablishingNewConnection(connection_out); /// Return only if replica is ready. - if (replica_index != -1 && replicas[replica_index].is_ready) + if (index != -1 && replicas[index].is_ready) return State::READY; } @@ -398,40 +306,23 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) return event.data.fd; } -int HedgedConnectionsFactory::checkPendingData() +void HedgedConnectionsFactory::removeReplicaFromEpoll(int index) { - for (auto & [fd, replica_index] : fd_to_replica_index) - if (replicas[replica_index].connection_establisher.result.entry->hasReadPendingData()) - return replica_index; + ReplicaStatus & replica = replicas[index]; + epoll.remove(replica.connection_establisher.getFileDescriptor()); + fd_to_replica_index.erase(replica.connection_establisher.getFileDescriptor()); - return -1; + replica.change_replica_timeout.reset(); + epoll.remove(replica.change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_index.erase(replica.change_replica_timeout.getDescriptor()); } -void HedgedConnectionsFactory::processSocketEvent(int replica_index, Connection *& connection_out) +void HedgedConnectionsFactory::processConnectionEstablisherEvent(int index, Connection *& connection_out) { - LOG_DEBUG(log, "processSocketEvent"); - - resetReplicaTimeouts(replica_index); - replicas[replica_index].connection_establisher.run(); - processConnectionEstablisherStage(replica_index, true); - if (replicas[replica_index].is_in_process) - addTimeouts(replica_index); - if (replicas[replica_index].is_ready) - connection_out = &*replicas[replica_index].connection_establisher.result.entry; -} - -void HedgedConnectionsFactory::processReceiveTimeout(int replica_index) -{ - LOG_DEBUG(log, "processReceiveTimeout"); - - resetReplicaTimeouts(replica_index); - ReplicaStatus & replica = replicas[replica_index]; - - replica.connection_establisher.fail_message = - "Code: 209, e.displayText() = DB::NetException: Timeout exceeded while reading from socket (" + replica.connection_establisher.result.entry->getDescription() + ")"; - replica.connection_establisher.resetResult(); - replica.connection_establisher.stage = ConnectionEstablisher::Stage::FAILED; - processFailedConnection(replica_index, true); + replicas[index].connection_establisher.resume(); + processConnectionEstablisherStage(index, true); + if (replicas[index].is_ready) + connection_out = replicas[index].connection_establisher.getConnection(); } HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(Connection *& connection_out) @@ -440,8 +331,10 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C for (size_t i = 0; i != replicas.size(); ++i) { /// Don't add unusable, failed replicas and replicas that are ready or in process. - if (!replicas[i].connection_establisher.result.entry.isNull() && replicas[i].connection_establisher.result.is_usable && - !replicas[i].is_in_process && !replicas[i].is_ready) + if (!replicas[i].connection_establisher.getResult().entry.isNull() + && replicas[i].connection_establisher.getResult().is_usable + && !replicas[i].connection_establisher.isInProcess() + && !replicas[i].is_ready) indexes.push_back(i); } @@ -454,12 +347,12 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C indexes.end(), [&](size_t lhs, size_t rhs) { - return replicas[lhs].connection_establisher.result.staleness < replicas[rhs].connection_establisher.result.staleness; + return replicas[lhs].connection_establisher.getResult().staleness < replicas[rhs].connection_establisher.getResult().staleness; }); ++ready_replicas_count; replicas[indexes[0]].is_ready = true; - connection_out = &*replicas[indexes[0]].connection_establisher.result.entry; + connection_out = replicas[indexes[0]].connection_establisher.getConnection(); return State::READY; } diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index b821a9c925e..896774daae3 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -3,17 +3,18 @@ #if defined(OS_LINUX) #include +#include +#include +#include +#include #include #include -#include #include #include namespace DB { -using TimerDescriptorPtr = std::shared_ptr; - /** Class for establishing hedged connections with replicas. * The process of establishing connection is divided on stages, on each stage if * replica doesn't respond for a long time, we start establishing connection with @@ -25,22 +26,6 @@ class HedgedConnectionsFactory public: using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool; - struct ReplicaStatus - { - ReplicaStatus(const ConnectionEstablisher & establisher) : connection_establisher(establisher) - { - epoll.add(receive_timeout.getDescriptor()); - epoll.add(change_replica_timeout.getDescriptor()); - } - - ConnectionEstablisher connection_establisher; - TimerDescriptor receive_timeout; - TimerDescriptor change_replica_timeout; - bool is_ready = false; - bool is_in_process = false; - Epoll epoll; - }; - enum class State { READY, @@ -48,6 +33,17 @@ public: CANNOT_CHOOSE, }; + struct ReplicaStatus + { + ReplicaStatus(ConnectionEstablisher connection_stablisher_) : connection_establisher(std::move(connection_stablisher_)) + { + } + + ConnectionEstablisher connection_establisher; + TimerDescriptor change_replica_timeout; + bool is_ready = false; + }; + HedgedConnectionsFactory(const ConnectionPoolWithFailoverPtr & pool_, const Settings * settings_, const ConnectionTimeouts & timeouts_, @@ -64,12 +60,12 @@ public: State getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out); /// Check if we can try to produce new READY replica. - bool canGetNewConnection() const { return ready_replicas_count + failed_pools_count < shuffled_pools.size(); } +// bool canGetNewConnection() const { return ready_replicas_count + failed_pools_count < shuffled_pools.size(); } /// Stop working with all replicas that are not READY. void stopChoosingReplicas(); - bool hasEventsInProcess() const { return epoll.size() > 0; } + bool hasEventsInProcess() const { return !epoll.empty(); } int getFileDescriptor() const { return epoll.getFileDescriptor(); } @@ -90,17 +86,11 @@ private: int getReadyFileDescriptor(bool blocking); - int checkPendingData(); - - void addTimeouts(int replica_index); - - void resetReplicaTimeouts(int replica_index); - void processFailedConnection(int replica_index, bool remove_from_epoll); - void processSocketEvent(int replica_index, Connection *& connection_out); + void processConnectionEstablisherEvent(int replica_index, Connection *& connection_out); - void processReceiveTimeout(int replica_index); + void removeReplicaFromEpoll(int index); /// Return NOT_READY state if there is no ready events, READY if replica is ready /// and CANNOT_CHOOSE if there is no more events in epoll. @@ -111,19 +101,22 @@ private: const ConnectionPoolWithFailoverPtr pool; const Settings * settings; const ConnectionTimeouts timeouts; - std::shared_ptr table_to_check; - std::vector replicas; std::vector shuffled_pools; + std::vector replicas; /// Map socket file descriptor to replica index. std::unordered_map fd_to_replica_index; + /// Map timeout for changing replica to replica index. + std::unordered_map timeout_fd_to_replica_index; + /// Indexes of replicas, that are in process of connection. size_t replicas_in_process_count = 0; /// Indexes of ready replicas. size_t ready_replicas_count = 0; + std::shared_ptr table_to_check; int last_used_index = -1; bool fallback_to_stale_replicas; Epoll epoll; diff --git a/src/Client/MultiplexedConnections.cpp b/src/Client/MultiplexedConnections.cpp index 577210792e9..8b2b7c49f26 100644 --- a/src/Client/MultiplexedConnections.cpp +++ b/src/Client/MultiplexedConnections.cpp @@ -265,7 +265,11 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac if (current_connection == nullptr) throw Exception("Logical error: no available replica", ErrorCodes::NO_AVAILABLE_REPLICA); - Packet packet = current_connection->receivePacket(std::move(async_callback)); + Packet packet; + { + AsyncCallbackSetter async_setter(current_connection, std::move(async_callback)); + packet = current_connection->receivePacket(); + } switch (packet.type) { diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h new file mode 100644 index 00000000000..4d42804d0a2 --- /dev/null +++ b/src/Client/PacketReceiver.h @@ -0,0 +1,143 @@ +#pragma once + +#if defined(OS_LINUX) + +#include +#include +#include + +namespace DB +{ + +/// Class for nonblocking packet receiving. It runs connection->receivePacket +/// in fiber and sets special read callback which is called when +/// reading from socket blocks. When read callback is called, +/// socket and receive timeout are added in epoll and execution returns to the main program. +/// So, you can poll this epoll file descriptor to determine when to resume +/// packet receiving (beside polling epoll descriptor, you also need to check connection->hasPendingData(), +/// because small packet can be read in buffer with the previous one, so new packet will be ready in buffer, +/// but there is no data socket to poll). +class PacketReceiver +{ +public: + PacketReceiver(Connection * connection_) : connection(connection_) + { + epoll.add(receive_timeout.getDescriptor()); + epoll.add(connection->getSocket()->impl()->sockfd()); + fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); + } + + /// Resume packet receiving. + void resume() + { + /// If there is no pending data, check receive timeout. + if (!connection->hasReadPendingData() && !checkReceiveTimeout()) + return; + + fiber = std::move(fiber).resume(); + if (exception) + std::rethrow_exception(std::move(exception)); + } + + void cancel() + { + Fiber to_destroy = std::move(fiber); + connection = nullptr; + } + + Packet getPacket() { return std::move(packet); } + + int getFileDescriptor() const { return epoll.getFileDescriptor(); } + + bool isPacketReady() const { return !is_read_in_process; } + + bool isReceiveTimeoutExpired() const { return is_receive_timeout_expired; } + +private: + /// When epoll file descriptor is ready, check if it's an expired timeout + bool checkReceiveTimeout() + { + bool is_socket_ready = false; + is_receive_timeout_expired = false; + + epoll_event events[2]; + events[0].data.fd = events[1].data.fd = -1; + size_t ready_count = epoll.getManyReady(2, events, true); + + for (size_t i = 0; i != ready_count; ++i) + { + if (events[i].data.fd == connection->getSocket()->impl()->sockfd()) + is_socket_ready = true; + if (events[i].data.fd == receive_timeout.getDescriptor()) + is_receive_timeout_expired = true; + } + + if (is_receive_timeout_expired && !is_socket_ready) + { + receive_timeout.reset(); + return false; + } + + return true; + } + + struct Routine + { + PacketReceiver & receiver; + + struct ReadCallback + { + PacketReceiver & receiver; + Fiber & sink; + + void operator()(int, const Poco::Timespan & timeout, const std::string &) + { + receiver.receive_timeout.setRelative(timeout); + receiver.is_read_in_process = true; + sink = std::move(sink).resume(); + receiver.is_read_in_process = false; + receiver.receive_timeout.reset(); + } + }; + + Fiber operator()(Fiber && sink) + { + try + { + AsyncCallbackSetter async_setter(receiver.connection, ReadCallback{receiver, sink}); + while (true) + { + receiver.packet = receiver.connection->receivePacket(); + sink = std::move(sink).resume(); + } + + } + catch (const boost::context::detail::forced_unwind &) + { + /// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited + /// It should not be caught or it will segfault. + /// Other exceptions must be caught + throw; + } + catch (...) + { + receiver.exception = std::current_exception(); + } + + return std::move(sink); + } + }; + + Connection * connection; + TimerDescriptor receive_timeout; + Epoll epoll; + Fiber fiber; + FiberStack fiber_stack; + Packet packet; + bool is_read_in_process = false; + bool is_receive_timeout_expired = false; + std::exception_ptr exception; +}; + +} +#endif diff --git a/src/Client/ya.make b/src/Client/ya.make index 7a664f328f7..af1dd05f1d4 100644 --- a/src/Client/ya.make +++ b/src/Client/ya.make @@ -11,6 +11,7 @@ PEERDIR( SRCS( Connection.cpp + ConnectionEstablisher.cpp ConnectionPoolWithFailover.cpp HedgedConnections.cpp HedgedConnectionsFactory.cpp diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index d085315b1a0..89c6c8abc49 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -21,20 +21,16 @@ Epoll::Epoll() : events_count(0) throwFromErrno("Cannot open epoll descriptor", DB::ErrorCodes::EPOLL_ERROR); } -Epoll::Epoll(Epoll && other) +Epoll::Epoll(Epoll && other) : epoll_fd(other.epoll_fd), events_count(other.events_count.load()) { - epoll_fd = other.epoll_fd; other.epoll_fd = -1; - int count = other.events_count; - events_count = count; } Epoll & Epoll::operator=(Epoll && other) { epoll_fd = other.epoll_fd; other.epoll_fd = -1; - int count = other.events_count; - events_count = count; + events_count.store(other.events_count.load()); return *this; } diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 1e1e86171a1..8fd8e0d6bdf 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -12,8 +12,7 @@ #define DBMS_DEFAULT_SEND_TIMEOUT_SEC 300 #define DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC 300 /// Timeouts for hedged requests. -#define DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_MS 100 -#define DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_MS 100 +#define DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS 100 #define DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC 2 /// Timeout for synchronous request-result protocol call (like Ping or TablesStatus). #define DBMS_DEFAULT_SYNC_REQUEST_TIMEOUT_SEC 5 diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d4f5602122c..98483fceced 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -55,8 +55,7 @@ class IColumn; M(Seconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "", 0) \ M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ - M(Milliseconds, receive_hello_timeout, DBMS_DEFAULT_RECEIVE_HELLO_TIMEOUT_MS, "Connection timeout for receiving hello from replica", 0) \ - M(Milliseconds, receive_tables_status_timeout, DBMS_DEFAULT_RECEIVE_TABLES_STATUS_TIMEOUT_MS, "Connection timeout for receiving tables status from replica", 0) \ + M(Milliseconds, hedged_connection_timeout, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \ M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data from replica", 0) \ M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \ M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ @@ -220,9 +219,8 @@ class IColumn; M(Milliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.", 0) \ \ /** Settings for testing hedged requests */ \ - M(Int64, sleep_before_send_hello, 0, "Time to sleep before sending hello in TCPHandler", 0) \ - M(Int64, sleep_before_send_tables_status, 0, "Time to sleep before sending tables status response in TCPHandler", 0) \ - M(Int64, sleep_before_send_data, 0, "Time to sleep before sending data in TCPHandler", 0) \ + M(Int64, sleep_in_send_tables_status, 0, "Time to sleep in sending tables status response in TCPHandler", 0) \ + M(Int64, sleep_in_send_data, 0, "Time to sleep in sending data in TCPHandler", 0) \ \ M(Bool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.", 0) \ M(Seconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.", 0) \ diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index a92f75bf980..5f12a4edc79 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -18,8 +18,7 @@ struct ConnectionTimeouts Poco::Timespan secure_connection_timeout; /// Timeouts for HedgedConnections - Poco::Timespan receive_hello_timeout; - Poco::Timespan receive_tables_status_timeout; + Poco::Timespan hedged_connection_timeout; Poco::Timespan receive_data_timeout; ConnectionTimeouts() = default; @@ -33,8 +32,7 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(0), http_keep_alive_timeout(0), secure_connection_timeout(connection_timeout), - receive_hello_timeout(receive_timeout_), - receive_tables_status_timeout(receive_timeout_), + hedged_connection_timeout(receive_timeout_), receive_data_timeout(receive_timeout_) { } @@ -49,8 +47,7 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(0), secure_connection_timeout(connection_timeout), - receive_hello_timeout(receive_timeout_), - receive_tables_status_timeout(receive_timeout_), + hedged_connection_timeout(receive_timeout_), receive_data_timeout(receive_timeout_) { } @@ -65,8 +62,7 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(http_keep_alive_timeout_), secure_connection_timeout(connection_timeout), - receive_hello_timeout(receive_timeout_), - receive_tables_status_timeout(receive_timeout_), + hedged_connection_timeout(receive_timeout_), receive_data_timeout(receive_timeout_) { } @@ -78,7 +74,6 @@ struct ConnectionTimeouts const Poco::Timespan & http_keep_alive_timeout_, const Poco::Timespan & secure_connection_timeout_, const Poco::Timespan & receive_hello_timeout_, - const Poco::Timespan & receive_tables_status_timeout_, const Poco::Timespan & receive_data_timeout_) : connection_timeout(connection_timeout_), send_timeout(send_timeout_), @@ -86,8 +81,7 @@ struct ConnectionTimeouts tcp_keep_alive_timeout(tcp_keep_alive_timeout_), http_keep_alive_timeout(http_keep_alive_timeout_), secure_connection_timeout(secure_connection_timeout_), - receive_hello_timeout(receive_hello_timeout_), - receive_tables_status_timeout(receive_tables_status_timeout_), + hedged_connection_timeout(receive_hello_timeout_), receive_data_timeout(receive_data_timeout_) { } @@ -108,8 +102,7 @@ struct ConnectionTimeouts saturate(tcp_keep_alive_timeout, limit), saturate(http_keep_alive_timeout, limit), saturate(secure_connection_timeout, limit), - saturate(receive_hello_timeout, limit), - saturate(receive_tables_status_timeout, limit), + saturate(hedged_connection_timeout, limit), saturate(receive_data_timeout, limit)); } diff --git a/src/IO/ConnectionTimeoutsContext.h b/src/IO/ConnectionTimeoutsContext.h index c6daae39e7a..c08ec2e8b92 100644 --- a/src/IO/ConnectionTimeoutsContext.h +++ b/src/IO/ConnectionTimeoutsContext.h @@ -23,8 +23,7 @@ inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const S settings.tcp_keep_alive_timeout, 0, settings.connect_timeout_with_failover_secure_ms, - settings.receive_hello_timeout, - settings.receive_tables_status_timeout, + settings.hedged_connection_timeout, settings.receive_data_timeout); } diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index fdbb6d0e3c7..8500959ec69 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -717,16 +717,18 @@ void TCPHandler::processTablesStatusRequest() response.table_states_by_id.emplace(table_name, std::move(status)); } + + writeVarUInt(Protocol::Server::TablesStatusResponse, *out); + /// For testing hedged requests const Settings & settings = query_context->getSettingsRef(); - if (settings.sleep_before_send_tables_status) + if (settings.sleep_in_send_tables_status) { - std::chrono::seconds sec(settings.sleep_before_send_tables_status); + out->next(); + std::chrono::seconds sec(settings.sleep_in_send_tables_status); std::this_thread::sleep_for(sec); } - - writeVarUInt(Protocol::Server::TablesStatusResponse, *out); response.write(*out, client_tcp_protocol_version); } @@ -940,14 +942,6 @@ void TCPHandler::receiveUnexpectedHello() void TCPHandler::sendHello() { - /// For testing hedged requests - const Settings & settings = query_context->getSettingsRef(); - if (settings.sleep_before_send_hello) - { - std::chrono::seconds sec(settings.sleep_before_send_hello); - std::this_thread::sleep_for(sec); - } - writeVarUInt(Protocol::Server::Hello, *out); writeStringBinary(DBMS_NAME, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); @@ -1405,20 +1399,21 @@ bool TCPHandler::isQueryCancelled() void TCPHandler::sendData(const Block & block) { - /// For testing hedged requests - const Settings & settings = query_context->getSettingsRef(); - if (settings.sleep_before_send_data) - { - std::chrono::seconds sec(settings.sleep_before_send_data); - std::this_thread::sleep_for(sec); - } - initBlockOutput(block); writeVarUInt(Protocol::Server::Data, *out); /// Send external table name (empty name is the main table) writeStringBinary("", *out); + /// For testing hedged requests + const Settings & settings = query_context->getSettingsRef(); + if (settings.sleep_in_send_data) + { + out->next(); + std::chrono::seconds sec(settings.sleep_in_send_data); + std::this_thread::sleep_for(sec); + } + state.block_out->write(block); state.maybe_compressed_out->next(); out->next(); diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index d3ac5c132cd..c7fd6d9ab26 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -166,7 +166,11 @@ def test_load_balancing_priority_round_robin(dist_table): def test_distributed_replica_max_ignored_errors(): settings = { +<<<<<<< Updated upstream 'use_hedged_requests' : 0, +======= + 'use_hedged_requests': 0, +>>>>>>> Stashed changes 'load_balancing': 'in_order', 'prefer_localhost_replica': 0, 'connect_timeout': 2, diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 24dc9202880..dc64b8a7ba9 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -10,23 +10,22 @@ from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager cluster = ClickHouseCluster(__file__) - NODES = {'node_' + str(i): None for i in (1, 2, 3)} + NODES['node'] = None sleep_time = 30 - @pytest.fixture(scope="module") def started_cluster(): cluster = ClickHouseCluster(__file__) NODES['node'] = cluster.add_instance( - 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) + 'node', with_zookeeper=True, stay_alive=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) for name in NODES: if name != 'node': - NODES[name] = cluster.add_instance(name, with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) - + NODES[name] = cluster.add_instance(name, with_zookeeper=True, user_configs=['configs/users1.xml']) + try: cluster.start() @@ -48,216 +47,128 @@ def started_cluster(): config = ''' - {sleep_before_send_hello} - {sleep_before_send_tables_status} - {sleep_before_send_data} + {sleep_in_send_tables_status} + {sleep_in_send_data} ''' def check_query(): + NODES['node'].restart_clickhouse() + # Without hedged requests select query will last more than 30 seconds, # with hedged requests it will last just around 1-2 second start = time.time() NODES['node'].query("SELECT * FROM distributed"); query_time = time.time() - start - print(query_time) + print("Query time:", query_time) - assert query_time < 5 - - -def test_send_hello_sleep(started_cluster): - NODES['node_1'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') - - check_query() - - -def test_send_hello_sleep2(started_cluster): - NODES['node_1'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') - - - NODES['node_2'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') - - check_query() + assert query_time < 10 def test_send_table_status_sleep(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + time.sleep(2) check_query() def test_send_table_status_sleep2(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + time.sleep(2) check_query() + def test_send_data(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + time.sleep(2) + check_query() def test_send_data2(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + time.sleep(2) check_query() def test_combination1(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + time.sleep(2) check_query() def test_combination2(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + time.sleep(2) check_query() def test_combination3(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + time.sleep(2) check_query() def test_combination4(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=sleep_time)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') - - check_query() - -def test_combination5(started_cluster): - NODES['node_1'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') - - NODES['node_2'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') - - check_query() - - -def test_combination6(started_cluster): - NODES['node_1'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') - - NODES['node_2'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0)) + time.sleep(2) check_query() diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index 95e32a0f3fc..b713cf14af4 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -20,11 +20,11 @@ sleep_time = 30 def started_cluster(): cluster = ClickHouseCluster(__file__) NODES['node'] = cluster.add_instance( - 'node', with_zookeeper=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) + 'node', with_zookeeper=True, stay_alive=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) for name in NODES: if name != 'node': - NODES[name] = cluster.add_instance(name, with_zookeeper=True, stay_alive=True, user_configs=['configs/users1.xml']) + NODES[name] = cluster.add_instance(name, with_zookeeper=True, user_configs=['configs/users1.xml']) try: cluster.start() @@ -47,78 +47,88 @@ def started_cluster(): config = ''' - {sleep_before_send_hello} - {sleep_before_send_tables_status} - {sleep_before_send_data} + {sleep_in_send_tables_status} + {sleep_in_send_data} ''' def check_query(): + NODES['node'].restart_clickhouse() + # Without hedged requests select query will last more than 30 seconds, # with hedged requests it will last just around 1-2 second start = time.time() NODES['node'].query("SELECT * FROM distributed"); query_time = time.time() - start - print(query_time) + print("Query time:", query_time) assert query_time < 5 -def test_send_hello_sleep(started_cluster): - NODES['node_1'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') - - - NODES['node_2'].replace_config( - '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=sleep_time, sleep_before_send_tables_status=0, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') - - check_query() - - def test_send_table_status_sleep(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=sleep_time, sleep_before_send_data=0)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + time.sleep(2) check_query() def test_send_data(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_1'].restart_clickhouse(sleep_time) - NODES['node_1'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', - config.format(sleep_before_send_hello=0, sleep_before_send_tables_status=0, sleep_before_send_data=sleep_time)) - - NODES['node_2'].restart_clickhouse(sleep_time) - NODES['node_2'].query('SYSTEM SYNC REPLICA replicated') + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + time.sleep(2) + check_query() + + +def test_combination1(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) + + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + + time.sleep(2) + check_query() + + +def test_combination2(started_cluster): + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) + + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + + NODES['node_4'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) + + time.sleep(2) check_query() From bc6fe4f28c4a991c05bf87610f3350c7a5512e75 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 17 Feb 2021 20:44:11 +0300 Subject: [PATCH 211/510] Minor fixes --- src/Client/Connection.cpp | 16 ++++++++++------ src/Client/Connection.h | 1 + src/Client/ConnectionPoolWithFailover.h | 1 - .../test_distributed_load_balancing/test.py | 4 ---- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index d30a6555da5..b6903ae6c92 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -66,10 +66,10 @@ void Connection::connect(const ConnectionTimeouts & timeouts) disconnect(); LOG_TRACE(log_wrapper.get(), "Connecting. Database: {}. User: {}{}{}", - default_database.empty() ? "(not specified)" : default_database, - user, - static_cast(secure) ? ". Secure" : "", - static_cast(compression) ? "" : ". Uncompressed"); + default_database.empty() ? "(not specified)" : default_database, + user, + static_cast(secure) ? ". Secure" : "", + static_cast(compression) ? "" : ". Uncompressed"); if (static_cast(secure)) { @@ -103,7 +103,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) #if defined(TCP_KEEPALIVE) TCP_KEEPALIVE #else - TCP_KEEPIDLE // __APPLE__ + TCP_KEEPIDLE // __APPLE__ #endif , timeouts.tcp_keep_alive_timeout); } @@ -120,7 +120,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) receiveHello(); LOG_TRACE(log_wrapper.get(), "Connected to {} server version {}.{}.{}.", - server_name, server_version_major, server_version_minor, server_version_patch); + server_name, server_version_major, server_version_minor, server_version_patch); } catch (Poco::Net::NetException & e) { @@ -151,6 +151,7 @@ void Connection::disconnect() connected = false; } + void Connection::sendHello() { /** Disallow control characters in user controlled parameters @@ -206,6 +207,7 @@ void Connection::sendHello() out->next(); } + void Connection::receiveHello() { /// Receive hello packet. @@ -337,6 +339,8 @@ void Connection::sendClusterNameAndSalt() bool Connection::ping() { + // LOG_TRACE(log_wrapper.get(), "Ping"); + TimeoutSetter timeout_setter(*socket, sync_request_timeout, true); try { diff --git a/src/Client/Connection.h b/src/Client/Connection.h index dd501b5f6ef..d317ecb56b3 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -196,6 +196,7 @@ public: Poco::Net::Socket * getSocket() { return socket.get(); } + /// Each time read from socket blocks and async_callback is set, it will be called. You can poll socket inside it. void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 2ecb0492747..023ef863bdf 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -31,7 +31,6 @@ enum class PoolMode GET_ALL }; - class ConnectionPoolWithFailover : public IConnectionPool, private PoolWithFailoverBase { public: diff --git a/tests/integration/test_distributed_load_balancing/test.py b/tests/integration/test_distributed_load_balancing/test.py index c7fd6d9ab26..d3ac5c132cd 100644 --- a/tests/integration/test_distributed_load_balancing/test.py +++ b/tests/integration/test_distributed_load_balancing/test.py @@ -166,11 +166,7 @@ def test_load_balancing_priority_round_robin(dist_table): def test_distributed_replica_max_ignored_errors(): settings = { -<<<<<<< Updated upstream 'use_hedged_requests' : 0, -======= - 'use_hedged_requests': 0, ->>>>>>> Stashed changes 'load_balancing': 'in_order', 'prefer_localhost_replica': 0, 'connect_timeout': 2, From acb5fb8179c2845890635582332790c94995df83 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Wed, 17 Feb 2021 20:58:04 +0300 Subject: [PATCH 212/510] Randomly shuffle replicas withing the same priority --- base/mysqlxx/PoolWithFailover.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 5bee75aab1b..e2d612d6bc4 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -1,3 +1,6 @@ +#include +#include + #include @@ -7,6 +10,8 @@ static bool startsWith(const std::string & s, const char * prefix) return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix)); } +/// This is thread-safe +std::random_device rd; using namespace mysqlxx; @@ -33,6 +38,13 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & con std::make_shared(config_, replica_name, default_connections_, max_connections_, config_name_.c_str())); } } + + static thread_local std::mt19937 rnd_generator(rd()); + for (auto & [_, replicas] : replicas_by_priority) + { + if (replicas.size() > 1) + std::shuffle(replicas.begin(), replicas.end(), rnd_generator); + } } else { From 56a5d1dafaa7cb08719277886000349490c47eda Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Feb 2021 21:48:26 +0300 Subject: [PATCH 213/510] Skip stateful functions --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 02e1914504d..456faeb72c2 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -123,6 +123,9 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes if (!filter) return 0; + if (filter->getExpression()->hasStatefulFunctions()) + return 0; + if (auto * aggregating = typeid_cast(child.get())) { const auto & params = aggregating->getParams(); From 0296d7d026ab3fb1a335d1a97a5154add718ad89 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Wed, 17 Feb 2021 21:51:05 +0300 Subject: [PATCH 214/510] Added some explanations on randomization --- base/mysqlxx/PoolWithFailover.cpp | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index e2d612d6bc4..9132773f727 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -10,7 +10,7 @@ static bool startsWith(const std::string & s, const char * prefix) return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix)); } -/// This is thread-safe +/// This reads from "/dev/urandom" and thus is thread-safe std::random_device rd; using namespace mysqlxx; @@ -39,6 +39,11 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & con } } + /// PoolWithFailover objects are stored in a cache inside PoolFactory. + /// This cache is reset by ExternalDictionariesLoader after every SYSTEM RELOAD DICTIONAR{Y|IES} + /// which triggers massive re-constructing of connection pools. + /// The state of PRNDGs like std::mt19937 is considered to be quite heavy + /// thus here we attempt to optimize its construction. static thread_local std::mt19937 rnd_generator(rd()); for (auto & [_, replicas] : replicas_by_priority) { From 80b6db7f729063778de0e2a5fc4d33d3fef27583 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 17 Feb 2021 22:07:14 +0300 Subject: [PATCH 215/510] Style --- src/Client/ConnectionPoolWithFailover.cpp | 3 --- src/Client/HedgedConnectionsFactory.cpp | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index ec9215e3bc1..acbb678d870 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -24,9 +24,6 @@ namespace DB namespace ErrorCodes { - extern const int ATTEMPT_TO_READ_AFTER_EOF; - extern const int NETWORK_ERROR; - extern const int SOCKET_TIMEOUT; extern const int LOGICAL_ERROR; } diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 3551814d603..6519e0c9a94 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -238,7 +238,7 @@ void HedgedConnectionsFactory::processConnectionEstablisherStage(int index, bool void HedgedConnectionsFactory::processFailedConnection(int index, bool remove_from_epoll) { ConnectionEstablisher & connection_establisher = replicas[index].connection_establisher; - + if (remove_from_epoll) removeReplicaFromEpoll(index); From ec4dafaa5f914e99acc8cede5b60e85458eab134 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 17 Feb 2021 22:19:39 +0300 Subject: [PATCH 216/510] Fix build. --- src/CMakeLists.txt | 4 ++-- src/Processors/ya.make | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 86db7742c97..7a7f160dd81 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -100,8 +100,8 @@ endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) -list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp) -list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h) +list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp) +list (APPEND dbms_headers Functions/IFunctionImpl.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h) list (APPEND dbms_sources AggregateFunctions/AggregateFunctionFactory.cpp diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 34ff61d03c5..71ddd07f6a2 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -114,6 +114,7 @@ SRCS( QueryPlan/MergingFinal.cpp QueryPlan/MergingSortedStep.cpp QueryPlan/OffsetStep.cpp + QueryPlan/Optimizations/filterPushDown.cpp QueryPlan/Optimizations/liftUpArrayJoin.cpp QueryPlan/Optimizations/limitPushDown.cpp QueryPlan/Optimizations/mergeExpressions.cpp From 6e244e7bb1722e23a9e616c7e8048ac2c8306885 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 22:32:38 +0300 Subject: [PATCH 217/510] Trying without fsync --- src/Coordination/Changelog.cpp | 2 +- src/Coordination/Changelog.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 9e1ed557430..a9693b2a47b 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -298,7 +298,7 @@ void Changelog::appendEntry(size_t index, nuraft::ptr log_ent if (current_writer->getEntriesWritten() == rotate_interval) rotate(index); - auto offset = current_writer->appendRecord(buildRecord(index, log_entry), true); + auto offset = current_writer->appendRecord(buildRecord(index, log_entry), false); if (!index_to_start_pos.try_emplace(index, offset).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "Record with index {} already exists", index); diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index e154c1c70c6..5f38f68750e 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -101,7 +101,7 @@ public: private: - void rotate(size_t new_start_log_idex); + void rotate(size_t new_start_log_idx); ChangelogRecord buildRecord(size_t index, nuraft::ptr log_entry) const; From ff663dc511a5daf955e559cdff0d47fa6a07f104 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Feb 2021 23:36:25 +0300 Subject: [PATCH 218/510] Fsync at server shutdown --- src/Coordination/Changelog.cpp | 13 ++++++++++++- src/Coordination/InMemoryStateManager.cpp | 5 +++++ src/Coordination/InMemoryStateManager.h | 2 ++ src/Coordination/NuKeeperServer.cpp | 1 + 4 files changed, 20 insertions(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index a9693b2a47b..2d1bbfb4440 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -467,6 +467,17 @@ void Changelog::flush() current_writer->flush(); } -Changelog::~Changelog() = default; +Changelog::~Changelog() +{ + try + { + if (current_writer) + current_writer->flush(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} } diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp index 6c4e95b993a..0423d2466f2 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/InMemoryStateManager.cpp @@ -66,6 +66,11 @@ void InMemoryStateManager::loadLogStore(size_t start_log_index) log_store->init(start_log_index); } +void InMemoryStateManager::flushLogStore() +{ + log_store->flush(); +} + void InMemoryStateManager::save_config(const nuraft::cluster_config & config) { // Just keep in memory in this example. diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/InMemoryStateManager.h index 8a7be7d0129..c53f00702d4 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/InMemoryStateManager.h @@ -27,6 +27,8 @@ public: void loadLogStore(size_t start_log_index); + void flushLogStore(); + nuraft::ptr load_config() override { return cluster_config; } void save_config(const nuraft::cluster_config & config) override; diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index a4582a5fbb8..8556fa85231 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -67,6 +67,7 @@ void NuKeeperServer::startup() void NuKeeperServer::shutdown() { state_machine->shutdownStorage(); + state_manager->flushLogStore(); if (!launcher.shutdown(coordination_settings->shutdown_timeout.totalSeconds())) LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); } From 5b597fdf446bb2039ae45d722ad423445a063a96 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Feb 2021 13:23:48 +0300 Subject: [PATCH 219/510] Force sync setting and ability to start with broken log --- src/Coordination/Changelog.cpp | 90 +++++++++++-------- src/Coordination/Changelog.h | 6 +- src/Coordination/CoordinationSettings.h | 3 +- src/Coordination/InMemoryStateManager.cpp | 6 +- src/Coordination/NuKeeperLogStore.cpp | 9 +- src/Coordination/NuKeeperLogStore.h | 3 +- src/Coordination/tests/gtest_for_build.cpp | 89 ++++++++++++------ tests/config/config.d/test_keeper_port.xml | 1 + .../configs/enable_test_keeper.xml | 1 + 9 files changed, 137 insertions(+), 71 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 2d1bbfb4440..4358fa062e8 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include namespace DB { @@ -37,7 +39,7 @@ ChangelogVersion fromString(const std::string & version_str) namespace { -static constexpr auto DEFAULT_PREFIX = "changelog"; +constexpr auto DEFAULT_PREFIX = "changelog"; std::string formatChangelogPath(const std::string & prefix, const ChangelogFileDescription & name) { @@ -151,39 +153,56 @@ public: size_t readChangelog(IndexToLogEntry & logs, size_t start_log_idx, IndexToOffset & index_to_offset) { size_t total_read = 0; - while (!read_buf.eof()) + try { - total_read += 1; - off_t pos = read_buf.count(); - ChangelogRecord record; - readIntBinary(record.header.version, read_buf); - readIntBinary(record.header.index, read_buf); - readIntBinary(record.header.term, read_buf); - readIntBinary(record.header.value_type, read_buf); - readIntBinary(record.header.blob_size, read_buf); - readIntBinary(record.header.blob_checksum, read_buf); - auto buffer = nuraft::buffer::alloc(record.header.blob_size); - auto buffer_begin = reinterpret_cast(buffer->data_begin()); - read_buf.readStrict(buffer_begin, record.header.blob_size); - index_to_offset[record.header.index] = pos; - - Checksum checksum = CityHash_v1_0_2::CityHash128(buffer_begin, record.header.blob_size); - if (checksum != record.header.blob_checksum) + while (!read_buf.eof()) { - throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, - "Checksums doesn't match for log {} (version {}), index {}, blob_size {}", - filepath, record.header.version, record.header.index, record.header.blob_size); - } - if (record.header.index < start_log_idx) - continue; + off_t pos = read_buf.count(); + ChangelogRecord record; + readIntBinary(record.header.version, read_buf); + readIntBinary(record.header.index, read_buf); + readIntBinary(record.header.term, read_buf); + readIntBinary(record.header.value_type, read_buf); + readIntBinary(record.header.blob_size, read_buf); + readIntBinary(record.header.blob_checksum, read_buf); + auto buffer = nuraft::buffer::alloc(record.header.blob_size); + auto buffer_begin = reinterpret_cast(buffer->data_begin()); + read_buf.readStrict(buffer_begin, record.header.blob_size); + index_to_offset[record.header.index] = pos; - auto log_entry = nuraft::cs_new(record.header.term, buffer, record.header.value_type); - if (!logs.try_emplace(record.header.index, log_entry).second) - throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filepath); + Checksum checksum = CityHash_v1_0_2::CityHash128(buffer_begin, record.header.blob_size); + if (checksum != record.header.blob_checksum) + { + throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, + "Checksums doesn't match for log {} (version {}), index {}, blob_size {}", + filepath, record.header.version, record.header.index, record.header.blob_size); + } + + if (logs.count(record.header.index) != 0) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filepath); + + total_read += 1; + + if (record.header.index < start_log_idx) + continue; + + auto log_entry = nuraft::cs_new(record.header.term, buffer, record.header.value_type); + + logs.emplace(record.header.index, log_entry); + } + } + catch (const Exception & ex) + { + LOG_WARNING(&Poco::Logger::get("RaftChangelog"), "Cannot completely read changelog on path {}, error: {}", filepath, ex.message()); + } + catch (...) + { + tryLogCurrentException(&Poco::Logger::get("RaftChangelog")); } return total_read; } + private: std::string filepath; ReadBufferFromFile read_buf; @@ -239,11 +258,12 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) } } - if (existing_changelogs.size() > 0 && read_from_last < entries_in_last) + if (!existing_changelogs.empty() && read_from_last < entries_in_last) { auto description = existing_changelogs.rbegin()->second; current_writer = std::make_unique(description.path, WriteMode::Append, description.from_log_idx); current_writer->setEntriesWritten(read_from_last); + current_writer->truncateToLength(index_to_start_pos[read_from_last]); } else { @@ -287,7 +307,7 @@ ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) +void Changelog::appendEntry(size_t index, nuraft::ptr log_entry, bool force_sync) { if (!current_writer) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); @@ -298,14 +318,14 @@ void Changelog::appendEntry(size_t index, nuraft::ptr log_ent if (current_writer->getEntriesWritten() == rotate_interval) rotate(index); - auto offset = current_writer->appendRecord(buildRecord(index, log_entry), false); + auto offset = current_writer->appendRecord(buildRecord(index, log_entry), force_sync); if (!index_to_start_pos.try_emplace(index, offset).second) throw Exception(ErrorCodes::LOGICAL_ERROR, "Record with index {} already exists", index); logs[index] = makeClone(log_entry); } -void Changelog::writeAt(size_t index, nuraft::ptr log_entry) +void Changelog::writeAt(size_t index, nuraft::ptr log_entry, bool force_sync) { if (index_to_start_pos.count(index) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); @@ -347,7 +367,7 @@ void Changelog::writeAt(size_t index, nuraft::ptr log_entry) current_writer->setEntriesWritten(entries_written); - appendEntry(index, log_entry); + appendEntry(index, log_entry, force_sync); } void Changelog::compact(size_t up_to_log_idx) @@ -441,7 +461,7 @@ nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, in return buf_out; } -void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer) +void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync) { buffer.pos(0); int num_logs = buffer.get_int(); @@ -456,9 +476,9 @@ void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer) LogEntryPtr log_entry = nuraft::log_entry::deserialize(*buf_local); if (i == 0 && logs.count(cur_idx)) - writeAt(cur_idx, log_entry); + writeAt(cur_idx, log_entry, force_sync); else - appendEntry(cur_idx, log_entry); + appendEntry(cur_idx, log_entry, force_sync); } } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 5f38f68750e..38d83819da2 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -64,9 +64,9 @@ public: void readChangelogAndInitWriter(size_t from_log_idx); - void appendEntry(size_t index, LogEntryPtr log_entry); + void appendEntry(size_t index, LogEntryPtr log_entry, bool force_sync); - void writeAt(size_t index, LogEntryPtr log_entry); + void writeAt(size_t index, LogEntryPtr log_entry, bool force_sync); void compact(size_t up_to_log_idx); @@ -88,7 +88,7 @@ public: nuraft::ptr serializeEntriesToBuffer(size_t index, int32_t cnt); - void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer); + void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync); void flush(); diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index 0f1afb3fffe..ba3d3a7141a 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -29,7 +29,8 @@ struct Settings; M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \ M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \ M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ - M(UInt64, rotate_log_storage_interval, 500000, "How many records will be stored in one log storage file", 0) + M(UInt64, rotate_log_storage_interval, 500000, "How many records will be stored in one log storage file", 0) \ + M(Bool, force_sync, true, " Call fsync on each change in RAFT changelog", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/InMemoryStateManager.cpp index 0423d2466f2..084ab043d12 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/InMemoryStateManager.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) : my_server_id(server_id_) , my_port(port) - , log_store(nuraft::cs_new(logs_path, 5000)) + , log_store(nuraft::cs_new(logs_path, 5000, true)) , cluster_config(nuraft::cs_new()) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); @@ -25,7 +25,9 @@ InMemoryStateManager::InMemoryStateManager( const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings) : my_server_id(my_server_id_) - , log_store(nuraft::cs_new(config.getString(config_prefix + ".log_storage_path"), coordination_settings->rotate_log_storage_interval)) + , log_store(nuraft::cs_new( + config.getString(config_prefix + ".log_storage_path"), + coordination_settings->rotate_log_storage_interval, coordination_settings->force_sync)) , cluster_config(nuraft::cs_new()) { diff --git a/src/Coordination/NuKeeperLogStore.cpp b/src/Coordination/NuKeeperLogStore.cpp index fa8d6d6c299..8834bdc4d69 100644 --- a/src/Coordination/NuKeeperLogStore.cpp +++ b/src/Coordination/NuKeeperLogStore.cpp @@ -3,8 +3,9 @@ namespace DB { -NuKeeperLogStore::NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_) +NuKeeperLogStore::NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_) : changelog(changelogs_path, rotate_interval_) + , force_sync(force_sync_) { } @@ -36,7 +37,7 @@ size_t NuKeeperLogStore::append(nuraft::ptr & entry) { std::lock_guard lock(changelog_lock); size_t idx = changelog.getNextEntryIndex(); - changelog.appendEntry(idx, entry); + changelog.appendEntry(idx, entry, force_sync); return idx; } @@ -44,7 +45,7 @@ size_t NuKeeperLogStore::append(nuraft::ptr & entry) void NuKeeperLogStore::write_at(size_t index, nuraft::ptr & entry) { std::lock_guard lock(changelog_lock); - changelog.writeAt(index, entry); + changelog.writeAt(index, entry, force_sync); } nuraft::ptr>> NuKeeperLogStore::log_entries(size_t start, size_t end) @@ -91,7 +92,7 @@ bool NuKeeperLogStore::flush() void NuKeeperLogStore::apply_pack(size_t index, nuraft::buffer & pack) { std::lock_guard lock(changelog_lock); - changelog.applyEntriesFromBuffer(index, pack); + changelog.applyEntriesFromBuffer(index, pack, force_sync); } size_t NuKeeperLogStore::size() const diff --git a/src/Coordination/NuKeeperLogStore.h b/src/Coordination/NuKeeperLogStore.h index 49d5dbfdf7c..0ff92220316 100644 --- a/src/Coordination/NuKeeperLogStore.h +++ b/src/Coordination/NuKeeperLogStore.h @@ -11,7 +11,7 @@ namespace DB class NuKeeperLogStore : public nuraft::log_store { public: - NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_); + NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_); void init(size_t from_log_idx); @@ -44,6 +44,7 @@ public: private: mutable std::mutex changelog_lock; Changelog changelog; + bool force_sync; }; } diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 81e1751c08c..3fd2db84e3e 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -6,7 +6,8 @@ #endif #if USE_NURAFT - +#include +#include #include #include #include @@ -20,6 +21,7 @@ #include #include #include +#include #include // Y_IGNORE #include #include @@ -372,7 +374,7 @@ DB::LogEntryPtr getLogEntry(const std::string & s, size_t term) TEST(CoordinationTest, ChangelogTestSimple) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -386,7 +388,7 @@ TEST(CoordinationTest, ChangelogTestSimple) TEST(CoordinationTest, ChangelogTestFile) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); auto entry = getLogEntry("hello world", 77); changelog.append(entry); @@ -407,7 +409,7 @@ TEST(CoordinationTest, ChangelogTestFile) TEST(CoordinationTest, ChangelogReadWrite) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 1000); + DB::NuKeeperLogStore changelog("./logs", 1000, true); changelog.init(1); for (size_t i = 0; i < 10; ++i) { @@ -415,7 +417,7 @@ TEST(CoordinationTest, ChangelogReadWrite) changelog.append(entry); } EXPECT_EQ(changelog.size(), 10); - DB::NuKeeperLogStore changelog_reader("./logs", 1000); + DB::NuKeeperLogStore changelog_reader("./logs", 1000, true); changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), 10); EXPECT_EQ(changelog_reader.last_entry()->get_term(), changelog.last_entry()->get_term()); @@ -434,7 +436,7 @@ TEST(CoordinationTest, ChangelogReadWrite) TEST(CoordinationTest, ChangelogWriteAt) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 1000); + DB::NuKeeperLogStore changelog("./logs", 1000, true); changelog.init(1); for (size_t i = 0; i < 10; ++i) { @@ -450,7 +452,7 @@ TEST(CoordinationTest, ChangelogWriteAt) EXPECT_EQ(changelog.entry_at(7)->get_term(), 77); EXPECT_EQ(changelog.next_slot(), 8); - DB::NuKeeperLogStore changelog_reader("./logs", 1000); + DB::NuKeeperLogStore changelog_reader("./logs", 1000, true); changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), changelog.size()); @@ -463,7 +465,7 @@ TEST(CoordinationTest, ChangelogWriteAt) TEST(CoordinationTest, ChangelogTestAppendAfterRead) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 7; ++i) { @@ -475,7 +477,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead) EXPECT_TRUE(fs::exists("./logs/changelog_1_5.bin")); EXPECT_TRUE(fs::exists("./logs/changelog_6_10.bin")); - DB::NuKeeperLogStore changelog_reader("./logs", 5); + DB::NuKeeperLogStore changelog_reader("./logs", 5, true); changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), 7); @@ -511,7 +513,7 @@ TEST(CoordinationTest, ChangelogTestAppendAfterRead) TEST(CoordinationTest, ChangelogTestCompaction) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 3; ++i) @@ -552,7 +554,7 @@ TEST(CoordinationTest, ChangelogTestCompaction) EXPECT_EQ(changelog.next_slot(), 8); EXPECT_EQ(changelog.last_entry()->get_term(), 60); /// And we able to read it - DB::NuKeeperLogStore changelog_reader("./logs", 5); + DB::NuKeeperLogStore changelog_reader("./logs", 5, true); changelog_reader.init(7); EXPECT_EQ(changelog_reader.size(), 1); EXPECT_EQ(changelog_reader.start_index(), 7); @@ -563,7 +565,7 @@ TEST(CoordinationTest, ChangelogTestCompaction) TEST(CoordinationTest, ChangelogTestBatchOperations) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 100); + DB::NuKeeperLogStore changelog("./logs", 100, true); changelog.init(1); for (size_t i = 0; i < 10; ++i) { @@ -575,7 +577,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperations) auto entries = changelog.pack(1, 5); - DB::NuKeeperLogStore apply_changelog("./logs", 100); + DB::NuKeeperLogStore apply_changelog("./logs", 100, true); apply_changelog.init(1); for (size_t i = 0; i < 10; ++i) @@ -605,7 +607,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperations) TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 100); + DB::NuKeeperLogStore changelog("./logs", 100, true); changelog.init(1); for (size_t i = 0; i < 10; ++i) { @@ -618,7 +620,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) auto entries = changelog.pack(5, 5); ChangelogDirTest test1("./logs1"); - DB::NuKeeperLogStore changelog_new("./logs1", 100); + DB::NuKeeperLogStore changelog_new("./logs1", 100, true); changelog_new.init(1); EXPECT_EQ(changelog_new.size(), 0); @@ -637,7 +639,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) EXPECT_EQ(changelog_new.start_index(), 5); EXPECT_EQ(changelog_new.next_slot(), 11); - DB::NuKeeperLogStore changelog_reader("./logs1", 100); + DB::NuKeeperLogStore changelog_reader("./logs1", 100, true); changelog_reader.init(5); } @@ -645,7 +647,7 @@ TEST(CoordinationTest, ChangelogTestBatchOperationsEmpty) TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 33; ++i) @@ -680,7 +682,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); - DB::NuKeeperLogStore changelog_read("./logs", 5); + DB::NuKeeperLogStore changelog_read("./logs", 5, true); changelog_read.init(1); EXPECT_EQ(changelog_read.size(), 7); EXPECT_EQ(changelog_read.start_index(), 1); @@ -691,7 +693,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtPreviousFile) TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 33; ++i) @@ -726,7 +728,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); - DB::NuKeeperLogStore changelog_read("./logs", 5); + DB::NuKeeperLogStore changelog_read("./logs", 5, true); changelog_read.init(1); EXPECT_EQ(changelog_read.size(), 11); EXPECT_EQ(changelog_read.start_index(), 1); @@ -737,7 +739,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtFileBorder) TEST(CoordinationTest, ChangelogTestWriteAtAllFiles) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 33; ++i) @@ -776,7 +778,7 @@ TEST(CoordinationTest, ChangelogTestWriteAtAllFiles) TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 35; ++i) @@ -795,7 +797,7 @@ TEST(CoordinationTest, ChangelogTestStartNewLogAfterRead) EXPECT_FALSE(fs::exists("./logs/changelog_36_40.bin")); - DB::NuKeeperLogStore changelog_reader("./logs", 5); + DB::NuKeeperLogStore changelog_reader("./logs", 5, true); changelog_reader.init(1); auto entry = getLogEntry("36_hello_world", 360); @@ -817,7 +819,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) { ChangelogDirTest test("./logs"); - DB::NuKeeperLogStore changelog("./logs", 5); + DB::NuKeeperLogStore changelog("./logs", 5, true); changelog.init(1); for (size_t i = 0; i < 35; ++i) @@ -837,7 +839,7 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) DB::WriteBufferFromFile plain_buf("./logs/changelog_11_15.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); plain_buf.truncate(0); - DB::NuKeeperLogStore changelog_reader("./logs", 5); + DB::NuKeeperLogStore changelog_reader("./logs", 5, true); changelog_reader.init(1); EXPECT_EQ(changelog_reader.size(), 10); @@ -867,4 +869,41 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); } +TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) +{ + ChangelogDirTest test("./logs"); + + DB::NuKeeperLogStore changelog("./logs", 20, true); + changelog.init(1); + + for (size_t i = 0; i < 35; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); + changelog.append(entry); + } + + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin")); + + DB::WriteBufferFromFile plain_buf("./logs/changelog_1_20.bin", DBMS_DEFAULT_BUFFER_SIZE, O_APPEND | O_CREAT | O_WRONLY); + plain_buf.truncate(140); + + DB::NuKeeperLogStore changelog_reader("./logs", 20, true); + changelog_reader.init(1); + + EXPECT_EQ(changelog_reader.size(), 2); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), 450); + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin")); + EXPECT_FALSE(fs::exists("./logs/changelog_21_40.bin")); +} + +int main(int argc, char ** argv) +{ + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); + testing::InitGoogleTest(&argc, argv); + return RUN_ALL_TESTS(); +} + #endif diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index 44123ffe9c1..88fbf027ce7 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -9,6 +9,7 @@ 30000 0 0 + false diff --git a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml index a8b8991f959..2cf9f8022d1 100644 --- a/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml +++ b/tests/integration/test_testkeeper_back_to_back/configs/enable_test_keeper.xml @@ -8,6 +8,7 @@ 5000 10000 trace + false From 7231a97085b34d0ee6fa14a23a085a0bd60cc01f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Feb 2021 14:15:16 +0300 Subject: [PATCH 220/510] Remove MaterializingStep --- .../QueryPlan/MaterializingStep.cpp | 39 ------------------- src/Processors/QueryPlan/MaterializingStep.h | 18 --------- src/Processors/ya.make | 1 - src/Storages/StorageView.cpp | 6 ++- 4 files changed, 4 insertions(+), 60 deletions(-) delete mode 100644 src/Processors/QueryPlan/MaterializingStep.cpp delete mode 100644 src/Processors/QueryPlan/MaterializingStep.h diff --git a/src/Processors/QueryPlan/MaterializingStep.cpp b/src/Processors/QueryPlan/MaterializingStep.cpp deleted file mode 100644 index f5313369020..00000000000 --- a/src/Processors/QueryPlan/MaterializingStep.cpp +++ /dev/null @@ -1,39 +0,0 @@ -#include -#include -#include - -#include - -namespace DB -{ - -static ITransformingStep::Traits getTraits() -{ - return ITransformingStep::Traits - { - { - .preserves_distinct_columns = true, - .returns_single_stream = false, - .preserves_number_of_streams = true, - .preserves_sorting = true, - }, - { - .preserves_number_of_rows = true, - } - }; -} - -MaterializingStep::MaterializingStep(const DataStream & input_stream_) - : ITransformingStep(input_stream_, materializeBlock(input_stream_.header), getTraits()) -{ -} - -void MaterializingStep::transformPipeline(QueryPipeline & pipeline) -{ - pipeline.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header); - }); -} - -} diff --git a/src/Processors/QueryPlan/MaterializingStep.h b/src/Processors/QueryPlan/MaterializingStep.h deleted file mode 100644 index 72b3133dfe4..00000000000 --- a/src/Processors/QueryPlan/MaterializingStep.h +++ /dev/null @@ -1,18 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -/// Materialize constants. See MaterializingTransform. -class MaterializingStep : public ITransformingStep -{ -public: - explicit MaterializingStep(const DataStream & input_stream_); - - String getName() const override { return "Materializing"; } - - void transformPipeline(QueryPipeline & pipeline) override; -}; - -} diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 71ddd07f6a2..a44272cf9c0 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -108,7 +108,6 @@ SRCS( QueryPlan/ITransformingStep.cpp QueryPlan/LimitByStep.cpp QueryPlan/LimitStep.cpp - QueryPlan/MaterializingStep.cpp QueryPlan/MergeSortingStep.cpp QueryPlan/MergingAggregatedStep.cpp QueryPlan/MergingFinal.cpp diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 38349ef8df9..1ee5ab3d0ca 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -15,7 +15,6 @@ #include #include -#include #include #include @@ -87,7 +86,10 @@ void StorageView::read( /// It's expected that the columns read from storage are not constant. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. - auto materializing = std::make_unique(query_plan.getCurrentDataStream()); + auto materializing_actions = std::make_shared(query_plan.getCurrentDataStream().header.getColumnsWithTypeAndName()); + materializing_actions->addMaterializingOutputActions(); + + auto materializing = std::make_unique(query_plan.getCurrentDataStream(), std::move(materializing_actions)); materializing->setStepDescription("Materialize constants after VIEW subquery"); query_plan.addStep(std::move(materializing)); From 2f5b4c20aee69acc1172d8637dfa5b98cca8d4c2 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 18 Feb 2021 14:21:48 +0300 Subject: [PATCH 221/510] Fix --- src/Client/ConnectionEstablisher.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index e529d366fdc..f92d878b670 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -47,6 +47,9 @@ void ConnectionEstablisher::Routine::ReadCallback::operator()(int fd, const Poco connection_establisher.receive_timeout.setRelative(timeout); fiber = std::move(fiber).resume(); connection_establisher.receive_timeout.reset(); +#else + (void) fd; + (void) timeout; #endif } @@ -87,7 +90,7 @@ void ConnectionEstablisher::resume() bool is_receive_timeout_alarmed = false; epoll_event events[2]; - events[0].data.fd = events[1].data.fd; + events[0].data.fd = events[1].data.fd = -1; size_t ready_count = epoll.getManyReady(2, events, true); for (size_t i = 0; i != ready_count; ++i) { From 7f815325ba92e487712488e6a368ab12133421b7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Feb 2021 14:42:09 +0300 Subject: [PATCH 222/510] More tests for broken changelog read --- src/Coordination/Changelog.cpp | 42 +++++++++++++------ src/Coordination/tests/gtest_for_build.cpp | 15 +++++++ .../configs/use_test_keeper.xml | 2 +- 3 files changed, 45 insertions(+), 14 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 4358fa062e8..12943bd9272 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -142,6 +142,13 @@ private: size_t start_index; }; +struct ChangelogReadResult +{ + size_t entries_read; + off_t last_position; + bool error; +}; + class ChangelogReader { public: @@ -150,14 +157,15 @@ public: , read_buf(filepath) {} - size_t readChangelog(IndexToLogEntry & logs, size_t start_log_idx, IndexToOffset & index_to_offset) + ChangelogReadResult readChangelog(IndexToLogEntry & logs, size_t start_log_idx, IndexToOffset & index_to_offset) { - size_t total_read = 0; + size_t previous_index = 0; + ChangelogReadResult result{}; try { while (!read_buf.eof()) { - off_t pos = read_buf.count(); + result.last_position = read_buf.count(); ChangelogRecord record; readIntBinary(record.header.version, read_buf); readIntBinary(record.header.index, read_buf); @@ -168,7 +176,11 @@ public: auto buffer = nuraft::buffer::alloc(record.header.blob_size); auto buffer_begin = reinterpret_cast(buffer->data_begin()); read_buf.readStrict(buffer_begin, record.header.blob_size); - index_to_offset[record.header.index] = pos; + + if (previous_index != 0 && previous_index + 1 != record.header.index) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Previous log entry {}, next log entry {}, seems like some entries skipped", previous_index, record.header.index); + + previous_index = record.header.index; Checksum checksum = CityHash_v1_0_2::CityHash128(buffer_begin, record.header.blob_size); if (checksum != record.header.blob_checksum) @@ -181,7 +193,7 @@ public: if (logs.count(record.header.index) != 0) throw Exception(ErrorCodes::CORRUPTED_DATA, "Duplicated index id {} in log {}", record.header.index, filepath); - total_read += 1; + result.entries_read += 1; if (record.header.index < start_log_idx) continue; @@ -189,18 +201,21 @@ public: auto log_entry = nuraft::cs_new(record.header.term, buffer, record.header.value_type); logs.emplace(record.header.index, log_entry); + index_to_offset[record.header.index] = result.last_position; } } catch (const Exception & ex) { + result.error = true; LOG_WARNING(&Poco::Logger::get("RaftChangelog"), "Cannot completely read changelog on path {}, error: {}", filepath, ex.message()); } catch (...) { + result.error = true; tryLogCurrentException(&Poco::Logger::get("RaftChangelog")); } - return total_read; + return result; } private: @@ -225,11 +240,11 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval void Changelog::readChangelogAndInitWriter(size_t from_log_idx) { - size_t read_from_last = 0; start_index = from_log_idx == 0 ? 1 : from_log_idx; size_t total_read = 0; size_t entries_in_last = 0; size_t incomplete_log_idx = 0; + ChangelogReadResult result{}; for (const auto & [start_idx, changelog_description] : existing_changelogs) { entries_in_last = changelog_description.to_log_idx - changelog_description.from_log_idx + 1; @@ -237,11 +252,11 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) if (changelog_description.to_log_idx >= from_log_idx) { ChangelogReader reader(changelog_description.path); - read_from_last = reader.readChangelog(logs, from_log_idx, index_to_start_pos); - total_read += read_from_last; + result = reader.readChangelog(logs, from_log_idx, index_to_start_pos); + total_read += result.entries_read; /// May happen after truncate and crash - if (read_from_last < entries_in_last) + if (result.entries_read < entries_in_last) { incomplete_log_idx = start_idx; break; @@ -258,12 +273,13 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_idx) } } - if (!existing_changelogs.empty() && read_from_last < entries_in_last) + if (!existing_changelogs.empty() && result.entries_read < entries_in_last) { auto description = existing_changelogs.rbegin()->second; current_writer = std::make_unique(description.path, WriteMode::Append, description.from_log_idx); - current_writer->setEntriesWritten(read_from_last); - current_writer->truncateToLength(index_to_start_pos[read_from_last]); + current_writer->setEntriesWritten(result.entries_read); + if (result.error) + current_writer->truncateToLength(result.last_position); } else { diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 3fd2db84e3e..457d0dbc52a 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -867,6 +867,11 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate) EXPECT_FALSE(fs::exists("./logs/changelog_21_25.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_26_30.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_31_35.bin")); + + DB::NuKeeperLogStore changelog_reader2("./logs", 5, true); + changelog_reader2.init(1); + EXPECT_EQ(changelog_reader2.size(), 11); + EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); } TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) @@ -895,6 +900,16 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader.last_entry()->get_term(), 450); EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin")); EXPECT_FALSE(fs::exists("./logs/changelog_21_40.bin")); + auto entry = getLogEntry("hello_world", 7777); + changelog_reader.append(entry); + EXPECT_EQ(changelog_reader.size(), 3); + EXPECT_EQ(changelog_reader.last_entry()->get_term(), 7777); + + + DB::NuKeeperLogStore changelog_reader2("./logs", 20, true); + changelog_reader2.init(1); + EXPECT_EQ(changelog_reader2.size(), 3); + EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); } int main(int argc, char ** argv) diff --git a/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml index 12dc7fd9447..2e48e91bca5 100644 --- a/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml +++ b/tests/integration/test_testkeeper_persistent_log/configs/use_test_keeper.xml @@ -1,7 +1,7 @@ - node1 + node 9181 From 904b4754ccbd5a63b95402ae913c57ea2a260b5c Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Feb 2021 14:47:37 +0300 Subject: [PATCH 223/510] Fix tidy --- src/Coordination/Changelog.cpp | 2 +- src/Coordination/Changelog.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 12943bd9272..a332ce37a8c 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -302,7 +302,7 @@ void Changelog::rotate(size_t new_start_log_idx) current_writer = std::make_unique(new_description.path, WriteMode::Rewrite, new_start_log_idx); } -ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) const +ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) { ChangelogRecordHeader header; header.index = index; diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 38d83819da2..779d057d285 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -103,7 +103,7 @@ private: void rotate(size_t new_start_log_idx); - ChangelogRecord buildRecord(size_t index, nuraft::ptr log_entry) const; + static ChangelogRecord buildRecord(size_t index, nuraft::ptr log_entry); private: std::string changelogs_dir; From 2aad067e7c092af8162f1048b93c80216ec2d8f9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Feb 2021 12:16:58 +0000 Subject: [PATCH 224/510] Support conversion for postgres numeric without precision and scale --- .../fetchPostgreSQLTableStructure.cpp | 35 ++++++++++++------- .../test_storage_postgresql/test.py | 8 ++--- 2 files changed, 27 insertions(+), 16 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index ec23cfc8794..15ce9a1baed 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -54,19 +54,30 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl res = std::make_shared(); else if (type.starts_with("numeric")) { - /// Numeric and decimal will both end up here as numeric. - res = DataTypeFactory::instance().get(type); - uint32_t precision = getDecimalPrecision(*res); - uint32_t scale = getDecimalScale(*res); + /// Numeric and decimal will both end up here as numeric. If it has type and precision, + /// there will be Numeric(x, y), otherwise just Numeric + uint32_t precision, scale; + if (type.ends_with(")")) + { + res = DataTypeFactory::instance().get(type); + precision = getDecimalPrecision(*res); + scale = getDecimalScale(*res); + + if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + else if (precision <= DecimalUtils::maxPrecision()) + res = std::make_shared>(precision, scale); + } + else + { + precision = DecimalUtils::maxPrecision(); + res = std::make_shared>(precision, precision); + } - if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); - else if (precision <= DecimalUtils::maxPrecision()) - res = std::make_shared>(precision, scale); } if (!res) diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 4f567c19f2b..03af32a4803 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -63,13 +63,13 @@ def test_postgres_conversions(started_cluster): cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, - h timestamp, i date, j numeric(5, 5), k decimal(5, 5))''') + h timestamp, i date, j decimal(5, 5), k numeric)''') node1.query(''' INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword') VALUES - (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.2, 0.2)''') + (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.22222, 0.22222)''') result = node1.query(''' - SELECT * FROM postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword')''') - assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.20000\t0.20000\n') + SELECT a, b, c, d, e, f, g, h, i, j, toDecimal32(k, 5) FROM postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword')''') + assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.22222\t0.22222\n') cursor.execute( '''CREATE TABLE IF NOT EXISTS test_array_dimensions From 0449546bca7319132a99693b6634ca8684aa41f3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Feb 2021 16:13:09 +0300 Subject: [PATCH 225/510] Support TotalsHaving. Update test. --- .../Optimizations/filterPushDown.cpp | 38 ++++++++++++++++--- src/Processors/QueryPlan/TotalsHavingStep.h | 2 + .../01655_plan_optimizations.reference | 9 +++++ .../0_stateless/01655_plan_optimizations.sh | 12 ++++++ 4 files changed, 56 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 456faeb72c2..4d01235e2fc 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -5,14 +5,17 @@ #include #include #include +#include +#include +#include +#include +#include +#include #include #include #include -#include "Processors/QueryPlan/FinishSortingStep.h" -#include "Processors/QueryPlan/MergeSortingStep.h" -#include "Processors/QueryPlan/MergingSortedStep.h" -#include "Processors/QueryPlan/PartialSortingStep.h" -#include +#include + #include namespace DB::ErrorCodes @@ -135,6 +138,31 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes return updated_steps; } + if (auto * totals_having = typeid_cast(child.get())) + { + /// If totals step has HAVING expression, skip it for now. + /// TODO: + /// We can merge HAING expression with current filer. + /// Alos, we can push down part of HAVING which depend only on aggregation keys. + if (totals_having->getActions()) + return 0; + + Names keys; + const auto & header = totals_having->getInputStreams().front().header; + for (const auto & column : header) + if (typeid_cast(column.type.get()) == nullptr) + keys.push_back(column.name); + + /// NOTE: this optimization changes TOTALS value. Example: + /// `select * from (select y, sum(x) from ( + /// select number as x, number % 4 as y from numbers(10) + /// ) group by y with totals) where y != 2` + /// Optimization will replace totals row `y, sum(x)` from `(0, 45)` to `(0, 37)`. + /// It is expected to ok, cause AST optimization `enable_optimize_predicate_expression = 1` also brakes it. + if (auto updated_steps = tryAddNewFilterStep(parent_node, nodes, keys)) + return updated_steps; + } + if (auto * array_join = typeid_cast(child.get())) { const auto & array_join_actions = array_join->arrayJoin(); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.h b/src/Processors/QueryPlan/TotalsHavingStep.h index 7c1638013e5..57d5cf7aad5 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.h +++ b/src/Processors/QueryPlan/TotalsHavingStep.h @@ -28,6 +28,8 @@ public: void describeActions(FormatSettings & settings) const override; + const ActionsDAGPtr & getActions() const { return actions_dag; } + private: bool overflow_row; ActionsDAGPtr actions_dag; diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index 7bc75dc0bf6..fa83c098412 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -112,3 +112,12 @@ PartialSorting Filter column: and(notEquals(x, 0), notEquals(y, 0)) 1 2 1 1 +> filter is pushed down before TOTALS HAVING and aggregating +TotalsHaving +Aggregating +Filter column: notEquals(y, 2) +0 12 +1 15 +3 10 + +0 37 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index f770643fc41..e47b03661e4 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -135,3 +135,15 @@ $CLICKHOUSE_CLIENT -q " select number % 2 as x, number % 3 as y from numbers(6) order by y desc ) where x != 0 and y != 0 settings enable_optimize_predicate_expression = 0" + +echo "> filter is pushed down before TOTALS HAVING and aggregating" +$CLICKHOUSE_CLIENT -q " + explain actions = 1 select * from ( + select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals + ) where y != 2 + settings enable_optimize_predicate_expression=0" | + grep -o "TotalsHaving\|Aggregating\|Filter column: notEquals(y, 2)" +$CLICKHOUSE_CLIENT -q " + select * from ( + select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals + ) where y != 2" \ No newline at end of file From 556dc81ab990803f082dc6365656e5aac58a0a03 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Feb 2021 16:32:01 +0300 Subject: [PATCH 226/510] Fix undefined-behavior in ReservoirSamplerDeterministic.h --- .../ReservoirSamplerDeterministic.h | 40 +++++++++++++------ 1 file changed, 27 insertions(+), 13 deletions(-) diff --git a/src/AggregateFunctions/ReservoirSamplerDeterministic.h b/src/AggregateFunctions/ReservoirSamplerDeterministic.h index 3b7817e9308..3013a17e1ca 100644 --- a/src/AggregateFunctions/ReservoirSamplerDeterministic.h +++ b/src/AggregateFunctions/ReservoirSamplerDeterministic.h @@ -56,7 +56,7 @@ class ReservoirSamplerDeterministic { bool good(const UInt32 hash) { - return hash == ((hash >> skip_degree) << skip_degree); + return !(hash & skip_mask); } public: @@ -135,11 +135,8 @@ public: throw Poco::Exception("Cannot merge ReservoirSamplerDeterministic's with different max sample size"); sorted = false; - if (b.skip_degree > skip_degree) - { - skip_degree = b.skip_degree; - thinOut(); - } + if (skip_degree < b.skip_degree) + setSkipDegree(b.skip_degree); for (const auto & sample : b.samples) if (good(sample.second)) @@ -184,22 +181,39 @@ private: size_t total_values = 0; /// How many values were inserted (regardless if they remain in sample or not). bool sorted = false; Array samples; - UInt8 skip_degree = 0; /// The number N determining that we save only one per 2^N elements in average. + + /// The number N determining that we store only one per 2^N elements in average. + UInt8 skip_degree = 0; + + /// skip_mask is calculated as (2 ^ skip_degree - 1). We store an element only if (hash & skip_mask) == 0. + /// For example, if skip_degree==0 then skip_mask==0 means we store each element; + /// if skip_degree==1 then skip_mask==0b0001 means we store one per 2 elements in average; + /// if skip_degree==4 then skip_mask==0b1111 means we store one per 16 elements in average. + UInt32 skip_mask = 0; void insertImpl(const T & v, const UInt32 hash) { /// Make a room for plus one element. while (samples.size() >= max_sample_size) - { - ++skip_degree; - if (skip_degree > detail::MAX_SKIP_DEGREE) - throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED}; - thinOut(); - } + setSkipDegree(skip_degree + 1); samples.emplace_back(v, hash); } + void setSkipDegree(UInt8 skip_degree_) + { + if (skip_degree_ == skip_degree) + return; + if (skip_degree_ > detail::MAX_SKIP_DEGREE) + throw DB::Exception{"skip_degree exceeds maximum value", DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED}; + skip_degree = skip_degree_; + if (skip_degree == detail::MAX_SKIP_DEGREE) + skip_mask = static_cast(-1); + else + skip_mask = (1 << skip_degree) - 1; + thinOut(); + } + void thinOut() { samples.resize(std::distance(samples.begin(), From 0336764426a2e5950dcc6ce27b6d89de09209368 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 18 Feb 2021 18:51:16 +0300 Subject: [PATCH 227/510] Fix tidy one more time --- src/Coordination/Changelog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index a332ce37a8c..4a3955e23ab 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -174,7 +174,7 @@ public: readIntBinary(record.header.blob_size, read_buf); readIntBinary(record.header.blob_checksum, read_buf); auto buffer = nuraft::buffer::alloc(record.header.blob_size); - auto buffer_begin = reinterpret_cast(buffer->data_begin()); + auto * buffer_begin = reinterpret_cast(buffer->data_begin()); read_buf.readStrict(buffer_begin, record.header.blob_size); if (previous_index != 0 && previous_index + 1 != record.header.index) From 5cfe245e2203cf4ca62bc5e72897ebd358a64b5b Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:21:12 +0300 Subject: [PATCH 228/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 92e674242df..1edebc26ccc 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -660,7 +660,7 @@ AS parseDateTimeBestEffortUS; ## parseDateTimeBestEffortUSOrNull {#parsedatetimebesteffortusornull} -Похожа на функцию [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но разница состоит в том, что возвращает `NULL`, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). +Работает аналогично функции [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но в отличие от нее возвращает `NULL`, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). **Синтаксис** From 1626833987b869c36096becebafbbb516939397d Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:21:25 +0300 Subject: [PATCH 229/510] Update docs/ru/sql-reference/functions/type-conversion-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 1edebc26ccc..80f24d53515 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -746,7 +746,7 @@ SELECT parseDateTimeBestEffortUSOrNull('10.2021') AS parseDateTimeBestEffortUSOr ## parseDateTimeBestEffortUSOrZero {#parsedatetimebesteffortusorzero} -Похожа на функцию [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но разница в том, что возвращает нулевую дату или нулевую дату со временем, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). +Работает аналогично функции [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но в отличие от нее возвращает нулевую дату или нулевую дату со временем, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). **Синтаксис** From 03640221a84828043770dd89e9fa2011af0ed126 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 18 Feb 2021 21:33:30 +0300 Subject: [PATCH 230/510] Add the zero date MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Добавил нулевую дату. --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 11d54790ac2..def37cef366 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -777,7 +777,7 @@ Result: ## parseDateTimeBestEffortUSOrZero {#parsedatetimebesteffortusorzero} -Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns zero date or zero date with time when it encounters a date format that cannot be processed. +Same as [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS) function except that it returns zero date (`1970-01-01`) or zero date with time (`1970-01-01 00:00:00`) when it encounters a date format that cannot be processed. **Syntax** diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 80f24d53515..4de2b5c6e3e 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -746,7 +746,7 @@ SELECT parseDateTimeBestEffortUSOrNull('10.2021') AS parseDateTimeBestEffortUSOr ## parseDateTimeBestEffortUSOrZero {#parsedatetimebesteffortusorzero} -Работает аналогично функции [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но в отличие от нее возвращает нулевую дату или нулевую дату со временем, если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). +Работает аналогично функции [parseDateTimeBestEffortUS](#parsedatetimebesteffortUS), но в отличие от нее возвращает нулевую дату (`1970-01-01`) или нулевую дату со временем (`1970-01-01 00:00:00`), если входная строка не может быть преобразована в тип данных [DateTime](../../sql-reference/data-types/datetime.md). **Синтаксис** From 0e8a951ac59d5d78f0bb7d9f1a1b78f7993560c4 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:38:54 +0300 Subject: [PATCH 231/510] Update docs/ru/operations/caches.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/caches.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/caches.md b/docs/ru/operations/caches.md index cf7118eb1f3..9a8092c3c39 100644 --- a/docs/ru/operations/caches.md +++ b/docs/ru/operations/caches.md @@ -24,6 +24,6 @@ toc_title: Кеши - Кеш страницы ОС. -Чтобы удалить кеш, используйте выражения типа [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md). +Чтобы очистить кеш, используйте выражение [SYSTEM DROP ... CACHE](../sql-reference/statements/system.md). [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/caches/) From 500d3561cf9433edb2ee58542b3ebd75cdd23b33 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:41:47 +0300 Subject: [PATCH 232/510] Update docs/ru/sql-reference/table-functions/file.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index ca1ac8b29db..b0b31e76098 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -7,7 +7,7 @@ toc_title: file Создаёт таблицу из файла. Данная табличная функция похожа на табличные функции [url](../../sql-reference/table-functions/url.md) and [hdfs](../../sql-reference/table-functions/hdfs.md). -Функция `file` может использоваться в запросах `SELECT` и `INSERT` движка таблиц [File](../../engines/table-engines/special/file.md). +Функция `file` может использоваться в запросах `SELECT` и `INSERT` при работе с движком таблиц [File](../../engines/table-engines/special/file.md). **Синтаксис** From 4b4c37a7198a612367fa615b99db6d78c7978fce Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:42:38 +0300 Subject: [PATCH 233/510] Update docs/en/sql-reference/table-functions/file.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/file.md b/docs/en/sql-reference/table-functions/file.md index e4ea59aface..da0999e66eb 100644 --- a/docs/en/sql-reference/table-functions/file.md +++ b/docs/en/sql-reference/table-functions/file.md @@ -74,7 +74,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U ## Globs in Path {#globs-in-path} -Multiple path components can have globs. For being processed file should exists and matches to the whole path pattern (not only suffix or prefix). +Multiple path components can have globs. For being processed file must exist and match to the whole path pattern (not only suffix or prefix). - `*` — Substitutes any number of any characters except `/` including empty string. - `?` — Substitutes any single character. From d79ea4f38361046d4916b780f59ee893410af32e Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:44:03 +0300 Subject: [PATCH 234/510] Update docs/ru/sql-reference/table-functions/file.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index b0b31e76098..a36fc1411b2 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -74,7 +74,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U ## Шаблоны в компонентах пути {#globs-in-path} -Шаблоны могут содержаться в нескольких компонентах пути. Обрабатываются только существующие файлы, название которых целиком удовлетворяет шаблону (не только суффиксом или префиксом). +При описании пути к файлу могут использоваться шаблоны. Обрабатываются только те файлы, у которых путь и название соответствуют шаблону полностью (а не только префикс или суффикс). - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. From 44bd6670da8511e7bae3a64d3a966c7a481ca291 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:44:40 +0300 Subject: [PATCH 235/510] Update docs/ru/sql-reference/table-functions/file.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/file.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index a36fc1411b2..3cb7043929a 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -85,7 +85,7 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U **Пример** -Предположим у нас есть несколько файлов со следующими относительными путями: +Предположим, у нас есть несколько файлов со следующими относительными путями: - 'some_dir/some_file_1' - 'some_dir/some_file_2' From fe4419b220f802577b69be2ffb2c42acbe7ad037 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:45:25 +0300 Subject: [PATCH 236/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index 435fb5bb6d7..a3ddd4cfe68 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -5,7 +5,7 @@ toc_title: remote # remote, remoteSecure {#remote-remotesecure} -Позволяет обратиться к удалённым серверам без создания таблицы типа [Distributed](../../engines/table-engines/special/distributed.md). Функция `remoteSecure` такая же, как и `remote`, но с защищенным соединением. +Позволяет обратиться к удалённым серверам без создания таблицы типа [Distributed](../../engines/table-engines/special/distributed.md). Функция `remoteSecure` работает аналогично `remote`, но использует защищенное соединение. Обе функции могут быть использованы в запросах типа `SELECT` и `INSERT`. From ee168507dbef224012d8d367181ad3591c40595b Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:45:48 +0300 Subject: [PATCH 237/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index a3ddd4cfe68..a48a176d75f 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -7,7 +7,7 @@ toc_title: remote Позволяет обратиться к удалённым серверам без создания таблицы типа [Distributed](../../engines/table-engines/special/distributed.md). Функция `remoteSecure` работает аналогично `remote`, но использует защищенное соединение. -Обе функции могут быть использованы в запросах типа `SELECT` и `INSERT`. +Обе функции могут использоваться в запросах `SELECT` и `INSERT`. **Синтаксис** From 95c07b19ecc933e38d82958a59996f51b4ad9e39 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:46:13 +0300 Subject: [PATCH 238/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index a48a176d75f..dd04f8458da 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -20,7 +20,7 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) **Параметры** -- `addresses_expr` — выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера — это `хост:порт`, или только `хост`. +- `addresses_expr` — выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера — это `host:port` или только `host`. Хост может быть указан в виде имени сервера, или в виде IPv4 или IPv6 адреса. IPv6 адрес указывается в квадратных скобках. From ce6263220a44ce5260a2ff28c55092ffd715c3ef Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:46:35 +0300 Subject: [PATCH 239/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index dd04f8458da..4dbb5863cdf 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -22,7 +22,7 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) - `addresses_expr` — выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера — это `host:port` или только `host`. - Хост может быть указан в виде имени сервера, или в виде IPv4 или IPv6 адреса. IPv6 адрес указывается в квадратных скобках. + Вместо параметра `host' может быть указано имя сервера или его адрес в формате IPv4 или IPv6. IPv6 адрес указывается в квадратных скобках. Порт — TCP-порт удалённого сервера. Если порт не указан, используется [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) из конфигурационного файла сервера, к которому обратились через функцию `remote` (по умолчанию - 9000), и [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure), к которому обратились через функцию `remoteSecure` (по умолчанию — 9440). From ef263d03569cdf3702b97215cef4ea1810404ff2 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:46:56 +0300 Subject: [PATCH 240/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index 4dbb5863cdf..48eb8d0c254 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -24,7 +24,7 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) Вместо параметра `host' может быть указано имя сервера или его адрес в формате IPv4 или IPv6. IPv6 адрес указывается в квадратных скобках. - Порт — TCP-порт удалённого сервера. Если порт не указан, используется [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) из конфигурационного файла сервера, к которому обратились через функцию `remote` (по умолчанию - 9000), и [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure), к которому обратились через функцию `remoteSecure` (по умолчанию — 9440). + `port` — TCP-порт удалённого сервера. Если порт не указан, используется [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) из конфигурационного файла сервера, к которому обратились через функцию `remote` (по умолчанию - 9000), и [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure), к которому обратились через функцию `remoteSecure` (по умолчанию — 9440). С IPv6-адресом обязательно нужно указывать порт. From 3ba3faa156274d0ded2875c7d922b6a38ca21462 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:47:28 +0300 Subject: [PATCH 241/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index 48eb8d0c254..05d5938c40d 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -42,7 +42,7 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) **Использование** -Использование табличной функции `remote` менее оптимально, чем создание таблицы типа `Distributed`, так как в этом случае соединения с серверами устанавливаются заново при каждом запросе. В случае задания имён хостов делается резолвинг имён, а также не ведётся подсчёт ошибок при работе с разными репликами. При обработке большого количества запросов всегда создавайте таблицу типа `Distributed` заранее, не используйте табличную функцию `remote`. +Использование табличной функции `remote` менее оптимально, чем создание таблицы типа `Distributed`, так как в этом случае соединения с серверами устанавливаются заново при каждом запросе. Если указываются имена серверов, то приходится также выполнять поиск сервера по имени. Кроме того, не ведётся сквозной подсчёт ошибок при работе с разными репликами. При обработке большого количества запросов всегда создавайте таблицу типа `Distributed`, использовать табличную функцию `remote` в таких случаях не рекомендуется. Табличная функция `remote` может быть полезна в следующих случаях: From d10c9f1bd37128a37a0b9e6c416f6b6e5d8d3f80 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:47:50 +0300 Subject: [PATCH 242/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index 05d5938c40d..a174bd12e94 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -46,8 +46,8 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) Табличная функция `remote` может быть полезна в следующих случаях: -- Обращение на конкретный сервер в целях сравнения данных, отладки и тестирования. -- Запросы между разными кластерами ClickHouse в целях исследований. +- Обращение на конкретный сервер для сравнения данных, отладки и тестирования. +- Запросы между разными кластерами ClickHouse для исследований. - Нечастые распределённые запросы, задаваемые вручную. - Распределённые запросы, где набор серверов определяется каждый раз заново. From 234ec940beba4aec7ae435d205acf7cfc232002e Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:48:13 +0300 Subject: [PATCH 243/510] Update docs/ru/sql-reference/table-functions/remote.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/remote.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index a174bd12e94..0cb32861d1e 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -82,7 +82,7 @@ example01-{01..02}-1 При наличии нескольких пар фигурных скобок генерируется прямое произведение соответствующих множеств. -Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае соответствующие множества адресов понимаются как реплики — запрос будет отправлен на первую живую реплику. При этом реплики перебираются в порядке, согласно текущей настройке [load_balancing](../../operations/settings/settings.md#settings-load_balancing). В этом примере указано два шарда, в каждом из которых имеется две реплики: +Адреса или их фрагменты в фигурных скобках можно указать через символ \|. В этом случае соответствующие множества адресов понимаются как реплики — запрос будет отправлен на первую живую реплику. При этом реплики перебираются в порядке, согласно текущей настройке [load_balancing](../../operations/settings/settings.md#settings-load_balancing). В этом примере указаны два шарда, в каждом из которых имеются две реплики: ``` text example01-{01..02}-{1|2} From 3222a9aecd0c47ec232dc2277edbaec192604431 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:48:32 +0300 Subject: [PATCH 244/510] Update docs/ru/sql-reference/table-functions/url.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/url.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index afb4a23b88e..ef97e269fbb 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -7,7 +7,7 @@ toc_title: url Функция `url` создает таблицу с помощью адреса `URL`, формата данных и структуры таблицы. -Функция `url` может быть использована в запросах `SELECT` и `INSERT` в таблицах движка [URL](../../engines/table-engines/special/url.md). +Функция `url` может быть использована в запросах `SELECT` и `INSERT` с таблицами на движке [URL](../../engines/table-engines/special/url.md). **Синтаксис** From 31e78ef9983d4e8de703f84fe3be069feb0f4297 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:48:54 +0300 Subject: [PATCH 245/510] Update docs/ru/sql-reference/table-functions/url.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/url.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index ef97e269fbb..8a4f6fe7d98 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -27,7 +27,7 @@ url(URL, format, structure) **Примеры** -Получение первых 3 строк таблицы, содержащей столбцы типа `String` и [UInt32](../../sql-reference/data-types/int-uint.md), с HTTP-сервера в формате [CSV](../../interfaces/formats.md/#csv). +Получение с HTTP-сервера первых 3 строк таблицы с данными в формате [CSV](../../interfaces/formats.md/#csv), содержащей столбцы типа [String](../../sql-reference/data-types/string.md) и [UInt32](../../sql-reference/data-types/int-uint.md). ``` sql SELECT * FROM url('http://127.0.0.1:12345/', CSV, 'column1 String, column2 UInt32') LIMIT 3; From 5eb5180ef8de2b0d7b0751b9bf765c3abe9c0ba0 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 18 Feb 2021 21:49:33 +0300 Subject: [PATCH 246/510] Update docs/ru/sql-reference/table-functions/url.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/table-functions/url.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index 8a4f6fe7d98..fe80f466f54 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -5,7 +5,7 @@ toc_title: url # url {#url} -Функция `url` создает таблицу с помощью адреса `URL`, формата данных и структуры таблицы. +Функция `url` берет данные по указанному адресу `URL` и создает из них таблицу указанной структуры со столбцами указанного формата. Функция `url` может быть использована в запросах `SELECT` и `INSERT` с таблицами на движке [URL](../../engines/table-engines/special/url.md). From f4b0b1110cb77c6901243cc1120615d9735a2da3 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Feb 2021 23:53:40 +0300 Subject: [PATCH 247/510] Fix test. --- tests/queries/0_stateless/01272_totals_and_filter_bug.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01272_totals_and_filter_bug.reference b/tests/queries/0_stateless/01272_totals_and_filter_bug.reference index 0db840561fd..5b407738cb8 100644 --- a/tests/queries/0_stateless/01272_totals_and_filter_bug.reference +++ b/tests/queries/0_stateless/01272_totals_and_filter_bug.reference @@ -1,6 +1,6 @@ 1 1 -0 2 +0 1 - test1 10 0 From 6e9bf682179229b4ae3d7f97ec3ab5c83229704b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 18 Feb 2021 23:54:42 +0300 Subject: [PATCH 248/510] Fix typo --- src/Processors/QueryPlan/Optimizations/filterPushDown.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 4d01235e2fc..1b84fee4857 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -143,7 +143,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 HAING expression with current filer. - /// Alos, we can push down part of HAVING which depend only on aggregation keys. + /// Also, we can push down part of HAVING which depend only on aggregation keys. if (totals_having->getActions()) return 0; From ee98b2a472aa05d28d36f859eefff0d359b45910 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 00:03:16 +0300 Subject: [PATCH 249/510] Better list requests --- src/Coordination/NuKeeperStorage.cpp | 37 ++++++++++++++-------------- src/Coordination/NuKeeperStorage.h | 8 +++--- 2 files changed, 24 insertions(+), 21 deletions(-) diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index 631f975cddc..fa57b8141a7 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -25,10 +25,10 @@ static String parentPath(const String & path) return "/"; } -static String baseName(const String & path) +static std::string_view getBaseNameView(const String & path) { - auto rslash_pos = path.rfind('/'); - return path.substr(rslash_pos + 1); + size_t basename_start = path.rfind('/'); + return std::string_view{&path[basename_start + 1], path.length() - basename_start - 1}; } static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type) @@ -167,14 +167,17 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest /// Increment sequential number even if node is not sequential ++it->second.seq_num; - response.path_created = path_created; - container.emplace(path_created, std::move(created_node)); + + auto [child_itr, created] = container.emplace(path_created, std::move(created_node)); + + auto child_path_view = getBaseNameView(child_itr->first); + it->second.children.insert(child_path_view); if (request.is_ephemeral) ephemerals[session_id].emplace(path_created); - undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first] + undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first, child_path_view] { container.erase(path_created); if (is_ephemeral) @@ -183,6 +186,7 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest --undo_parent.stat.cversion; --undo_parent.stat.numChildren; --undo_parent.seq_num; + undo_parent.children.erase(child_path_view); }; ++it->second.stat.cversion; @@ -250,21 +254,25 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest if (prev_node.is_ephemeral) ephemerals[session_id].erase(request.path); - container.erase(it); + auto child_basename_view = getBaseNameView(it->first); auto & parent = container.at(parentPath(request.path)); --parent.stat.numChildren; ++parent.stat.cversion; + parent.children.erase(child_basename_view); response.error = Coordination::Error::ZOK; + container.erase(it); + undo = [prev_node, &container, &ephemerals, session_id, path = request.path] { if (prev_node.is_ephemeral) ephemerals[session_id].emplace(path); - container.emplace(path, prev_node); + auto [itr, inserted] = container.emplace(path, prev_node); auto & undo_parent = container.at(parentPath(path)); ++undo_parent.stat.numChildren; --undo_parent.stat.cversion; + undo_parent.children.insert(getBaseNameView(itr->first)); }; } @@ -370,17 +378,10 @@ struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest if (path_prefix.empty()) throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR); - if (path_prefix.back() != '/') - path_prefix += '/'; + for (const auto & name : it->second.children) + response.names.emplace_back(name); - /// Fairly inefficient. - for (auto child_it = container.upper_bound(path_prefix); - child_it != container.end() && startsWith(child_it->first, path_prefix); - ++child_it) - { - if (parentPath(child_it->first) == request.path) - response.names.emplace_back(baseName(child_it->first)); - } + std::sort(response.names.begin(), response.names.end()); response.stat = it->second.stat; response.error = Coordination::Error::ZOK; diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index 20ab1982b4e..bd1fc087d09 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -16,6 +16,7 @@ using namespace DB; struct NuKeeperStorageRequest; using NuKeeperStorageRequestPtr = std::shared_ptr; using ResponseCallback = std::function; +using ChildrenRefSet = std::unordered_set; class NuKeeperStorage { @@ -30,6 +31,7 @@ public: bool is_sequental = false; Coordination::Stat stat{}; int32_t seq_num = 0; + ChildrenRefSet children; }; struct ResponseForSession @@ -48,9 +50,9 @@ public: using RequestsForSessions = std::vector; - using Container = std::map; - using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; + using Container = std::unordered_map; + using Ephemerals = std::unordered_map>; + using SessionAndWatcher = std::unordered_map>; using SessionAndTimeout = std::unordered_map; using SessionIDs = std::vector; From 839d6f7072d6de6b71cc497027ca40715968535e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 00:09:41 +0300 Subject: [PATCH 250/510] Revert "Better list requests" This reverts commit ee98b2a472aa05d28d36f859eefff0d359b45910. --- src/Coordination/NuKeeperStorage.cpp | 37 ++++++++++++++-------------- src/Coordination/NuKeeperStorage.h | 8 +++--- 2 files changed, 21 insertions(+), 24 deletions(-) diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index fa57b8141a7..631f975cddc 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -25,10 +25,10 @@ static String parentPath(const String & path) return "/"; } -static std::string_view getBaseNameView(const String & path) +static String baseName(const String & path) { - size_t basename_start = path.rfind('/'); - return std::string_view{&path[basename_start + 1], path.length() - basename_start - 1}; + auto rslash_pos = path.rfind('/'); + return path.substr(rslash_pos + 1); } static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type) @@ -167,17 +167,14 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest /// Increment sequential number even if node is not sequential ++it->second.seq_num; + response.path_created = path_created; - - auto [child_itr, created] = container.emplace(path_created, std::move(created_node)); - - auto child_path_view = getBaseNameView(child_itr->first); - it->second.children.insert(child_path_view); + container.emplace(path_created, std::move(created_node)); if (request.is_ephemeral) ephemerals[session_id].emplace(path_created); - undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first, child_path_view] + undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first] { container.erase(path_created); if (is_ephemeral) @@ -186,7 +183,6 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest --undo_parent.stat.cversion; --undo_parent.stat.numChildren; --undo_parent.seq_num; - undo_parent.children.erase(child_path_view); }; ++it->second.stat.cversion; @@ -254,25 +250,21 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest if (prev_node.is_ephemeral) ephemerals[session_id].erase(request.path); - auto child_basename_view = getBaseNameView(it->first); + container.erase(it); auto & parent = container.at(parentPath(request.path)); --parent.stat.numChildren; ++parent.stat.cversion; - parent.children.erase(child_basename_view); response.error = Coordination::Error::ZOK; - container.erase(it); - undo = [prev_node, &container, &ephemerals, session_id, path = request.path] { if (prev_node.is_ephemeral) ephemerals[session_id].emplace(path); - auto [itr, inserted] = container.emplace(path, prev_node); + container.emplace(path, prev_node); auto & undo_parent = container.at(parentPath(path)); ++undo_parent.stat.numChildren; --undo_parent.stat.cversion; - undo_parent.children.insert(getBaseNameView(itr->first)); }; } @@ -378,10 +370,17 @@ struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest if (path_prefix.empty()) throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR); - for (const auto & name : it->second.children) - response.names.emplace_back(name); + if (path_prefix.back() != '/') + path_prefix += '/'; - std::sort(response.names.begin(), response.names.end()); + /// Fairly inefficient. + for (auto child_it = container.upper_bound(path_prefix); + child_it != container.end() && startsWith(child_it->first, path_prefix); + ++child_it) + { + if (parentPath(child_it->first) == request.path) + response.names.emplace_back(baseName(child_it->first)); + } response.stat = it->second.stat; response.error = Coordination::Error::ZOK; diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index bd1fc087d09..20ab1982b4e 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -16,7 +16,6 @@ using namespace DB; struct NuKeeperStorageRequest; using NuKeeperStorageRequestPtr = std::shared_ptr; using ResponseCallback = std::function; -using ChildrenRefSet = std::unordered_set; class NuKeeperStorage { @@ -31,7 +30,6 @@ public: bool is_sequental = false; Coordination::Stat stat{}; int32_t seq_num = 0; - ChildrenRefSet children; }; struct ResponseForSession @@ -50,9 +48,9 @@ public: using RequestsForSessions = std::vector; - using Container = std::unordered_map; - using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; + using Container = std::map; + using Ephemerals = std::unordered_map>; + using SessionAndWatcher = std::unordered_map>; using SessionAndTimeout = std::unordered_map; using SessionIDs = std::vector; From 879d9206c1a30eb04a4444cc5a30cf1455089176 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Fri, 19 Feb 2021 02:47:28 +0400 Subject: [PATCH 251/510] Add tests for row-level security with prewhere --- tests/integration/test_row_policy/test.py | 33 +++++++++++++++++++ .../prewhere_with_row_level_filter.xml | 16 +++++++++ 2 files changed, 49 insertions(+) create mode 100644 tests/performance/prewhere_with_row_level_filter.xml diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 8919aeab0c5..c11e1b1e21c 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -107,6 +107,7 @@ def test_cannot_trick_row_policy_with_keyword_with(): assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1]]) assert node.query("WITH 0 AS a SELECT a FROM mydb.filtered_table1") == TSV([[0], [0]]) assert node.query("WITH 0 AS a SELECT b FROM mydb.filtered_table1") == TSV([[0], [1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a IN(0, 1) WHERE b IN(0, 1)") == TSV([[0], [1]]) def test_policy_from_users_xml_affects_only_user_assigned(): @@ -121,6 +122,38 @@ def test_policy_from_users_xml_affects_only_user_assigned(): assert node.query("SELECT * FROM mydb.local", user="another") == TSV([[1, 0], [1, 1]]) +def test_with_prewhere(): + copy_policy_xml('normal_filters.xml') + assert node.query("SELECT * FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 5, 2, 1]]) + assert node.query("SELECT a FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4]]) + assert node.query("SELECT a, b FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 5]]) + assert node.query("SELECT b, c FROM mydb.filtered_table2 WHERE a > 1") == TSV([[5, 2]]) + assert node.query("SELECT d FROM mydb.filtered_table2 WHERE a > 1") == TSV([[1]]) + + assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 5, 2, 1]]) + assert node.query("SELECT a FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4]]) + assert node.query("SELECT a, b FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 5]]) + assert node.query("SELECT b, c FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[5, 2]]) + assert node.query("SELECT d FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[1]]) + + assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1, 2, 3, 4]]) + assert node.query("SELECT a FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1]]) + assert node.query("SELECT b FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[2]]) + assert node.query("SELECT a, b FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1, 2]]) + assert node.query("SELECT a, c FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1, 3]]) + assert node.query("SELECT b, d FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[2, 4]]) + assert node.query("SELECT c, d FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[3, 4]]) + + +def test_with_throwif_in_prewhere(): + copy_policy_xml('no_filters.xml') + assert 'expected' in node.query_and_get_error("SELECT throwIf(a = 0, 'expected') FROM mydb.filtered_table2 PREWHERE b < 10") + + copy_policy_xml('normal_filters.xml') + assert node.query("SELECT throwIf(a = 0, 'pwned') FROM mydb.filtered_table2 PREWHERE b < 10") == TSV([ + [4, 5, 2, 1], [1, 2, 3, 4]]) + + def test_change_of_users_xml_changes_row_policies(): copy_policy_xml('normal_filters.xml') assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) diff --git a/tests/performance/prewhere_with_row_level_filter.xml b/tests/performance/prewhere_with_row_level_filter.xml new file mode 100644 index 00000000000..d73690ca811 --- /dev/null +++ b/tests/performance/prewhere_with_row_level_filter.xml @@ -0,0 +1,16 @@ + + DROP TABLE IF EXISTS test_prl; + CREATE TABLE test_prl (n UInt64) ENGINE MergeTree ORDER BY n; + CREATE ROW POLICY OR REPLACE test_prl_policy ON test_prl AS PERMISSIVE FOR SELECT USING n % 7 TO ALL; + + INSERT INTO test_prl SELECT number FROM numbers(50000000); + + SELECT * FROM test_prl; + SELECT * FROM test_prl WHERE n % 3 AND n % 5; + SELECT * FROM test_prl PREWHERE n % 3 AND n % 5; + SELECT * FROM test_prl PREWHERE n % 3 WHERE n % 5; + SELECT * FROM test_prl PREWHERE n % 5 WHERE n % 3; + + DROP ROW POLICY IF EXISTS test_prl_policy ON test_prl; + DROP TABLE IF EXISTS test_prl; + From 88a6d4e206c362dcafc0d8751cb2a6a450178ee8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 10:05:52 +0300 Subject: [PATCH 252/510] Revert "Revert "Better list requests"" This reverts commit 839d6f7072d6de6b71cc497027ca40715968535e. --- src/Coordination/NuKeeperStorage.cpp | 37 ++++++++++++++-------------- src/Coordination/NuKeeperStorage.h | 8 +++--- 2 files changed, 24 insertions(+), 21 deletions(-) diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index 631f975cddc..fa57b8141a7 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -25,10 +25,10 @@ static String parentPath(const String & path) return "/"; } -static String baseName(const String & path) +static std::string_view getBaseNameView(const String & path) { - auto rslash_pos = path.rfind('/'); - return path.substr(rslash_pos + 1); + size_t basename_start = path.rfind('/'); + return std::string_view{&path[basename_start + 1], path.length() - basename_start - 1}; } static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type) @@ -167,14 +167,17 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest /// Increment sequential number even if node is not sequential ++it->second.seq_num; - response.path_created = path_created; - container.emplace(path_created, std::move(created_node)); + + auto [child_itr, created] = container.emplace(path_created, std::move(created_node)); + + auto child_path_view = getBaseNameView(child_itr->first); + it->second.children.insert(child_path_view); if (request.is_ephemeral) ephemerals[session_id].emplace(path_created); - undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first] + undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first, child_path_view] { container.erase(path_created); if (is_ephemeral) @@ -183,6 +186,7 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest --undo_parent.stat.cversion; --undo_parent.stat.numChildren; --undo_parent.seq_num; + undo_parent.children.erase(child_path_view); }; ++it->second.stat.cversion; @@ -250,21 +254,25 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest if (prev_node.is_ephemeral) ephemerals[session_id].erase(request.path); - container.erase(it); + auto child_basename_view = getBaseNameView(it->first); auto & parent = container.at(parentPath(request.path)); --parent.stat.numChildren; ++parent.stat.cversion; + parent.children.erase(child_basename_view); response.error = Coordination::Error::ZOK; + container.erase(it); + undo = [prev_node, &container, &ephemerals, session_id, path = request.path] { if (prev_node.is_ephemeral) ephemerals[session_id].emplace(path); - container.emplace(path, prev_node); + auto [itr, inserted] = container.emplace(path, prev_node); auto & undo_parent = container.at(parentPath(path)); ++undo_parent.stat.numChildren; --undo_parent.stat.cversion; + undo_parent.children.insert(getBaseNameView(itr->first)); }; } @@ -370,17 +378,10 @@ struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest if (path_prefix.empty()) throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR); - if (path_prefix.back() != '/') - path_prefix += '/'; + for (const auto & name : it->second.children) + response.names.emplace_back(name); - /// Fairly inefficient. - for (auto child_it = container.upper_bound(path_prefix); - child_it != container.end() && startsWith(child_it->first, path_prefix); - ++child_it) - { - if (parentPath(child_it->first) == request.path) - response.names.emplace_back(baseName(child_it->first)); - } + std::sort(response.names.begin(), response.names.end()); response.stat = it->second.stat; response.error = Coordination::Error::ZOK; diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index 20ab1982b4e..bd1fc087d09 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -16,6 +16,7 @@ using namespace DB; struct NuKeeperStorageRequest; using NuKeeperStorageRequestPtr = std::shared_ptr; using ResponseCallback = std::function; +using ChildrenRefSet = std::unordered_set; class NuKeeperStorage { @@ -30,6 +31,7 @@ public: bool is_sequental = false; Coordination::Stat stat{}; int32_t seq_num = 0; + ChildrenRefSet children; }; struct ResponseForSession @@ -48,9 +50,9 @@ public: using RequestsForSessions = std::vector; - using Container = std::map; - using Ephemerals = std::unordered_map>; - using SessionAndWatcher = std::unordered_map>; + using Container = std::unordered_map; + using Ephemerals = std::unordered_map>; + using SessionAndWatcher = std::unordered_map>; using SessionAndTimeout = std::unordered_map; using SessionIDs = std::vector; From b72b13bab05fc6f90396f335471023673c98c31f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 10:25:55 +0300 Subject: [PATCH 253/510] Better list performance --- src/Coordination/NuKeeperStorage.cpp | 27 +++++++++++++-------------- src/Coordination/NuKeeperStorage.h | 4 ++-- 2 files changed, 15 insertions(+), 16 deletions(-) diff --git a/src/Coordination/NuKeeperStorage.cpp b/src/Coordination/NuKeeperStorage.cpp index fa57b8141a7..bb433474dc9 100644 --- a/src/Coordination/NuKeeperStorage.cpp +++ b/src/Coordination/NuKeeperStorage.cpp @@ -25,10 +25,10 @@ static String parentPath(const String & path) return "/"; } -static std::string_view getBaseNameView(const String & path) +static std::string getBaseName(const String & path) { size_t basename_start = path.rfind('/'); - return std::string_view{&path[basename_start + 1], path.length() - basename_start - 1}; + return std::string{&path[basename_start + 1], path.length() - basename_start - 1}; } static NuKeeperStorage::ResponsesForSessions processWatchesImpl(const String & path, NuKeeperStorage::Watches & watches, NuKeeperStorage::Watches & list_watches, Coordination::Event event_type) @@ -169,15 +169,15 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest ++it->second.seq_num; response.path_created = path_created; - auto [child_itr, created] = container.emplace(path_created, std::move(created_node)); + container.emplace(path_created, std::move(created_node)); - auto child_path_view = getBaseNameView(child_itr->first); - it->second.children.insert(child_path_view); + auto child_path = getBaseName(path_created); + it->second.children.insert(child_path); if (request.is_ephemeral) ephemerals[session_id].emplace(path_created); - undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first, child_path_view] + undo = [&container, &ephemerals, session_id, path_created, is_ephemeral = request.is_ephemeral, parent_path = it->first, child_path] { container.erase(path_created); if (is_ephemeral) @@ -186,7 +186,7 @@ struct NuKeeperStorageCreateRequest final : public NuKeeperStorageRequest --undo_parent.stat.cversion; --undo_parent.stat.numChildren; --undo_parent.seq_num; - undo_parent.children.erase(child_path_view); + undo_parent.children.erase(child_path); }; ++it->second.stat.cversion; @@ -254,25 +254,25 @@ struct NuKeeperStorageRemoveRequest final : public NuKeeperStorageRequest if (prev_node.is_ephemeral) ephemerals[session_id].erase(request.path); - auto child_basename_view = getBaseNameView(it->first); + auto child_basename = getBaseName(it->first); auto & parent = container.at(parentPath(request.path)); --parent.stat.numChildren; ++parent.stat.cversion; - parent.children.erase(child_basename_view); + parent.children.erase(child_basename); response.error = Coordination::Error::ZOK; container.erase(it); - undo = [prev_node, &container, &ephemerals, session_id, path = request.path] + undo = [prev_node, &container, &ephemerals, session_id, path = request.path, child_basename] { if (prev_node.is_ephemeral) ephemerals[session_id].emplace(path); - auto [itr, inserted] = container.emplace(path, prev_node); + container.emplace(path, prev_node); auto & undo_parent = container.at(parentPath(path)); ++undo_parent.stat.numChildren; --undo_parent.stat.cversion; - undo_parent.children.insert(getBaseNameView(itr->first)); + undo_parent.children.insert(child_basename); }; } @@ -378,8 +378,7 @@ struct NuKeeperStorageListRequest final : public NuKeeperStorageRequest if (path_prefix.empty()) throw DB::Exception("Logical error: path cannot be empty", ErrorCodes::LOGICAL_ERROR); - for (const auto & name : it->second.children) - response.names.emplace_back(name); + response.names.insert(response.names.end(), it->second.children.begin(), it->second.children.end()); std::sort(response.names.begin(), response.names.end()); diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index bd1fc087d09..299fad4eea0 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -16,7 +16,7 @@ using namespace DB; struct NuKeeperStorageRequest; using NuKeeperStorageRequestPtr = std::shared_ptr; using ResponseCallback = std::function; -using ChildrenRefSet = std::unordered_set; +using ChildrenSet = std::unordered_set; class NuKeeperStorage { @@ -31,7 +31,7 @@ public: bool is_sequental = false; Coordination::Stat stat{}; int32_t seq_num = 0; - ChildrenRefSet children; + ChildrenSet children; }; struct ResponseForSession From 86a74ca6b5cd3618d574431d0c94a44ebac93baf Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 12:24:50 +0300 Subject: [PATCH 254/510] Fix size deserialization --- src/Coordination/NuKeeperStorage.h | 2 +- src/Coordination/NuKeeperStorageSerializer.cpp | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Coordination/NuKeeperStorage.h b/src/Coordination/NuKeeperStorage.h index 299fad4eea0..1a2e6202bf0 100644 --- a/src/Coordination/NuKeeperStorage.h +++ b/src/Coordination/NuKeeperStorage.h @@ -31,7 +31,7 @@ public: bool is_sequental = false; Coordination::Stat stat{}; int32_t seq_num = 0; - ChildrenSet children; + ChildrenSet children{}; }; struct ResponseForSession diff --git a/src/Coordination/NuKeeperStorageSerializer.cpp b/src/Coordination/NuKeeperStorageSerializer.cpp index 298df45cde0..c29d0d1f1fa 100644 --- a/src/Coordination/NuKeeperStorageSerializer.cpp +++ b/src/Coordination/NuKeeperStorageSerializer.cpp @@ -59,13 +59,16 @@ void NuKeeperStorageSerializer::deserialize(NuKeeperStorage & storage, ReadBuffe size_t container_size; Coordination::read(container_size, in); - while (storage.container.size() < container_size) + + size_t current_size = 0; + while (current_size < container_size) { std::string path; Coordination::read(path, in); NuKeeperStorage::Node node; readNode(node, in); storage.container[path] = node; + current_size++; } size_t ephemerals_size; Coordination::read(ephemerals_size, in); From 6c9322bb2e779067d005879592157b5dba5074ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 12:57:39 +0300 Subject: [PATCH 255/510] Sane constant while reading requests --- src/Server/NuKeeperTCPHandler.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index e855e2c68f7..f25ca4a42ce 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -342,6 +342,7 @@ void NuKeeperTCPHandler::runImpl() PollResult result = poll_wrapper->poll(session_timeout); if (result.has_requests && !close_received) { + size_t requests_read = 0; do { auto [received_op, received_xid] = receiveRequest(); @@ -358,6 +359,10 @@ void NuKeeperTCPHandler::runImpl() LOG_TRACE(log, "Received heartbeat for session #{}", session_id); session_stopwatch.restart(); } + + if (requests_read > 50) + break; + requests_read++; } while (in->available()); } From 3d954c43142b28c0643b504a7f4d9333142b3fe0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 14:06:17 +0300 Subject: [PATCH 256/510] Better request/response logic --- src/Server/NuKeeperTCPHandler.cpp | 70 ++++++++++++++----------------- 1 file changed, 32 insertions(+), 38 deletions(-) diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index f25ca4a42ce..081821504d3 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -40,7 +40,7 @@ namespace ErrorCodes struct PollResult { - size_t ready_responses_count{0}; + bool has_response{false}; bool has_requests{false}; bool error{false}; }; @@ -92,8 +92,22 @@ struct SocketInterruptablePollWrapper return pipe.fds_rw[1]; } - PollResult poll(Poco::Timespan remaining_time) + PollResult poll(Poco::Timespan remaining_time, const std::shared_ptr & in) { + PollResult result{}; + if (response_in.available() != 0) + { + UInt8 dummy; + readIntBinary(dummy, response_in); + result.has_response = true; + } + + if (in->available() != 0) + result.has_requests = true; + + if (result.has_response) + return result; + std::array outputs = {-1, -1}; #if defined(POCO_HAVE_FD_EPOLL) int rc; @@ -148,7 +162,6 @@ struct SocketInterruptablePollWrapper outputs[1] = pipe.fds_rw[0]; #endif - PollResult result{}; if (rc < 0) { result.error = true; @@ -169,16 +182,8 @@ struct SocketInterruptablePollWrapper else { UInt8 dummy; - do - { - /// All ready responses stored in responses queue, - /// but we have to count amount of ready responses in pipe - /// and process them only. Otherwise states of response_in - /// and response queue will be inconsistent and race condition is possible. - readIntBinary(dummy, response_in); - result.ready_responses_count++; - } - while (response_in.available()); + readIntBinary(dummy, response_in); + result.has_response = true; } } } @@ -339,42 +344,32 @@ void NuKeeperTCPHandler::runImpl() { using namespace std::chrono_literals; - PollResult result = poll_wrapper->poll(session_timeout); + PollResult result = poll_wrapper->poll(session_timeout, in); if (result.has_requests && !close_received) { - size_t requests_read = 0; - do + auto [received_op, received_xid] = receiveRequest(); + + if (received_op == Coordination::OpNum::Close) { - auto [received_op, received_xid] = receiveRequest(); - - if (received_op == Coordination::OpNum::Close) - { - LOG_DEBUG(log, "Received close event with xid {} for session id #{}", received_xid, session_id); - close_xid = received_xid; - close_received = true; - break; - } - else if (received_op == Coordination::OpNum::Heartbeat) - { - LOG_TRACE(log, "Received heartbeat for session #{}", session_id); - session_stopwatch.restart(); - } - - if (requests_read > 50) - break; - requests_read++; + LOG_DEBUG(log, "Received close event with xid {} for session id #{}", received_xid, session_id); + close_xid = received_xid; + close_received = true; + } + else if (received_op == Coordination::OpNum::Heartbeat) + { + LOG_TRACE(log, "Received heartbeat for session #{}", session_id); + session_stopwatch.restart(); } - while (in->available()); } /// Process exact amount of responses from pipe /// otherwise state of responses queue and signaling pipe /// became inconsistent and race condition is possible. - while (result.ready_responses_count != 0) + if (result.has_response) { Coordination::ZooKeeperResponsePtr response; if (!responses->tryPop(response)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "We must have at least {} ready responses, but queue is empty. It's a bug.", result.ready_responses_count); + throw Exception(ErrorCodes::LOGICAL_ERROR, "We must have ready response, but queue is empty. It's a bug."); if (response->xid == close_xid) { @@ -388,7 +383,6 @@ void NuKeeperTCPHandler::runImpl() nu_keeper_storage_dispatcher->finishSession(session_id); return; } - result.ready_responses_count--; } if (result.error) From 5c9420c0779c648db5a42ecbb8f6db43cb98a76d Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 15:56:51 +0300 Subject: [PATCH 257/510] More correct epoll usage --- src/Server/NuKeeperTCPHandler.cpp | 46 ++++++++++++++++--------------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index 081821504d3..92c7f4b968f 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -70,14 +70,14 @@ struct SocketInterruptablePollWrapper if (epollfd < 0) throwFromErrno("Cannot epoll_create", ErrorCodes::SYSTEM_ERROR); - socket_event.events = EPOLLIN | EPOLLERR; + socket_event.events = EPOLLIN | EPOLLERR | EPOLLPRI; socket_event.data.fd = sockfd; if (epoll_ctl(epollfd, EPOLL_CTL_ADD, sockfd, &socket_event) < 0) { ::close(epollfd); throwFromErrno("Cannot insert socket into epoll queue", ErrorCodes::SYSTEM_ERROR); } - pipe_event.events = EPOLLIN | EPOLLERR; + pipe_event.events = EPOLLIN | EPOLLERR | EPOLLPRI; pipe_event.data.fd = pipe.fds_rw[0]; if (epoll_ctl(epollfd, EPOLL_CTL_ADD, pipe.fds_rw[0], &pipe_event) < 0) { @@ -108,11 +108,12 @@ struct SocketInterruptablePollWrapper if (result.has_response) return result; - std::array outputs = {-1, -1}; + bool socket_ready = false; + bool fd_ready = false; #if defined(POCO_HAVE_FD_EPOLL) int rc; epoll_event evout[2]; - memset(evout, 0, sizeof(evout)); + evout[0].data.fd = evout[1].data.fd = -1; do { Poco::Timestamp start; @@ -129,10 +130,13 @@ struct SocketInterruptablePollWrapper } while (rc < 0 && errno == EINTR); - if (rc >= 1 && evout[0].events & EPOLLIN) - outputs[0] = evout[0].data.fd; - if (rc == 2 && evout[1].events & EPOLLIN) - outputs[1] = evout[1].data.fd; + for (int i = 0; i < rc; ++i) + { + if (evout[i].data.fd == sockfd) + socket_ready = true; + if (evout[i].data.fd == pipe.fds_rw[0]) + fd_ready = true; + } #else pollfd poll_buf[2]; poll_buf[0].fd = sockfd; @@ -156,10 +160,11 @@ struct SocketInterruptablePollWrapper } } while (rc < 0 && errno == POCO_EINTR); + if (rc >= 1 && poll_buf[0].revents & POLLIN) - outputs[0] = sockfd; + socket_ready = true; if (rc == 2 && poll_buf[1].revents & POLLIN) - outputs[1] = pipe.fds_rw[0]; + fd_ready = true; #endif if (rc < 0) @@ -173,19 +178,15 @@ struct SocketInterruptablePollWrapper } else { - for (auto fd : outputs) + if (socket_ready) { - if (fd != -1) - { - if (fd == sockfd) - result.has_requests = true; - else - { - UInt8 dummy; - readIntBinary(dummy, response_in); - result.has_response = true; - } - } + result.has_requests = true; + } + if (fd_ready) + { + UInt8 dummy; + readIntBinary(dummy, response_in); + result.has_response = true; } } return result; @@ -368,6 +369,7 @@ void NuKeeperTCPHandler::runImpl() if (result.has_response) { Coordination::ZooKeeperResponsePtr response; + if (!responses->tryPop(response)) throw Exception(ErrorCodes::LOGICAL_ERROR, "We must have ready response, but queue is empty. It's a bug."); From 068c9cfbf7a58dd7e624b3d1557ccdbaf227bf34 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 17:13:29 +0300 Subject: [PATCH 258/510] Fix logs level --- docker/test/fasttest/run.sh | 1 + src/Coordination/LoggerWrapper.h | 32 ++++++++++++++++++++++++-------- 2 files changed, 25 insertions(+), 8 deletions(-) diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 53a0de21d5b..0ace1cd39da 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -70,6 +70,7 @@ function start_server --path "$FASTTEST_DATA" --user_files_path "$FASTTEST_DATA/user_files" --top_level_domains_path "$FASTTEST_DATA/top_level_domains" + --test_keeper_server.log_storage_path "$FASTTEST_DATA/coordination" ) clickhouse-server "${opts[@]}" &>> "$FASTTEST_OUTPUT/server.log" & server_pid=$! diff --git a/src/Coordination/LoggerWrapper.h b/src/Coordination/LoggerWrapper.h index 755b72c06cc..25a1969d2e9 100644 --- a/src/Coordination/LoggerWrapper.h +++ b/src/Coordination/LoggerWrapper.h @@ -9,12 +9,26 @@ namespace DB class LoggerWrapper : public nuraft::logger { +private: + + static inline const std::unordered_map LEVELS = + { + {LogsLevel::trace, Poco::Message::Priority::PRIO_TRACE}, + {LogsLevel::debug, Poco::Message::Priority::PRIO_DEBUG}, + {LogsLevel::information, Poco::Message::PRIO_INFORMATION}, + {LogsLevel::warning, Poco::Message::PRIO_WARNING}, + {LogsLevel::error, Poco::Message::PRIO_ERROR}, + {LogsLevel::fatal, Poco::Message::PRIO_FATAL} + }; + static inline const int LEVEL_MAX = static_cast(LogsLevel::trace); + static inline const int LEVEL_MIN = static_cast(LogsLevel::none); + public: LoggerWrapper(const std::string & name, LogsLevel level_) : log(&Poco::Logger::get(name)) - , level(static_cast(level_)) + , level(level_) { - log->setLevel(level); + log->setLevel(static_cast(LEVELS.at(level))); } void put_details( @@ -24,24 +38,26 @@ public: size_t /* line_number */, const std::string & msg) override { - LOG_IMPL(log, static_cast(level_), static_cast(level_), msg); + LogsLevel db_level = static_cast(level_); + LOG_IMPL(log, db_level, LEVELS.at(db_level), msg); } void set_level(int level_) override { - level_ = std::min(6, std::max(1, level_)); - log->setLevel(level_); - level = level_; + level_ = std::min(LEVEL_MAX, std::max(LEVEL_MIN, level_)); + level = static_cast(level_); + log->setLevel(static_cast(LEVELS.at(level))); } int get_level() override { - return level; + LogsLevel lvl = level; + return static_cast(lvl); } private: Poco::Logger * log; - std::atomic level; + std::atomic level; }; } From 12d05c27922eb1010eaede6fdf891995240dc644 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 17:47:53 +0300 Subject: [PATCH 259/510] Better startup --- contrib/NuRaft | 2 +- src/Coordination/NuKeeperServer.cpp | 9 +-------- 2 files changed, 2 insertions(+), 9 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 7adf7ae33e7..c250d5ad58c 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 7adf7ae33e7d5c307342431b577c8ab1025ee793 +Subproject commit c250d5ad58c82e751264df40a94da682a2fc3519 diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 8556fa85231..c2917e3ab76 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -161,7 +161,7 @@ bool NuKeeperServer::isLeaderAlive() const nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */) { - if (type == nuraft::cb_func::Type::BecomeFresh || type == nuraft::cb_func::Type::BecomeLeader) + if ((type == nuraft::cb_func::InitialBatchCommited && isLeader()) || type == nuraft::cb_func::BecomeFresh) { std::unique_lock lock(initialized_mutex); initialized_flag = true; @@ -176,13 +176,6 @@ void NuKeeperServer::waitInit() int64_t timeout = coordination_settings->startup_timeout.totalMilliseconds(); if (!initialized_cv.wait_for(lock, std::chrono::milliseconds(timeout), [&] { return initialized_flag; })) throw Exception(ErrorCodes::RAFT_ERROR, "Failed to wait RAFT initialization"); - - /// TODO FIXME somehow - while (isLeader() && raft_instance->get_committed_log_idx() != raft_instance->get_last_log_idx()) - { - LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Loading from log store {}/{}", raft_instance->get_committed_log_idx(), raft_instance->get_last_log_idx()); - std::this_thread::sleep_for(std::chrono::milliseconds(100)); - } } std::unordered_set NuKeeperServer::getDeadSessions() From bfccfd9eece9a7e1f1c70755034a837d8cee8078 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Fri, 19 Feb 2021 19:32:35 +0400 Subject: [PATCH 260/510] Fix tests Add tests WIP: Failures are legit! --- .../test_row_policy/normal_filter2_table2.xml | 16 +++++ tests/integration/test_row_policy/test.py | 63 ++++++++++++++----- 2 files changed, 65 insertions(+), 14 deletions(-) create mode 100644 tests/integration/test_row_policy/normal_filter2_table2.xml diff --git a/tests/integration/test_row_policy/normal_filter2_table2.xml b/tests/integration/test_row_policy/normal_filter2_table2.xml new file mode 100644 index 00000000000..aca6bddc334 --- /dev/null +++ b/tests/integration/test_row_policy/normal_filter2_table2.xml @@ -0,0 +1,16 @@ + + + + + + + + + + a > 0 + + + + + + diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index c11e1b1e21c..25d90ae25b9 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -103,11 +103,32 @@ def test_join(): def test_cannot_trick_row_policy_with_keyword_with(): - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) - assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1]]) assert node.query("WITH 0 AS a SELECT a FROM mydb.filtered_table1") == TSV([[0], [0]]) assert node.query("WITH 0 AS a SELECT b FROM mydb.filtered_table1") == TSV([[0], [1]]) - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a IN(0, 1) WHERE b IN(0, 1)") == TSV([[0], [1]]) + + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 WHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a IN(0, 1) WHERE b IN(0, 1)") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE b IN(0, 1) WHERE a IN(0, 1)") == TSV([[1, 0], [1, 1]]) + + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 WHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE a IN(0, 1) WHERE b IN(0, 1)") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE b IN(0, 1) WHERE a IN(0, 1)") == TSV([[0, 0], [0, 1]]) + + assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3") == TSV([[0, 1], [1, 0]]) + assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0]]) + assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0]]) + assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0]]) + assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0]]) + + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 1], [1, 0], [0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) def test_policy_from_users_xml_affects_only_user_assigned(): @@ -123,17 +144,17 @@ def test_policy_from_users_xml_affects_only_user_assigned(): def test_with_prewhere(): - copy_policy_xml('normal_filters.xml') - assert node.query("SELECT * FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 5, 2, 1]]) + copy_policy_xml('normal_filter2_table2.xml') + assert node.query("SELECT * FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 3, 2, 1]]) assert node.query("SELECT a FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4]]) - assert node.query("SELECT a, b FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 5]]) - assert node.query("SELECT b, c FROM mydb.filtered_table2 WHERE a > 1") == TSV([[5, 2]]) + assert node.query("SELECT a, b FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 3]]) + assert node.query("SELECT b, c FROM mydb.filtered_table2 WHERE a > 1") == TSV([[3, 2]]) assert node.query("SELECT d FROM mydb.filtered_table2 WHERE a > 1") == TSV([[1]]) - assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 5, 2, 1]]) + assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 3, 2, 1]]) assert node.query("SELECT a FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4]]) - assert node.query("SELECT a, b FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 5]]) - assert node.query("SELECT b, c FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[5, 2]]) + assert node.query("SELECT a, b FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 3]]) + assert node.query("SELECT b, c FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[3, 2]]) assert node.query("SELECT d FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[1]]) assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1, 2, 3, 4]]) @@ -145,13 +166,22 @@ def test_with_prewhere(): assert node.query("SELECT c, d FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[3, 4]]) +def test_with_throwif_in_where(): + copy_policy_xml('no_filters.xml') + assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'expected') = 0") + + copy_policy_xml('normal_filter2_table2.xml') + assert node.query("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'pwned') = 0") == TSV([ + [1, 2, 3, 4], [4, 3, 2, 1]]) + + def test_with_throwif_in_prewhere(): copy_policy_xml('no_filters.xml') - assert 'expected' in node.query_and_get_error("SELECT throwIf(a = 0, 'expected') FROM mydb.filtered_table2 PREWHERE b < 10") + assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 PREWHERE throwIf(a = 0, 'expected') = 0") - copy_policy_xml('normal_filters.xml') - assert node.query("SELECT throwIf(a = 0, 'pwned') FROM mydb.filtered_table2 PREWHERE b < 10") == TSV([ - [4, 5, 2, 1], [1, 2, 3, 4]]) + copy_policy_xml('normal_filter2_table2.xml') + assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE throwIf(a = 0, 'pwned') = 0") == TSV([ + [1, 2, 3, 4], [4, 3, 2, 1]]) def test_change_of_users_xml_changes_row_policies(): @@ -176,6 +206,11 @@ def test_change_of_users_xml_changes_row_policies(): assert node.query("SELECT * FROM mydb.filtered_table2") == TSV([[0, 0, 0, 0], [0, 0, 6, 0]]) assert node.query("SELECT * FROM mydb.filtered_table3") == TSV([[0, 1], [1, 0]]) + copy_policy_xml('normal_filter2_table2.xml') + assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]]) + assert node.query("SELECT * FROM mydb.filtered_table2") == TSV([[1, 2, 3, 4], [4, 3, 2, 1]]) + assert node.query("SELECT * FROM mydb.filtered_table3") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]]) + copy_policy_xml('no_filters.xml') assert node.query("SELECT * FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1], [1, 0], [1, 1]]) assert node.query("SELECT * FROM mydb.filtered_table2") == TSV( From 5e5b3b80cee11b5c3c8e72c1ccc5f4cfb5be585e Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Fri, 19 Feb 2021 19:46:52 +0400 Subject: [PATCH 261/510] Remove debug printouts --- src/Interpreters/InterpreterSelectQuery.cpp | 12 ------------ .../MergeTree/MergeTreeBaseSelectProcessor.cpp | 9 --------- 2 files changed, 21 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 826be1e5143..2d6a22cce2f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -106,10 +106,6 @@ namespace ErrorCodes /// Assumes `storage` is set and the table filter (row-level security) is not empty. String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, const Names & prerequisite_columns) const { - // std::cerr << "----- InterpreterSelectQuery::generateFilterActions\n"; - // for (const auto & name : prerequisite_columns) - // std::cerr << name << std::endl; - const auto & db_name = table_id.getDatabaseName(); const auto & table_name = table_id.getTableName(); @@ -144,7 +140,6 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co auto syntax_result = TreeRewriter(*context).analyzeSelect(query_ast, TreeRewriterResult({}, storage, metadata_snapshot)); SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot); actions = analyzer.simpleSelectActions(); - //std::cerr << actions-> return expr_list->children.at(0)->getColumnName(); } @@ -527,10 +522,6 @@ void InterpreterSelectQuery::buildQueryPlan(QueryPlan & query_plan) { executeImpl(query_plan, input, std::move(input_pipe)); - // WriteBufferFromOwnString buf; - // query_plan.explainPlan(buf, {.header = true, .actions = true}); - // std::cerr << buf.str(); - /// We must guarantee that result structure is the same as in getSampleBlock() if (!blocksHaveEqualStructure(query_plan.getCurrentDataStream().header, result_header)) { @@ -826,7 +817,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu const bool does_storage_support_prewhere = !input && !input_pipe && storage && storage->supportsPrewhere(); if (does_storage_support_prewhere && settings.optimize_move_to_prewhere) { - // std::cerr << "----- Moving row level filter to prewhere\n"; /// Execute row level filter in prewhere as a part of "move to prewhere" optimization. expressions.prewhere_info = std::make_shared( std::move(expressions.filter_info->actions), @@ -839,7 +829,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu else { /// Add row level security actions to prewhere. - // std::cerr << expressions.filter_info->actions->dumpDAG() << std::endl; expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions); expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); expressions.prewhere_info->row_level_filter_actions->projectInput(false); @@ -1658,7 +1647,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (prewhere_info) { - // std::cerr << "-------- filling prewhere info \n"; query_info.prewhere_info = std::make_shared(); query_info.prewhere_info->prewhere_actions = std::make_shared(prewhere_info->prewhere_actions); diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 90da45cc6d8..4911f9982d5 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -334,13 +334,9 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P { if (prewhere_info) { - // std::cerr << "0: " << block.dumpStructure() << std::endl; - if (prewhere_info->alias_actions) prewhere_info->alias_actions->execute(block); - // std::cerr << "1: " << block.dumpStructure() << std::endl; - if (prewhere_info->row_level_filter) { prewhere_info->row_level_filter->execute(block); @@ -351,13 +347,10 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P ErrorCodes::LOGICAL_ERROR); } } - // std::cerr << "2: " << block.dumpStructure() << std::endl; if (prewhere_info->prewhere_actions) prewhere_info->prewhere_actions->execute(block); - // std::cerr << "3: " << block.dumpStructure() << std::endl; - auto & prewhere_column = block.getByName(prewhere_info->prewhere_column_name); if (!prewhere_column.type->canBeUsedInBooleanContext()) { @@ -372,8 +365,6 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P auto & ctn = block.getByName(prewhere_info->prewhere_column_name); ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst(); } - - // std::cerr << "4: " << block.dumpStructure() << std::endl; } } From ad374ec0953926af32227aea9744fc9c09da65ca Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 19:05:26 +0300 Subject: [PATCH 262/510] Rename file --- src/Coordination/NuKeeperServer.cpp | 4 ++-- src/Coordination/NuKeeperServer.h | 4 ++-- ...ryStateManager.cpp => NuKeeperStateManager.cpp} | 14 +++++++------- ...MemoryStateManager.h => NuKeeperStateManager.h} | 6 +++--- src/Coordination/tests/gtest_for_build.cpp | 6 +++--- 5 files changed, 17 insertions(+), 17 deletions(-) rename src/Coordination/{InMemoryStateManager.cpp => NuKeeperStateManager.cpp} (88%) rename src/Coordination/{InMemoryStateManager.h => NuKeeperStateManager.h} (94%) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index c2917e3ab76..c0dc3f85343 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -26,7 +26,7 @@ NuKeeperServer::NuKeeperServer( : server_id(server_id_) , coordination_settings(coordination_settings_) , state_machine(nuraft::cs_new(responses_queue_, coordination_settings)) - , state_manager(nuraft::cs_new(server_id, "test_keeper_server", config, coordination_settings)) + , state_manager(nuraft::cs_new(server_id, "test_keeper_server", config, coordination_settings)) , responses_queue(responses_queue_) { } diff --git a/src/Coordination/NuKeeperServer.h b/src/Coordination/NuKeeperServer.h index a8d269eb9eb..40f3efec76a 100644 --- a/src/Coordination/NuKeeperServer.h +++ b/src/Coordination/NuKeeperServer.h @@ -2,7 +2,7 @@ #include // Y_IGNORE #include -#include +#include #include #include #include @@ -20,7 +20,7 @@ private: nuraft::ptr state_machine; - nuraft::ptr state_manager; + nuraft::ptr state_manager; nuraft::raft_launcher launcher; diff --git a/src/Coordination/InMemoryStateManager.cpp b/src/Coordination/NuKeeperStateManager.cpp similarity index 88% rename from src/Coordination/InMemoryStateManager.cpp rename to src/Coordination/NuKeeperStateManager.cpp index 084ab043d12..14e8badd92f 100644 --- a/src/Coordination/InMemoryStateManager.cpp +++ b/src/Coordination/NuKeeperStateManager.cpp @@ -1,4 +1,4 @@ -#include +#include #include namespace DB @@ -9,7 +9,7 @@ namespace ErrorCodes extern const int RAFT_ERROR; } -InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) +NuKeeperStateManager::NuKeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) : my_server_id(server_id_) , my_port(port) , log_store(nuraft::cs_new(logs_path, 5000, true)) @@ -19,7 +19,7 @@ InMemoryStateManager::InMemoryStateManager(int server_id_, const std::string & h cluster_config->get_servers().push_back(peer_config); } -InMemoryStateManager::InMemoryStateManager( +NuKeeperStateManager::NuKeeperStateManager( int my_server_id_, const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config, @@ -63,17 +63,17 @@ InMemoryStateManager::InMemoryStateManager( throw Exception(ErrorCodes::RAFT_ERROR, "At least one of servers should be able to start as leader (without )"); } -void InMemoryStateManager::loadLogStore(size_t start_log_index) +void NuKeeperStateManager::loadLogStore(size_t start_log_index) { log_store->init(start_log_index); } -void InMemoryStateManager::flushLogStore() +void NuKeeperStateManager::flushLogStore() { log_store->flush(); } -void InMemoryStateManager::save_config(const nuraft::cluster_config & config) +void NuKeeperStateManager::save_config(const nuraft::cluster_config & config) { // Just keep in memory in this example. // Need to write to disk here, if want to make it durable. @@ -81,7 +81,7 @@ void InMemoryStateManager::save_config(const nuraft::cluster_config & config) cluster_config = nuraft::cluster_config::deserialize(*buf); } -void InMemoryStateManager::save_state(const nuraft::srv_state & state) +void NuKeeperStateManager::save_state(const nuraft::srv_state & state) { // Just keep in memory in this example. // Need to write to disk here, if want to make it durable. diff --git a/src/Coordination/InMemoryStateManager.h b/src/Coordination/NuKeeperStateManager.h similarity index 94% rename from src/Coordination/InMemoryStateManager.h rename to src/Coordination/NuKeeperStateManager.h index c53f00702d4..66229a3b8d1 100644 --- a/src/Coordination/InMemoryStateManager.h +++ b/src/Coordination/NuKeeperStateManager.h @@ -10,16 +10,16 @@ namespace DB { -class InMemoryStateManager : public nuraft::state_mgr +class NuKeeperStateManager : public nuraft::state_mgr { public: - InMemoryStateManager( + NuKeeperStateManager( int server_id_, const std::string & config_prefix, const Poco::Util::AbstractConfiguration & config, const CoordinationSettingsPtr & coordination_settings); - InMemoryStateManager( + NuKeeperStateManager( int server_id_, const std::string & host, int port, diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 457d0dbc52a..f871f39a906 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -9,7 +9,7 @@ #include #include #include -#include +#include #include #include #include @@ -100,7 +100,7 @@ struct SimpliestRaftServer , port(port_) , endpoint(hostname + ":" + std::to_string(port)) , state_machine(nuraft::cs_new()) - , state_manager(nuraft::cs_new(server_id, hostname, port, logs_path)) + , state_manager(nuraft::cs_new(server_id, hostname, port, logs_path)) { state_manager->loadLogStore(1); nuraft::raft_params params; @@ -151,7 +151,7 @@ struct SimpliestRaftServer nuraft::ptr state_machine; // State manager. - nuraft::ptr state_manager; + nuraft::ptr state_manager; // Raft launcher. nuraft::raft_launcher launcher; From 7ee72dfd0c46f0884c446003dfd3676644f6b19e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Feb 2021 22:24:20 +0300 Subject: [PATCH 263/510] Missed tests --- .../configs/use_test_keeper.xml | 8 ++ .../__init__.py | 1 + .../configs/enable_test_keeper1.xml | 39 ++++++++ .../configs/enable_test_keeper2.xml | 39 ++++++++ .../configs/enable_test_keeper3.xml | 39 ++++++++ .../configs/log_conf.xml | 12 +++ .../configs/use_test_keeper.xml | 16 +++ .../test.py | 98 +++++++++++++++++++ 8 files changed, 252 insertions(+) create mode 100644 tests/integration/test_testkeeper_back_to_back/configs/use_test_keeper.xml create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/__init__.py create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper1.xml create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper2.xml create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper3.xml create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/configs/log_conf.xml create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/configs/use_test_keeper.xml create mode 100644 tests/integration/test_testkeeper_persistent_log_multinode/test.py diff --git a/tests/integration/test_testkeeper_back_to_back/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_back_to_back/configs/use_test_keeper.xml new file mode 100644 index 00000000000..12dc7fd9447 --- /dev/null +++ b/tests/integration/test_testkeeper_back_to_back/configs/use_test_keeper.xml @@ -0,0 +1,8 @@ + + + + node1 + 9181 + + + diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/__init__.py b/tests/integration/test_testkeeper_persistent_log_multinode/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper1.xml b/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper1.xml new file mode 100644 index 00000000000..a47e5eae09a --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper1.xml @@ -0,0 +1,39 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + true + true + 2 + + + 3 + node3 + 44444 + true + true + 1 + + + + diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper2.xml b/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper2.xml new file mode 100644 index 00000000000..18681f0dc95 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper2.xml @@ -0,0 +1,39 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + true + true + 2 + + + 3 + node3 + 44444 + true + true + 1 + + + + diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper3.xml b/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper3.xml new file mode 100644 index 00000000000..184d3724219 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/configs/enable_test_keeper3.xml @@ -0,0 +1,39 @@ + + + 9181 + 3 + /var/lib/clickhouse/coordination/log + + + 5000 + 10000 + trace + + + + + 1 + node1 + 44444 + true + 3 + + + 2 + node2 + 44444 + true + true + 2 + + + 3 + node3 + 44444 + true + true + 1 + + + + diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/configs/log_conf.xml b/tests/integration/test_testkeeper_persistent_log_multinode/configs/log_conf.xml new file mode 100644 index 00000000000..318a6bca95d --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/configs/log_conf.xml @@ -0,0 +1,12 @@ + + 3 + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/configs/use_test_keeper.xml b/tests/integration/test_testkeeper_persistent_log_multinode/configs/use_test_keeper.xml new file mode 100644 index 00000000000..b6139005d2f --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/configs/use_test_keeper.xml @@ -0,0 +1,16 @@ + + + + node1 + 9181 + + + node2 + 9181 + + + node3 + 9181 + + + diff --git a/tests/integration/test_testkeeper_persistent_log_multinode/test.py b/tests/integration/test_testkeeper_persistent_log_multinode/test.py new file mode 100644 index 00000000000..cb9cf5a59d1 --- /dev/null +++ b/tests/integration/test_testkeeper_persistent_log_multinode/test.py @@ -0,0 +1,98 @@ +#!/usr/bin/env python3 +import pytest +from helpers.cluster import ClickHouseCluster +import random +import string +import os +import time + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) +node2 = cluster.add_instance('node2', main_configs=['configs/enable_test_keeper2.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) +node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) + +from kazoo.client import KazooClient, KazooState + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def get_fake_zk(nodename, timeout=30.0): + _fake_zk_instance = KazooClient(hosts=cluster.get_instance_ip(nodename) + ":9181", timeout=timeout) + def reset_listener(state): + nonlocal _fake_zk_instance + print("Fake zk callback called for state", state) + if state != KazooState.CONNECTED: + _fake_zk_instance._reset() + + _fake_zk_instance.add_listener(reset_listener) + _fake_zk_instance.start() + return _fake_zk_instance + +def stop_zk(zk): + try: + if zk: + zk.stop() + zk.close() + except: + pass + +def test_restart_multinode(started_cluster): + try: + node1_zk = node2_zk = node3_zk = None + + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + + for i in range(100): + node1_zk.create("/test_read_write_multinode_node" + str(i), ("somedata" + str(i)).encode()) + + for i in range(100): + if i % 10 == 0: + node1_zk.delete("/test_read_write_multinode_node" + str(i)) + + node2_zk.sync("/test_read_write_multinode_node0") + node3_zk.sync("/test_read_write_multinode_node0") + + for i in range(100): + if i % 10 != 0: + assert node2_zk.get("/test_read_write_multinode_node" + str(i))[0] == ("somedata" + str(i)).encode() + assert node3_zk.get("/test_read_write_multinode_node" + str(i))[0] == ("somedata" + str(i)).encode() + else: + assert node2_zk.exists("/test_read_write_multinode_node" + str(i)) is None + assert node3_zk.exists("/test_read_write_multinode_node" + str(i)) is None + + finally: + for zk in [node1_zk, node2_zk, node3_zk]: + stop_zk(zk) + + node1.restart_clickhouse(kill=True) + node2.restart_clickhouse(kill=True) + node3.restart_clickhouse(kill=True) + for i in range(100): + try: + node1_zk = get_fake_zk("node1") + node2_zk = get_fake_zk("node2") + node3_zk = get_fake_zk("node3") + for i in range(100): + if i % 10 != 0: + assert node1_zk.get("/test_read_write_multinode_node" + str(i))[0] == ("somedata" + str(i)).encode() + assert node2_zk.get("/test_read_write_multinode_node" + str(i))[0] == ("somedata" + str(i)).encode() + assert node3_zk.get("/test_read_write_multinode_node" + str(i))[0] == ("somedata" + str(i)).encode() + else: + assert node1_zk.exists("/test_read_write_multinode_node" + str(i)) is None + assert node2_zk.exists("/test_read_write_multinode_node" + str(i)) is None + assert node3_zk.exists("/test_read_write_multinode_node" + str(i)) is None + break + except Exception as ex: + print("Got exception as ex", ex) + finally: + for zk in [node1_zk, node2_zk, node3_zk]: + stop_zk(zk) From 2ab643170dada778ed2914d97077207f0caa06bc Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 20 Feb 2021 00:45:19 +0400 Subject: [PATCH 264/510] Add 'SETTINGS optimize_move_to_prewhere = 0' in WHERE-only SELECTs --- tests/integration/test_row_policy/test.py | 44 +++++++++---------- .../prewhere_with_row_level_filter.xml | 2 +- 2 files changed, 23 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 25d90ae25b9..c83e9cb8ec3 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -106,26 +106,26 @@ def test_cannot_trick_row_policy_with_keyword_with(): assert node.query("WITH 0 AS a SELECT a FROM mydb.filtered_table1") == TSV([[0], [0]]) assert node.query("WITH 0 AS a SELECT b FROM mydb.filtered_table1") == TSV([[0], [1]]) - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 WHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[1, 0], [1, 1]]) - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[1, 0], [1, 1]]) - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a IN(0, 1) WHERE b IN(0, 1)") == TSV([[1, 0], [1, 1]]) - assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE b IN(0, 1) WHERE a IN(0, 1)") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 WHERE a >= 0 AND b >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a >= 0 AND b >= 0") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE a >= 0 WHERE b >= 0") == TSV([[1, 0], [1, 1]]) + assert node.query("WITH 0 AS a SELECT * FROM mydb.filtered_table1 PREWHERE b >= 0 WHERE a >= 0") == TSV([[1, 0], [1, 1]]) - assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1]]) - assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 WHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[0, 0], [0, 1]]) - assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE a IN(0, 1) AND b IN(0, 1)") == TSV([[0, 0], [0, 1]]) - assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE a IN(0, 1) WHERE b IN(0, 1)") == TSV([[0, 0], [0, 1]]) - assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE b IN(0, 1) WHERE a IN(0, 1)") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 WHERE a >= 0 AND b >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE a >= 0 AND b >= 0") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE a >= 0 WHERE b >= 0") == TSV([[0, 0], [0, 1]]) + assert node.query("WITH 0 AS a SELECT a, b FROM mydb.filtered_table1 PREWHERE b >= 0 WHERE a >= 0") == TSV([[0, 0], [0, 1]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3") == TSV([[0, 1], [1, 0]]) - assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0]]) + assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 1], [1, 0], [0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 1], [1, 0], [0, 0]]) assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) @@ -145,17 +145,17 @@ def test_policy_from_users_xml_affects_only_user_assigned(): def test_with_prewhere(): copy_policy_xml('normal_filter2_table2.xml') - assert node.query("SELECT * FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 3, 2, 1]]) - assert node.query("SELECT a FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4]]) - assert node.query("SELECT a, b FROM mydb.filtered_table2 WHERE a > 1") == TSV([[4, 3]]) - assert node.query("SELECT b, c FROM mydb.filtered_table2 WHERE a > 1") == TSV([[3, 2]]) - assert node.query("SELECT d FROM mydb.filtered_table2 WHERE a > 1") == TSV([[1]]) + assert node.query("SELECT * FROM mydb.filtered_table2 WHERE a > 1 SETTINGS optimize_move_to_prewhere = 0") == TSV([[4, 3, 2, 1]]) + assert node.query("SELECT a FROM mydb.filtered_table2 WHERE a > 1 SETTINGS optimize_move_to_prewhere = 0") == TSV([[4]]) + assert node.query("SELECT a, b FROM mydb.filtered_table2 WHERE a > 1 SETTINGS optimize_move_to_prewhere = 0") == TSV([[4, 3]]) + assert node.query("SELECT b, c FROM mydb.filtered_table2 WHERE a > 1 SETTINGS optimize_move_to_prewhere = 0") == TSV([[3, 2]]) + assert node.query("SELECT d FROM mydb.filtered_table2 WHERE a > 1 SETTINGS optimize_move_to_prewhere = 0") == TSV([[1]]) - assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 3, 2, 1]]) - assert node.query("SELECT a FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4]]) + assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 3, 2, 1]]) + assert node.query("SELECT a FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4]]) assert node.query("SELECT a, b FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[4, 3]]) assert node.query("SELECT b, c FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[3, 2]]) - assert node.query("SELECT d FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[1]]) + assert node.query("SELECT d FROM mydb.filtered_table2 PREWHERE a > 1") == TSV([[1]]) assert node.query("SELECT * FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1, 2, 3, 4]]) assert node.query("SELECT a FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[1]]) @@ -168,10 +168,10 @@ def test_with_prewhere(): def test_with_throwif_in_where(): copy_policy_xml('no_filters.xml') - assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'expected') = 0") + assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'expected') = 0 SETTINGS optimize_move_to_prewhere = 0") copy_policy_xml('normal_filter2_table2.xml') - assert node.query("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'pwned') = 0") == TSV([ + assert node.query("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'pwned') = 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([ [1, 2, 3, 4], [4, 3, 2, 1]]) diff --git a/tests/performance/prewhere_with_row_level_filter.xml b/tests/performance/prewhere_with_row_level_filter.xml index d73690ca811..03c73a88f89 100644 --- a/tests/performance/prewhere_with_row_level_filter.xml +++ b/tests/performance/prewhere_with_row_level_filter.xml @@ -6,7 +6,7 @@ INSERT INTO test_prl SELECT number FROM numbers(50000000); SELECT * FROM test_prl; - SELECT * FROM test_prl WHERE n % 3 AND n % 5; + SELECT * FROM test_prl WHERE n % 3 AND n % 5 SETTINGS optimize_move_to_prewhere = 0; SELECT * FROM test_prl PREWHERE n % 3 AND n % 5; SELECT * FROM test_prl PREWHERE n % 3 WHERE n % 5; SELECT * FROM test_prl PREWHERE n % 5 WHERE n % 3; From 9bd9ea9fbcb9ffa9bf606e75228384c175851e69 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 20 Feb 2021 00:45:58 +0300 Subject: [PATCH 265/510] Try to fix sigsev --- src/Client/Connection.cpp | 3 +-- src/Client/Connection.h | 7 ++++--- src/Client/ConnectionPoolWithFailover.h | 2 +- src/Client/PacketReceiver.h | 6 ++++-- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index b6903ae6c92..80d44a336a5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -109,8 +109,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) } in = std::make_shared(*socket); - if (async_callback) - in->setAsyncCallback(std::move(async_callback)); + in->setAsyncCallback(std::move(async_callback)); out = std::make_shared(*socket); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index d317ecb56b3..95fbb902c2c 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -199,9 +199,10 @@ public: /// Each time read from socket blocks and async_callback is set, it will be called. You can poll socket inside it. void setAsyncCallback(AsyncCallback async_callback_) { - async_callback = std::move(async_callback_); if (in) - in->setAsyncCallback(std::move(async_callback)); + in->setAsyncCallback(std::move(async_callback_)); + else + async_callback = std::move(async_callback_); } private: @@ -291,7 +292,7 @@ private: LoggerWrapper log_wrapper; - AsyncCallback async_callback; + AsyncCallback async_callback = {}; void connect(const ConnectionTimeouts & timeouts); void sendHello(); diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 023ef863bdf..3bd39977566 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -99,7 +99,7 @@ private: /// Try to get a connection from the pool and check that it is good. /// If table_to_check is not null and the check is enabled in settings, check that replication delay /// for this table is not too large. - TryResult tryGetEntry( + static TryResult tryGetEntry( IConnectionPool & pool, const ConnectionTimeouts & timeouts, std::string & fail_message, diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index 4d42804d0a2..79d805be877 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -104,10 +104,12 @@ private: { try { - AsyncCallbackSetter async_setter(receiver.connection, ReadCallback{receiver, sink}); while (true) { - receiver.packet = receiver.connection->receivePacket(); + { + AsyncCallbackSetter async_setter(receiver.connection, ReadCallback{receiver, sink}); + receiver.packet = receiver.connection->receivePacket(); + } sink = std::move(sink).resume(); } From 21ee685ef0f1910d42e0e5a47b010b2eb4cc9a71 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 Feb 2021 19:38:46 +0000 Subject: [PATCH 266/510] Fix brotly --- src/IO/BrotliWriteBuffer.cpp | 2 +- ...7_http_compression_prefer_brotli.reference | 23 +++++++++++++++++++ .../01057_http_compression_prefer_brotli.sh | 2 ++ 3 files changed, 26 insertions(+), 1 deletion(-) diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp index d14c94ca43d..e562cc70e61 100644 --- a/src/IO/BrotliWriteBuffer.cpp +++ b/src/IO/BrotliWriteBuffer.cpp @@ -86,7 +86,7 @@ void BrotliWriteBuffer::nextImpl() throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED); } } - while (in_available > 0 || out_capacity == 0); + while (in_available > 0); } void BrotliWriteBuffer::finish() diff --git a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference index 5dd396a38c9..c28cbee8485 100644 --- a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference +++ b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference @@ -9,3 +9,26 @@ 999997 999998 999999 + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999998" + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999999" + } + ], + + "rows": 1000000, + + "rows_before_limit_at_least": 1048080, + diff --git a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh index e5f4d12ee18..f93062d43a7 100755 --- a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh +++ b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh @@ -11,3 +11,5 @@ ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip,deflate,br' "${CLICKHOUSE_URL}& ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip,deflate' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT 1' | gzip -d ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM numbers(1000000)' | gzip -d | tail -n3 ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM numbers(1000000)' | brotli -d | tail -n3 + +${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23 From 85ea5cafc46dbfa280c0376aa6f96d9b2ae92b13 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 20 Feb 2021 03:18:16 +0400 Subject: [PATCH 267/510] Always apply row_level_filter immediately --- .../MergeTree/MergeTreeRangeReader.cpp | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index b4b8e4309b5..a3e4fbf6e6b 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -865,7 +865,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r return; const auto & header = merge_tree_reader->getColumns(); - size_t num_columns = header.size(); + const auto num_columns = header.size(); if (result.columns.size() != num_columns) throw Exception("Invalid number of columns passed to MergeTreeRangeReader. " @@ -900,8 +900,25 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (prewhere_info->row_level_filter) { prewhere_info->row_level_filter->execute(block); + const auto filter_column_pos = block.getPositionByName(prewhere_info->row_level_column_name); result.addFilter(block.getByPosition(filter_column_pos).column); + + result.columns.clear(); + result.columns.reserve(block.columns()); + + for (auto & col : block) + result.columns.emplace_back(std::move(col.column)); + + const auto * result_filter = result.getFilter(); + filterColumns(result.columns, result_filter->getData()); + + auto it = block.begin(); + for (auto & col : result.columns) + it++->column = std::move(col); + + result.columns.clear(); + result.clearFilter(); } prewhere_info->prewhere_actions->execute(block); From e1868d1392d9834d84e4d9f1f0230429e7df2e3c Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Feb 2021 05:13:31 +0000 Subject: [PATCH 268/510] Move test into separate file --- .../01057_http_compression_prefer_brotli.sh | 1 - ...tli_http_compression_json_format.reference | 23 +++++++++++++++++++ ...ong_brotli_http_compression_json_format.sh | 7 ++++++ 3 files changed, 30 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.reference create mode 100755 tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.sh diff --git a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh index f93062d43a7..22ab745d7c0 100755 --- a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh +++ b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.sh @@ -12,4 +12,3 @@ ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip,deflate' "${CLICKHOUSE_URL}& ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM numbers(1000000)' | gzip -d | tail -n3 ${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d 'SELECT number FROM numbers(1000000)' | brotli -d | tail -n3 -${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23 diff --git a/tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.reference b/tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.reference new file mode 100644 index 00000000000..7c089a2fd05 --- /dev/null +++ b/tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.reference @@ -0,0 +1,23 @@ + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999998" + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999999" + } + ], + + "rows": 1000000, + + "rows_before_limit_at_least": 1048080, + diff --git a/tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.sh b/tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.sh new file mode 100755 index 00000000000..a187d778fdb --- /dev/null +++ b/tests/queries/0_stateless/01722_long_brotli_http_compression_json_format.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: br' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | brotli -d | tail -n30 | head -n23 From 8361904b4d7cdbc8b00390de17e5194055cdc38e Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 20 Feb 2021 10:47:29 +0400 Subject: [PATCH 269/510] Add cases for throwIf() --- tests/integration/test_row_policy/test.py | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index c83e9cb8ec3..22da7b10b0b 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -166,7 +166,17 @@ def test_with_prewhere(): assert node.query("SELECT c, d FROM mydb.filtered_table2 PREWHERE a < 4 WHERE b < 10") == TSV([[3, 4]]) -def test_with_throwif_in_where(): +def test_throwif_error_in_where_with_same_condition_as_filter(): + copy_policy_xml('normal_filter2_table2.xml') + assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a > 0, 'expected') = 0 SETTINGS optimize_move_to_prewhere = 0") + + +def test_throwif_error_in_prewhere_with_same_condition_as_filter(): + copy_policy_xml('normal_filter2_table2.xml') + assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 PREWHERE throwIf(a > 0, 'expected') = 0") + + +def test_throwif_in_where_doesnt_expose_restricted_data(): copy_policy_xml('no_filters.xml') assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 WHERE throwIf(a = 0, 'expected') = 0 SETTINGS optimize_move_to_prewhere = 0") @@ -175,7 +185,7 @@ def test_with_throwif_in_where(): [1, 2, 3, 4], [4, 3, 2, 1]]) -def test_with_throwif_in_prewhere(): +def test_throwif_in_prewhere_doesnt_expose_restricted_data(): copy_policy_xml('no_filters.xml') assert 'expected' in node.query_and_get_error("SELECT * FROM mydb.filtered_table2 PREWHERE throwIf(a = 0, 'expected') = 0") From f820047cc841fa2b129e3f3d20ebcc0c28d1940c Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 19 Feb 2021 15:48:48 +0000 Subject: [PATCH 270/510] Fix --- .../PostgreSQL/fetchPostgreSQLTableStructure.cpp | 7 +++++-- tests/integration/test_storage_postgresql/test.py | 8 ++++---- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 15ce9a1baed..e065a497115 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -56,7 +56,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl { /// Numeric and decimal will both end up here as numeric. If it has type and precision, /// there will be Numeric(x, y), otherwise just Numeric - uint32_t precision, scale; + UInt32 precision, scale; if (type.ends_with(")")) { res = DataTypeFactory::instance().get(type); @@ -71,11 +71,14 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl res = std::make_shared>(precision, scale); else if (precision <= DecimalUtils::maxPrecision()) res = std::make_shared>(precision, scale); + else + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Presicion {} and scale {} are too big and not supported", precision, scale); } else { precision = DecimalUtils::maxPrecision(); - res = std::make_shared>(precision, precision); + scale = precision >> 1; + res = std::make_shared>(precision, scale); } } diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 03af32a4803..cee495438a2 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -63,13 +63,13 @@ def test_postgres_conversions(started_cluster): cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( a smallint, b integer, c bigint, d real, e double precision, f serial, g bigserial, - h timestamp, i date, j decimal(5, 5), k numeric)''') + h timestamp, i date, j decimal(5, 3), k numeric)''') node1.query(''' INSERT INTO TABLE FUNCTION postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword') VALUES - (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 0.22222, 0.22222)''') + (-32768, -2147483648, -9223372036854775808, 1.12345, 1.1234567890, 2147483647, 9223372036854775807, '2000-05-12 12:12:12', '2000-05-12', 22.222, 22.222)''') result = node1.query(''' - SELECT a, b, c, d, e, f, g, h, i, j, toDecimal32(k, 5) FROM postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword')''') - assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t0.22222\t0.22222\n') + SELECT a, b, c, d, e, f, g, h, i, j, toDecimal128(k, 3) FROM postgresql('postgres1:5432', 'clickhouse', 'test_types', 'postgres', 'mysecretpassword')''') + assert(result == '-32768\t-2147483648\t-9223372036854775808\t1.12345\t1.123456789\t2147483647\t9223372036854775807\t2000-05-12 12:12:12\t2000-05-12\t22.222\t22.222\n') cursor.execute( '''CREATE TABLE IF NOT EXISTS test_array_dimensions From 0d88366b2775bdcb60ae3eb18bc9fcb2ce7eef01 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Feb 2021 07:07:50 +0000 Subject: [PATCH 271/510] Add forgotten .reference file update --- ...7_http_compression_prefer_brotli.reference | 23 ------------------- 1 file changed, 23 deletions(-) diff --git a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference index c28cbee8485..5dd396a38c9 100644 --- a/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference +++ b/tests/queries/0_stateless/01057_http_compression_prefer_brotli.reference @@ -9,26 +9,3 @@ 999997 999998 999999 - }, - { - "datetime": "2020-12-12", - "pipeline": "test-pipeline", - "host": "clickhouse-test-host-001.clickhouse.com", - "home": "clickhouse", - "detail": "clickhouse", - "row_number": "999998" - }, - { - "datetime": "2020-12-12", - "pipeline": "test-pipeline", - "host": "clickhouse-test-host-001.clickhouse.com", - "home": "clickhouse", - "detail": "clickhouse", - "row_number": "999999" - } - ], - - "rows": 1000000, - - "rows_before_limit_at_least": 1048080, - From a38a31c954aa03251767f769f8c6b5584165b2dd Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 20 Feb 2021 09:58:24 +0000 Subject: [PATCH 272/510] Fix typos check --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index e065a497115..d3a42ead3f6 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -72,7 +72,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl else if (precision <= DecimalUtils::maxPrecision()) res = std::make_shared>(precision, scale); else - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Presicion {} and scale {} are too big and not supported", precision, scale); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Precision {} and scale {} are too big and not supported", precision, scale); } else { From 673e24d7ef8ea1f181cc79e4df53a09564dc8332 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 20 Feb 2021 14:00:16 +0300 Subject: [PATCH 273/510] Refactor --- src/Interpreters/ActionsDAG.h | 2 + src/Interpreters/ExpressionAnalyzer.cpp | 17 +- src/Interpreters/InterpreterSelectQuery.cpp | 51 +++-- .../MergeTreeBaseSelectProcessor.cpp | 2 + .../MergeTree/MergeTreeBlockReadUtils.cpp | 24 +- .../MergeTree/MergeTreeRangeReader.cpp | 213 +++++++++++------- src/Storages/MergeTree/MergeTreeRangeReader.h | 6 +- 7 files changed, 196 insertions(+), 119 deletions(-) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index d3f1d65d454..18f6f9a89ee 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -208,6 +208,8 @@ public: const Context & context, bool can_replace = false); + void addNodeToIndex(const Node * node) { index.insert(const_cast(node)); } + /// Call addAlias several times. void addAliases(const NamesWithAliases & aliases); /// Add alias actions and remove unused columns from index. Also specify result columns order in index. diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 3145df23b95..2dc8d137abe 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1405,7 +1405,8 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (storage && filter_info_) { filter_info = filter_info_; - query_analyzer.appendPreliminaryFilter(chain, filter_info->actions, filter_info->column_name); + filter_info->do_remove_column = true; + //query_analyzer.appendPreliminaryFilter(chain, filter_info->actions, filter_info->column_name); } if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) @@ -1574,11 +1575,11 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si { size_t next_step_i = 0; - if (hasFilter()) - { - const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); - filter_info->do_remove_column = step.can_remove_required_output.at(0); - } + // if (hasFilter()) + // { + // const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); + // filter_info->do_remove_column = step.can_remove_required_output.at(0); + // } if (hasPrewhere()) { @@ -1605,8 +1606,8 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si void ExpressionAnalysisResult::removeExtraColumns() const { - if (hasFilter()) - filter_info->actions->projectInput(); + // if (hasFilter()) + // filter_info->actions->projectInput(); if (hasWhere()) before_where->projectInput(); if (hasHaving()) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 95b42bede8b..54481cbe873 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -141,7 +141,15 @@ String InterpreterSelectQuery::generateFilterActions(ActionsDAGPtr & actions, co SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, *context, metadata_snapshot); actions = analyzer.simpleSelectActions(); - return expr_list->children.at(0)->getColumnName(); + auto column_name = expr_list->children.at(0)->getColumnName(); + actions->removeUnusedActions({column_name}); + actions->projectInput(false); + + ActionsDAG::Index index; + for (const auto * node : actions->getInputs()) + actions->addNodeToIndex(node); + + return column_name; } InterpreterSelectQuery::InterpreterSelectQuery( @@ -444,16 +452,22 @@ InterpreterSelectQuery::InterpreterSelectQuery( if (storage) { - source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()); - /// Fix source_header for filter actions. if (row_policy_filter) { filter_info = std::make_shared(); filter_info->column_name = generateFilterActions(filter_info->actions, required_columns); - source_header = metadata_snapshot->getSampleBlockForColumns( - filter_info->actions->getRequiredColumns().getNames(), storage->getVirtuals(), storage->getStorageID()); + + auto required_columns_from_filter = filter_info->actions->getRequiredColumns(); + + for (const auto & column : required_columns_from_filter) + { + if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) + required_columns.push_back(column.name); + } } + + source_header = metadata_snapshot->getSampleBlockForColumns(required_columns, storage->getVirtuals(), storage->getStorageID()); } /// Calculate structure of the result. @@ -834,6 +848,7 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu expressions.prewhere_info = std::make_shared( std::move(expressions.filter_info->actions), std::move(expressions.filter_info->column_name)); + expressions.prewhere_info->prewhere_actions->projectInput(false); expressions.prewhere_info->remove_prewhere_column = expressions.filter_info->do_remove_column; expressions.prewhere_info->need_filter = true; expressions.filter_info = nullptr; @@ -845,19 +860,19 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions); expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); expressions.prewhere_info->row_level_filter_actions->projectInput(false); - if (expressions.filter_info->do_remove_column) - { - /// Instead of removing column, add it to prewhere_actions input (but not in index). - /// It will be removed at prewhere_actions execution. - const auto & index = expressions.prewhere_info->row_level_filter_actions->getIndex(); - auto it = index.find(expressions.prewhere_info->row_level_column_name); - if (it == index.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found column {} in row level security filter {}", - expressions.prewhere_info->row_level_column_name, expressions.prewhere_info->row_level_filter_actions->dumpDAG()); - const auto & node = *it; + // if (expressions.filter_info->do_remove_column) + // { + // /// Instead of removing column, add it to prewhere_actions input (but not in index). + // /// It will be removed at prewhere_actions execution. + // const auto & index = expressions.prewhere_info->row_level_filter_actions->getIndex(); + // auto it = index.find(expressions.prewhere_info->row_level_column_name); + // if (it == index.end()) + // throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found column {} in row level security filter {}", + // expressions.prewhere_info->row_level_column_name, expressions.prewhere_info->row_level_filter_actions->dumpDAG()); + // const auto & node = *it; - expressions.prewhere_info->prewhere_actions->addInput(node->result_name, node->result_type, true, false); - } + // expressions.prewhere_info->prewhere_actions->addInput(node->result_name, node->result_type, true, false); + // } expressions.filter_info = nullptr; } @@ -1285,7 +1300,7 @@ void InterpreterSelectQuery::addEmptySourceToQueryPlan(QueryPlan & query_plan, c header, prewhere_info.row_level_filter, prewhere_info.row_level_column_name, - false); + true); }); } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 4911f9982d5..6bf164dd824 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -346,6 +346,8 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P throw Exception("Invalid type for filter in PREWHERE: " + row_level_column.type->getName(), ErrorCodes::LOGICAL_ERROR); } + + block.erase(prewhere_info->row_level_column_name); } if (prewhere_info->prewhere_actions) diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index ed5fc48dad1..bf5fd307b1d 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -271,11 +271,22 @@ MergeTreeReadTaskColumns getReadTaskColumns( { if (prewhere_info->alias_actions) pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); - else if (prewhere_info->row_level_filter) - pre_column_names = prewhere_info->row_level_filter->getRequiredColumns(); - else if (prewhere_info->prewhere_actions) + else + { pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); + if (prewhere_info->row_level_filter) + { + NameSet names(pre_column_names.begin(), pre_column_names.end()); + + for (auto & name : prewhere_info->row_level_filter->getRequiredColumns()) + { + if (names.count(name) == 0) + pre_column_names.push_back(name); + } + } + } + if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); @@ -293,6 +304,13 @@ MergeTreeReadTaskColumns getReadTaskColumns( column_names = post_column_names; } + // std::cerr << "---------- Pre column names\n"; + // for (const auto & col : pre_column_names) + // std::cerr << col << std::endl; + // std::cerr << "----------- Post column names\n"; + // for (const auto & col : column_names) + // std::cerr << col << std::endl; + MergeTreeReadTaskColumns result; if (check_columns) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index a3e4fbf6e6b..e72039f7172 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -33,6 +33,25 @@ static void filterColumns(Columns & columns, const IColumn::Filter & filter) } } +static void filterColumns(Columns & columns, const ColumnPtr & filter) +{ + ConstantFilterDescription const_descr(*filter); + if (const_descr.always_true) + return; + + if (const_descr.always_false) + { + for (auto & col : columns) + if (col) + col = col->cloneEmpty(); + + return; + } + + FilterDescription descr(*filter); + filterColumns(columns, *descr.data); +} + MergeTreeRangeReader::DelayedStream::DelayedStream( size_t from_mark, IMergeTreeReader * merge_tree_reader_) @@ -315,7 +334,7 @@ void MergeTreeRangeReader::ReadResult::setFilterConstFalse() num_rows = 0; } -void MergeTreeRangeReader::ReadResult::optimize(bool can_read_incomplete_granules) +void MergeTreeRangeReader::ReadResult::optimize(bool can_read_incomplete_granules, bool allow_filter_columns) { if (total_rows_per_granule == 0 || filter == nullptr) return; @@ -347,7 +366,7 @@ void MergeTreeRangeReader::ReadResult::optimize(bool can_read_incomplete_granule filter_holder_original = std::move(filter_holder); /// Check if const 1 after shrink - if (countBytesInResultFilter(filter->getData()) + total_zero_rows_in_tails == total_rows_per_granule) + if (allow_filter_columns && countBytesInResultFilter(filter->getData()) + total_zero_rows_in_tails == total_rows_per_granule) { total_rows_per_granule = total_rows_per_granule - total_zero_rows_in_tails; num_rows = total_rows_per_granule; @@ -451,79 +470,32 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con return count; } -void MergeTreeRangeReader::ReadResult::addFilter(const ColumnPtr & new_filter) +void MergeTreeRangeReader::ReadResult::setFilter(const ColumnPtr & new_filter) { + if (!new_filter && filter) + throw Exception("Can't replace existing filter with empty.", ErrorCodes::LOGICAL_ERROR); + if (filter) { - if (!new_filter) - throw Exception("Can't add an empty filter to the existing one.", ErrorCodes::LOGICAL_ERROR); + size_t new_size = new_filter->size(); - const auto new_size = new_filter->size(); if (new_size != total_rows_per_granule) - throw Exception("Can't add the new filter because it's size is " + toString(new_size) + " but " + throw Exception("Can't set filter because it's size is " + toString(new_size) + " but " + toString(total_rows_per_granule) + " rows was read.", ErrorCodes::LOGICAL_ERROR); } ConstantFilterDescription const_description(*new_filter); if (const_description.always_true) - { - if (!filter) - setFilterConstTrue(); - } + setFilterConstTrue(); else if (const_description.always_false) - { clear(); - } else { - FilterDescription description(*new_filter); - auto new_holder = (description.data_holder ? description.data_holder : new_filter); - const auto * new_holder_cast = typeid_cast(new_holder.get()); - - if (!new_holder_cast) - throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); - - if (filter) - { - MutableColumnPtr new_mutable_holder = IColumn::mutate(std::move(new_holder)); - auto * new_mutable_holder_cast = typeid_cast(new_mutable_holder.get()); - - if (!new_mutable_holder_cast) - throw Exception("addFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); - - const auto & data = filter->getData(); - const auto * it = data.begin(); - - auto & new_data = new_mutable_holder_cast->getData(); - auto * n_it = new_data.begin(); - - while (it != data.end() && n_it != new_data.end()) - { - *n_it = (*n_it && *it); - ++it; - ++n_it; - } - - ConstantFilterDescription new_const_description(*new_mutable_holder); - if (new_const_description.always_true) - { - setFilterConstTrue(); - } - else if (new_const_description.always_false) - { - clear(); - } - else - { - filter_holder = std::move(new_mutable_holder); - filter = new_mutable_holder_cast; - } - } - else - { - filter_holder = std::move(new_holder); - filter = new_holder_cast; - } + FilterDescription filter_description(*new_filter); + filter_holder = filter_description.data_holder ? filter_description.data_holder : new_filter; + filter = typeid_cast(filter_holder.get()); + if (!filter) + throw Exception("setFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); } } @@ -565,7 +537,10 @@ MergeTreeRangeReader::MergeTreeRangeReader( prewhere_info->alias_actions->execute(sample_block, true); if (prewhere_info->row_level_filter) + { prewhere_info->row_level_filter->execute(sample_block, true); + sample_block.erase(prewhere_info->row_level_column_name); + } if (prewhere_info->prewhere_actions) prewhere_info->prewhere_actions->execute(sample_block, true); @@ -859,20 +834,76 @@ Columns MergeTreeRangeReader::continueReadingChain(ReadResult & result, size_t & return columns; } +static void checkCombindeFiltersSize(size_t bytes_in_first_filter, size_t second_filter_size) +{ + if (bytes_in_first_filter != second_filter_size) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot combine filters because number of bytes in a first filter ({}) " + "does not match second filter size ({})", bytes_in_first_filter, second_filter_size); +} + +static ColumnPtr combineFilters(ColumnPtr first, ColumnPtr second) +{ + ConstantFilterDescription firsrt_const_descr(*first); + + if (firsrt_const_descr.always_true) + { + checkCombindeFiltersSize(first->size(), second->size()); + return second; + } + + if (firsrt_const_descr.always_false) + { + checkCombindeFiltersSize(0, second->size()); + return first; + } + + auto mut_first = IColumn::mutate(std::move(first)); + FilterDescription firsrt_descr(*mut_first); + + size_t bytes_in_first_filter = countBytesInFilter(*firsrt_descr.data); + checkCombindeFiltersSize(bytes_in_first_filter, second->size()); + + ConstantFilterDescription second_const_descr(*second); + + if (second_const_descr.always_true) + return mut_first; + + if (second_const_descr.always_false) + return second->cloneResized(mut_first->size()); + + FilterDescription second_descr(*second); + auto & first_data = const_cast(*firsrt_descr.data); + const auto * second_data = second_descr.data->data(); + + for (auto & val : first_data) + { + if (val) + { + val = *second_data; + ++second_data; + } + } + + return mut_first; +} + void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { if (!prewhere_info) return; const auto & header = merge_tree_reader->getColumns(); - const auto num_columns = header.size(); + size_t num_columns = header.size(); if (result.columns.size() != num_columns) throw Exception("Invalid number of columns passed to MergeTreeRangeReader. " "Expected " + toString(num_columns) + ", " "got " + toString(result.columns.size()), ErrorCodes::LOGICAL_ERROR); - size_t prewhere_column_pos = 0; + ColumnPtr filter; + ColumnPtr row_level_filter; + size_t prewhere_column_pos; { /// Restore block from columns list. @@ -900,44 +931,47 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (prewhere_info->row_level_filter) { prewhere_info->row_level_filter->execute(block); + auto row_level_filter_pos = block.getPositionByName(prewhere_info->row_level_column_name); + row_level_filter = block.getByPosition(row_level_filter_pos).column; + block.erase(row_level_filter_pos); - const auto filter_column_pos = block.getPositionByName(prewhere_info->row_level_column_name); - result.addFilter(block.getByPosition(filter_column_pos).column); - - result.columns.clear(); - result.columns.reserve(block.columns()); - - for (auto & col : block) - result.columns.emplace_back(std::move(col.column)); - - const auto * result_filter = result.getFilter(); - filterColumns(result.columns, result_filter->getData()); - - auto it = block.begin(); - for (auto & col : result.columns) - it++->column = std::move(col); - - result.columns.clear(); - result.clearFilter(); + auto columns = block.getColumns(); + filterColumns(columns, row_level_filter); + block.setColumns(columns); } prewhere_info->prewhere_actions->execute(block); prewhere_column_pos = block.getPositionByName(prewhere_info->prewhere_column_name); - result.addFilter(block.getByPosition(prewhere_column_pos).column); - - block.getByPosition(prewhere_column_pos).column.reset(); result.columns.clear(); result.columns.reserve(block.columns()); - for (auto & col : block) result.columns.emplace_back(std::move(col.column)); + + filter.swap(result.columns[prewhere_column_pos]); } + if (result.getFilter()) + { + /// TODO: implement for prewhere chain. + /// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter. + throw Exception("MergeTreeRangeReader chain with several prewhere actions in not implemented.", + ErrorCodes::LOGICAL_ERROR); + } + + if (filter && row_level_filter) + { + row_level_filter = combineFilters(std::move(row_level_filter), filter); + result.setFilter(row_level_filter); + + } + else + result.setFilter(filter); + /// If there is a WHERE, we filter in there, and only optimize IO and shrink columns here if (!last_reader_in_chain) - result.optimize(merge_tree_reader->canReadIncompleteGranules()); + result.optimize(merge_tree_reader->canReadIncompleteGranules(), prewhere_info->row_level_filter == nullptr); /// If we read nothing or filter gets optimized to nothing if (result.totalRowsPerGranule() == 0) @@ -962,7 +996,12 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r { /// filter might be shrunk while columns not const auto * result_filter = result.getFilterOriginal(); - filterColumns(result.columns, result_filter->getData()); + + if (row_level_filter) + filterColumns(result.columns, filter); + else + filterColumns(result.columns, result_filter->getData()); + result.need_filter = true; bool has_column = false; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 884d2dbafd1..18075e52bdd 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -153,10 +153,10 @@ public: void addRows(size_t rows) { num_read_rows += rows; } void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); } - /// Apply a filter on top of the existing one (AND'ed) or set it if there isn't any. - void addFilter(const ColumnPtr & new_filter); + /// Set filter or replace old one. Filter must have more zeroes than previous. + void setFilter(const ColumnPtr & new_filter); /// For each granule calculate the number of filtered rows at the end. Remove them and update filter. - void optimize(bool can_read_incomplete_granules); + void optimize(bool can_read_incomplete_granules, bool allow_filter_columns); /// Remove all rows from granules. void clear(); From a8bc2722f43a98bf9bee8cae777d490af62b5f0d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 20 Feb 2021 14:04:32 +0300 Subject: [PATCH 274/510] Fix getHeaderForProcessingStage --- src/Interpreters/getHeaderForProcessingStage.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Interpreters/getHeaderForProcessingStage.cpp b/src/Interpreters/getHeaderForProcessingStage.cpp index 3adbab8413f..b56b90cdf3f 100644 --- a/src/Interpreters/getHeaderForProcessingStage.cpp +++ b/src/Interpreters/getHeaderForProcessingStage.cpp @@ -47,7 +47,10 @@ Block getHeaderForProcessingStage( auto & prewhere_info = *query_info.prewhere_info; if (prewhere_info.row_level_filter) + { prewhere_info.row_level_filter->execute(header); + header.erase(prewhere_info.row_level_column_name); + } if (prewhere_info.prewhere_actions) prewhere_info.prewhere_actions->execute(header); From 89dd15a91df89a3975e68ad3f6d4651f517e33ba Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Feb 2021 14:04:38 +0300 Subject: [PATCH 275/510] Checksum for header and logging, better names --- src/Coordination/Changelog.cpp | 239 +++++++++++++------------- src/Coordination/Changelog.h | 61 ++++--- src/Coordination/NuKeeperLogStore.cpp | 3 +- src/Coordination/NuKeeperLogStore.h | 2 + 4 files changed, 168 insertions(+), 137 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 4a3955e23ab..3d3c1ad230d 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -20,22 +20,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -std::string toString(const ChangelogVersion & version) -{ - if (version == ChangelogVersion::V0) - return "V0"; - - throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown chagelog version {}", static_cast(version)); -} - -ChangelogVersion fromString(const std::string & version_str) -{ - if (version_str == "V0") - return ChangelogVersion::V0; - - throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unknown chagelog version {}", version_str); -} - namespace { @@ -44,11 +28,10 @@ constexpr auto DEFAULT_PREFIX = "changelog"; std::string formatChangelogPath(const std::string & prefix, const ChangelogFileDescription & name) { std::filesystem::path path(prefix); - path /= std::filesystem::path(name.prefix + "_" + std::to_string(name.from_log_idx) + "_" + std::to_string(name.to_log_idx) + ".bin"); + path /= std::filesystem::path(name.prefix + "_" + std::to_string(name.from_log_index) + "_" + std::to_string(name.to_log_index) + ".bin"); return path; } - ChangelogFileDescription getChangelogFileDescription(const std::string & path_str) { std::filesystem::path path(path_str); @@ -60,8 +43,8 @@ ChangelogFileDescription getChangelogFileDescription(const std::string & path_st ChangelogFileDescription result; result.prefix = filename_parts[0]; - result.from_log_idx = parse(filename_parts[1]); - result.to_log_idx = parse(filename_parts[2]); + result.from_log_index = parse(filename_parts[1]); + result.to_log_index = parse(filename_parts[2]); result.path = path_str; return result; } @@ -71,6 +54,17 @@ LogEntryPtr makeClone(const LogEntryPtr & entry) return cs_new(entry->get_term(), nuraft::buffer::clone(entry->get_buf()), entry->get_val_type()); } +Checksum computeRecordChecksum(const ChangelogRecord & record) +{ + const auto * header_start = reinterpret_cast(&record.header); + auto sum = CityHash_v1_0_2::CityHash128(header_start, sizeof(record.header)); + + if (record.header.blob_size != 0) + sum = CityHash_v1_0_2::CityHash128WithSeed(reinterpret_cast(record.blob->data_begin()), record.header.blob_size, sum); + + return sum; +} + } class ChangelogWriter @@ -86,12 +80,9 @@ public: off_t appendRecord(ChangelogRecord && record, bool sync) { off_t result = plain_buf.count(); - writeIntBinary(record.header.version, plain_buf); - writeIntBinary(record.header.index, plain_buf); - writeIntBinary(record.header.term, plain_buf); - writeIntBinary(record.header.value_type, plain_buf); - writeIntBinary(record.header.blob_size, plain_buf); - writeIntBinary(record.header.blob_checksum, plain_buf); + writeIntBinary(computeRecordChecksum(record), plain_buf); + + writePODBinary(record.header, plain_buf); if (record.header.blob_size != 0) plain_buf.write(reinterpret_cast(record.blob->data_begin()), record.blob->size()); @@ -157,7 +148,7 @@ public: , read_buf(filepath) {} - ChangelogReadResult readChangelog(IndexToLogEntry & logs, size_t start_log_idx, IndexToOffset & index_to_offset) + ChangelogReadResult readChangelog(IndexToLogEntry & logs, size_t start_log_index, IndexToOffset & index_to_offset, Poco::Logger * log) { size_t previous_index = 0; ChangelogReadResult result{}; @@ -166,24 +157,31 @@ public: while (!read_buf.eof()) { result.last_position = read_buf.count(); + Checksum record_checksum; + readIntBinary(record_checksum, read_buf); + ChangelogRecord record; - readIntBinary(record.header.version, read_buf); - readIntBinary(record.header.index, read_buf); - readIntBinary(record.header.term, read_buf); - readIntBinary(record.header.value_type, read_buf); - readIntBinary(record.header.blob_size, read_buf); - readIntBinary(record.header.blob_checksum, read_buf); - auto buffer = nuraft::buffer::alloc(record.header.blob_size); - auto * buffer_begin = reinterpret_cast(buffer->data_begin()); - read_buf.readStrict(buffer_begin, record.header.blob_size); + readPODBinary(record.header, read_buf); + if (record.header.version > CURRENT_CHANGELOG_VERSION) + throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported changelog version {} on path {}", record.header.version, filepath); + + if (record.header.blob_size != 0) + { + auto buffer = nuraft::buffer::alloc(record.header.blob_size); + auto * buffer_begin = reinterpret_cast(buffer->data_begin()); + read_buf.readStrict(buffer_begin, record.header.blob_size); + record.blob = buffer; + } + else + record.blob = nullptr; if (previous_index != 0 && previous_index + 1 != record.header.index) throw Exception(ErrorCodes::CORRUPTED_DATA, "Previous log entry {}, next log entry {}, seems like some entries skipped", previous_index, record.header.index); previous_index = record.header.index; - Checksum checksum = CityHash_v1_0_2::CityHash128(buffer_begin, record.header.blob_size); - if (checksum != record.header.blob_checksum) + Checksum checksum = computeRecordChecksum(record); + if (checksum != record_checksum) { throw Exception(ErrorCodes::CHECKSUM_DOESNT_MATCH, "Checksums doesn't match for log {} (version {}), index {}, blob_size {}", @@ -195,10 +193,10 @@ public: result.entries_read += 1; - if (record.header.index < start_log_idx) + if (record.header.index < start_log_index) continue; - auto log_entry = nuraft::cs_new(record.header.term, buffer, record.header.value_type); + auto log_entry = nuraft::cs_new(record.header.term, record.blob, record.header.value_type); logs.emplace(record.header.index, log_entry); index_to_offset[record.header.index] = result.last_position; @@ -206,13 +204,16 @@ public: } catch (const Exception & ex) { + if (ex.code() == ErrorCodes::UNKNOWN_FORMAT_VERSION) + throw ex; + result.error = true; - LOG_WARNING(&Poco::Logger::get("RaftChangelog"), "Cannot completely read changelog on path {}, error: {}", filepath, ex.message()); + LOG_WARNING(log, "Cannot completely read changelog on path {}, error: {}", filepath, ex.message()); } catch (...) { result.error = true; - tryLogCurrentException(&Poco::Logger::get("RaftChangelog")); + tryLogCurrentException(log); } return result; @@ -223,9 +224,10 @@ private: ReadBufferFromFile read_buf; }; -Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval_) +Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval_, Poco::Logger * log_) : changelogs_dir(changelogs_dir_) , rotate_interval(rotate_interval_) + , log(log_) { namespace fs = std::filesystem; if (!fs::exists(changelogs_dir)) @@ -234,96 +236,104 @@ Changelog::Changelog(const std::string & changelogs_dir_, size_t rotate_interval for (const auto & p : fs::directory_iterator(changelogs_dir)) { auto file_description = getChangelogFileDescription(p.path()); - existing_changelogs[file_description.from_log_idx] = file_description; + existing_changelogs[file_description.from_log_index] = file_description; } } -void Changelog::readChangelogAndInitWriter(size_t from_log_idx) +void Changelog::readChangelogAndInitWriter(size_t from_log_index) { - start_index = from_log_idx == 0 ? 1 : from_log_idx; + start_index = from_log_index == 0 ? 1 : from_log_index; size_t total_read = 0; size_t entries_in_last = 0; - size_t incomplete_log_idx = 0; + size_t incomplete_log_index = 0; ChangelogReadResult result{}; - for (const auto & [start_idx, changelog_description] : existing_changelogs) - { - entries_in_last = changelog_description.to_log_idx - changelog_description.from_log_idx + 1; - if (changelog_description.to_log_idx >= from_log_idx) + for (const auto & [start_index, changelog_description] : existing_changelogs) + { + entries_in_last = changelog_description.to_log_index - changelog_description.from_log_index + 1; + + if (changelog_description.to_log_index >= from_log_index) { ChangelogReader reader(changelog_description.path); - result = reader.readChangelog(logs, from_log_idx, index_to_start_pos); + result = reader.readChangelog(logs, from_log_index, index_to_start_pos, log); total_read += result.entries_read; - /// May happen after truncate and crash + /// May happen after truncate, crash or simply unfinished log if (result.entries_read < entries_in_last) { - incomplete_log_idx = start_idx; + incomplete_log_index = start_index; break; } } } - if (incomplete_log_idx != 0) + if (incomplete_log_index != 0) { - for (auto itr = existing_changelogs.upper_bound(incomplete_log_idx); itr != existing_changelogs.end();) + /// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them. + for (auto itr = existing_changelogs.upper_bound(incomplete_log_index); itr != existing_changelogs.end();) { + LOG_WARNING(log, "Removing changelog {}, beacuse it's goes after broken changelog entry", itr->second.path); std::filesystem::remove(itr->second.path); itr = existing_changelogs.erase(itr); } + + /// Continue to write into existing log + if (!existing_changelogs.empty()) + { + auto description = existing_changelogs.rbegin()->second; + LOG_TRACE(log, "Continue to write into {}", description.path); + current_writer = std::make_unique(description.path, WriteMode::Append, description.from_log_index); + current_writer->setEntriesWritten(result.entries_read); + + /// Truncate all broken entries from log + if (result.error) + { + LOG_WARNING(log, "Read finished with error, truncating all broken log entries"); + current_writer->truncateToLength(result.last_position); + } + } } - if (!existing_changelogs.empty() && result.entries_read < entries_in_last) - { - auto description = existing_changelogs.rbegin()->second; - current_writer = std::make_unique(description.path, WriteMode::Append, description.from_log_idx); - current_writer->setEntriesWritten(result.entries_read); - if (result.error) - current_writer->truncateToLength(result.last_position); - } - else - { + /// Start new log if we don't initialize writer from previous log + if (!current_writer) rotate(start_index + total_read); - } } -void Changelog::rotate(size_t new_start_log_idx) +void Changelog::rotate(size_t new_start_log_index) { + //// doesn't exist on init if (current_writer) current_writer->flush(); ChangelogFileDescription new_description; new_description.prefix = DEFAULT_PREFIX; - new_description.from_log_idx = new_start_log_idx; - new_description.to_log_idx = new_start_log_idx + rotate_interval - 1; + new_description.from_log_index = new_start_log_index; + new_description.to_log_index = new_start_log_index + rotate_interval - 1; new_description.path = formatChangelogPath(changelogs_dir, new_description); - existing_changelogs[new_start_log_idx] = new_description; - current_writer = std::make_unique(new_description.path, WriteMode::Rewrite, new_start_log_idx); + + LOG_TRACE(log, "Starting new changelog {}", new_description.path); + existing_changelogs[new_start_log_index] = new_description; + current_writer = std::make_unique(new_description.path, WriteMode::Rewrite, new_start_log_index); } -ChangelogRecord Changelog::buildRecord(size_t index, nuraft::ptr log_entry) +ChangelogRecord Changelog::buildRecord(size_t index, const LogEntryPtr & log_entry) { ChangelogRecordHeader header; + header.version = ChangelogVersion::V0; header.index = index; header.term = log_entry->get_term(); header.value_type = log_entry->get_val_type(); auto buffer = log_entry->get_buf_ptr(); if (buffer) - { header.blob_size = buffer->size(); - header.blob_checksum = CityHash_v1_0_2::CityHash128(reinterpret_cast(buffer->data_begin()), buffer->size()); - } else - { header.blob_size = 0; - header.blob_checksum = std::make_pair(0, 0); - } return ChangelogRecord{header, buffer}; } -void Changelog::appendEntry(size_t index, nuraft::ptr log_entry, bool force_sync) +void Changelog::appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync) { if (!current_writer) throw Exception(ErrorCodes::LOGICAL_ERROR, "Changelog must be initialized before appending records"); @@ -341,13 +351,13 @@ void Changelog::appendEntry(size_t index, nuraft::ptr log_ent logs[index] = makeClone(log_entry); } -void Changelog::writeAt(size_t index, nuraft::ptr log_entry, bool force_sync) +void Changelog::writeAt(size_t index, const LogEntryPtr & log_entry, bool force_sync) { if (index_to_start_pos.count(index) == 0) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write at index {} because changelog doesn't contain it", index); - bool need_rollback = index < current_writer->getStartIndex(); - if (need_rollback) + bool go_to_previous_file = index < current_writer->getStartIndex(); + if (go_to_previous_file) { auto index_changelog = existing_changelogs.lower_bound(index); ChangelogFileDescription description; @@ -357,14 +367,15 @@ void Changelog::writeAt(size_t index, nuraft::ptr log_entry, description = std::prev(index_changelog)->second; current_writer = std::make_unique(description.path, WriteMode::Append, index_changelog->first); - current_writer->setEntriesWritten(description.to_log_idx - description.from_log_idx + 1); + current_writer->setEntriesWritten(description.to_log_index - description.from_log_index + 1); } auto entries_written = current_writer->getEntriesWritten(); current_writer->truncateToLength(index_to_start_pos[index]); - if (need_rollback) + if (go_to_previous_file) { + /// Remove all subsequent files auto to_remove_itr = existing_changelogs.upper_bound(index); for (auto itr = to_remove_itr; itr != existing_changelogs.end();) { @@ -373,11 +384,14 @@ void Changelog::writeAt(size_t index, nuraft::ptr log_entry, } } - /// Rollback in memory state - for (auto itr = logs.lower_bound(index); itr != logs.end();) + /// Remove redundant logs from memory + for (size_t i = index; ; ++i) { - index_to_start_pos.erase(itr->first); - itr = logs.erase(itr); + auto log_itr = logs.find(i); + if (log_itr == logs.end()) + break; + logs.erase(log_itr); + index_to_start_pos.erase(i); entries_written--; } @@ -386,37 +400,32 @@ void Changelog::writeAt(size_t index, nuraft::ptr log_entry, appendEntry(index, log_entry, force_sync); } -void Changelog::compact(size_t up_to_log_idx) +void Changelog::compact(size_t up_to_log_index) { for (auto itr = existing_changelogs.begin(); itr != existing_changelogs.end();) { - if (itr->second.to_log_idx <= up_to_log_idx) + /// Remove all completely outdated changelog files + if (itr->second.to_log_index <= up_to_log_index) { - for (size_t idx = itr->second.from_log_idx; idx <= itr->second.to_log_idx; ++idx) - { - auto index_pos = index_to_start_pos.find(idx); - if (index_pos == index_to_start_pos.end()) - break; - index_to_start_pos.erase(index_pos); - } + + LOG_INFO(log, "Removing changelog {} because of compaction", itr->second.path); + std::erase_if(index_to_start_pos, [right_index = itr->second.to_log_index] (const auto & item) { return item.first <= right_index; }); std::filesystem::remove(itr->second.path); itr = existing_changelogs.erase(itr); } - else + else /// Files are ordered, so all subsequent should exist break; } - auto start = logs.begin(); - auto end = logs.upper_bound(up_to_log_idx); - logs.erase(start, end); - start_index = up_to_log_idx + 1; + start_index = up_to_log_index + 1; + std::erase_if(logs, [up_to_log_index] (const auto & item) { return item.first <= up_to_log_index; }); } LogEntryPtr Changelog::getLastEntry() const { static LogEntryPtr fake_entry = nuraft::cs_new(0, nuraft::buffer::alloc(sizeof(size_t))); - size_t next_idx = getNextEntryIndex() - 1; - auto entry = logs.find(next_idx); + size_t next_index = getNextEntryIndex() - 1; + auto entry = logs.find(next_index); if (entry == logs.end()) return fake_entry; @@ -437,10 +446,10 @@ LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end) return ret; } -LogEntryPtr Changelog::entryAt(size_t idx) +LogEntryPtr Changelog::entryAt(size_t index) { nuraft::ptr src = nullptr; - auto entry = logs.find(idx); + auto entry = logs.find(index); if (entry == logs.end()) return nullptr; @@ -448,12 +457,12 @@ LogEntryPtr Changelog::entryAt(size_t idx) return makeClone(src); } -nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, int32_t cnt) +nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, int32_t count) { std::vector> returned_logs; size_t size_total = 0; - for (size_t i = index; i < index + cnt; ++i) + for (size_t i = index; i < index + count; ++i) { auto entry = logs.find(i); if (entry == logs.end()) @@ -464,9 +473,9 @@ nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, in returned_logs.push_back(buf); } - nuraft::ptr buf_out = nuraft::buffer::alloc(sizeof(int32_t) + cnt * sizeof(int32_t) + size_total); + nuraft::ptr buf_out = nuraft::buffer::alloc(sizeof(int32_t) + count * sizeof(int32_t) + size_total); buf_out->pos(0); - buf_out->put(static_cast(cnt)); + buf_out->put(static_cast(count)); for (auto & entry : returned_logs) { @@ -484,17 +493,17 @@ void Changelog::applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bo for (int i = 0; i < num_logs; ++i) { - size_t cur_idx = index + i; + size_t cur_index = index + i; int buf_size = buffer.get_int(); nuraft::ptr buf_local = nuraft::buffer::alloc(buf_size); buffer.get(buf_local); LogEntryPtr log_entry = nuraft::log_entry::deserialize(*buf_local); - if (i == 0 && logs.count(cur_idx)) - writeAt(cur_idx, log_entry, force_sync); + if (i == 0 && logs.count(cur_index)) + writeAt(cur_index, log_entry, force_sync); else - appendEntry(cur_idx, log_entry, force_sync); + appendEntry(cur_index, log_entry, force_sync); } } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 779d057d285..38679d604de 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -15,60 +15,68 @@ using Checksum = CityHash_v1_0_2::uint128; using LogEntryPtr = nuraft::ptr; using LogEntries = std::vector; using LogEntriesPtr = nuraft::ptr; +using BufferPtr = nuraft::ptr; using IndexToOffset = std::unordered_map; -using IndexToLogEntry = std::map; +using IndexToLogEntry = std::unordered_map; enum class ChangelogVersion : uint8_t { V0 = 0, }; -std::string toString(const ChangelogVersion & version); -ChangelogVersion fromString(const std::string & version_str); - static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0; -struct ChangelogRecordHeader +struct __attribute__((__packed__)) ChangelogRecordHeader { ChangelogVersion version = CURRENT_CHANGELOG_VERSION; - size_t index; + size_t index; /// entry log number size_t term; nuraft::log_val_type value_type; size_t blob_size; - Checksum blob_checksum; }; +/// Changelog record on disk struct ChangelogRecord { ChangelogRecordHeader header; nuraft::ptr blob; }; +/// changelog_fromindex_toindex.bin +/// [fromindex, toindex] <- inclusive struct ChangelogFileDescription { std::string prefix; - size_t from_log_idx; - size_t to_log_idx; + size_t from_log_index; + size_t to_log_index; std::string path; }; class ChangelogWriter; +/// Simpliest changelog with files rotation. +/// No compression, no metadata, just entries with headers one by one +/// Able to read broken files/entries and discard them. class Changelog { public: - Changelog(const std::string & changelogs_dir_, size_t rotate_interval_); + Changelog(const std::string & changelogs_dir_, size_t rotate_interval_, Poco::Logger * log_); - void readChangelogAndInitWriter(size_t from_log_idx); + /// Read changelog from files on changelogs_dir_ skipping all entries before from_log_index + /// Truncate broken entries, remove files after broken entries. + void readChangelogAndInitWriter(size_t from_log_index); - void appendEntry(size_t index, LogEntryPtr log_entry, bool force_sync); + /// Add entry to log with index. Call fsync if force_sync true. + void appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync); - void writeAt(size_t index, LogEntryPtr log_entry, bool force_sync); + /// Write entry at index and truncate all subsequent entries. + void writeAt(size_t index, const LogEntryPtr & log_entry, bool force_sync); - void compact(size_t up_to_log_idx); + /// Remove log files with to_log_index <= up_to_log_index. + void compact(size_t up_to_log_index); size_t getNextEntryIndex() const { @@ -80,16 +88,22 @@ public: return start_index; } + /// Last entry in log, or fake entry with term 0 if log is empty LogEntryPtr getLastEntry() const; - LogEntriesPtr getLogEntriesBetween(size_t start_index, size_t end_idx); + /// Return log entries between [start, end) + LogEntriesPtr getLogEntriesBetween(size_t start_index, size_t end_index); - LogEntryPtr entryAt(size_t idx); + /// Return entry at position index + LogEntryPtr entryAt(size_t index); - nuraft::ptr serializeEntriesToBuffer(size_t index, int32_t cnt); + /// Serialize entries from index into buffer + BufferPtr serializeEntriesToBuffer(size_t index, int32_t count); + /// Apply entries from buffer overriding existing entries void applyEntriesFromBuffer(size_t index, nuraft::buffer & buffer, bool force_sync); + /// Fsync log to disk void flush(); size_t size() const @@ -97,20 +111,25 @@ public: return logs.size(); } + /// Fsync log to disk ~Changelog(); private: - void rotate(size_t new_start_log_idx); + /// Starts new file [new_start_log_index, new_start_log_index + rotate_interval] + void rotate(size_t new_start_log_index); - static ChangelogRecord buildRecord(size_t index, nuraft::ptr log_entry); + /// Pack log_entry into changelog record + static ChangelogRecord buildRecord(size_t index, const LogEntryPtr & log_entry); private: - std::string changelogs_dir; + const std::string changelogs_dir; + const size_t rotate_interval; + Poco::Logger * log; + std::map existing_changelogs; std::unique_ptr current_writer; IndexToOffset index_to_start_pos; - const size_t rotate_interval; IndexToLogEntry logs; size_t start_index = 0; }; diff --git a/src/Coordination/NuKeeperLogStore.cpp b/src/Coordination/NuKeeperLogStore.cpp index 8834bdc4d69..6aba078bb80 100644 --- a/src/Coordination/NuKeeperLogStore.cpp +++ b/src/Coordination/NuKeeperLogStore.cpp @@ -4,7 +4,8 @@ namespace DB { NuKeeperLogStore::NuKeeperLogStore(const std::string & changelogs_path, size_t rotate_interval_, bool force_sync_) - : changelog(changelogs_path, rotate_interval_) + : log(&Poco::Logger::get("NuKeeperLogStore")) + , changelog(changelogs_path, rotate_interval_, log) , force_sync(force_sync_) { } diff --git a/src/Coordination/NuKeeperLogStore.h b/src/Coordination/NuKeeperLogStore.h index 0ff92220316..a94b662fda4 100644 --- a/src/Coordination/NuKeeperLogStore.h +++ b/src/Coordination/NuKeeperLogStore.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB { @@ -43,6 +44,7 @@ public: private: mutable std::mutex changelog_lock; + Poco::Logger * log; Changelog changelog; bool force_sync; }; From e7f792c94d2835676f82fd7942f6f8a591fe7e4d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Feb 2021 14:28:39 +0300 Subject: [PATCH 276/510] Fix typos --- src/Coordination/Changelog.cpp | 2 +- src/Coordination/Changelog.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 3d3c1ad230d..efb0f2798e2 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -272,7 +272,7 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_index) /// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them. for (auto itr = existing_changelogs.upper_bound(incomplete_log_index); itr != existing_changelogs.end();) { - LOG_WARNING(log, "Removing changelog {}, beacuse it's goes after broken changelog entry", itr->second.path); + LOG_WARNING(log, "Removing changelog {}, because it's goes after broken changelog entry", itr->second.path); std::filesystem::remove(itr->second.path); itr = existing_changelogs.erase(itr); } diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 38679d604de..f758edc27ed 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -56,7 +56,7 @@ struct ChangelogFileDescription class ChangelogWriter; -/// Simpliest changelog with files rotation. +/// Simplest changelog with files rotation. /// No compression, no metadata, just entries with headers one by one /// Able to read broken files/entries and discard them. class Changelog From d723f25fbd6474675d4e846c8a187418a540153a Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Sat, 20 Feb 2021 16:51:31 +0300 Subject: [PATCH 277/510] delete extra text --- docs/en/sql-reference/functions/array-functions.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/functions/array-functions.md b/docs/en/sql-reference/functions/array-functions.md index 531c5e5be49..c9c418d57a4 100644 --- a/docs/en/sql-reference/functions/array-functions.md +++ b/docs/en/sql-reference/functions/array-functions.md @@ -1378,7 +1378,6 @@ SELECT arrayMax([1, 2, 4]) AS res; Result: -``` text ```text ┌─res─┐ │ 4 │ From 8a876b9510da3fcd23dd3f3efa308d7cb52a6410 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 20 Feb 2021 17:19:11 +0300 Subject: [PATCH 278/510] Fix aliases for row level actions. --- src/Interpreters/InterpreterSelectQuery.cpp | 55 +++++++++++---------- 1 file changed, 29 insertions(+), 26 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 54481cbe873..835c0c0e50f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1410,30 +1410,30 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (storage) { /// Append columns from the table filter to required - if (row_policy_filter) - { - ActionsDAG * row_policy_dag = nullptr; - if (expressions.filter_info) - row_policy_dag = expressions.filter_info->actions.get(); - else if (expressions.prewhere_info) - { - if (expressions.prewhere_info->row_level_filter_actions) - row_policy_dag = expressions.prewhere_info->row_level_filter_actions.get(); - else if (expressions.prewhere_info->prewhere_actions) - row_policy_dag = expressions.prewhere_info->prewhere_actions.get(); - } + // if (row_policy_filter) + // { + // ActionsDAG * row_policy_dag = nullptr; + // if (expressions.filter_info) + // row_policy_dag = expressions.filter_info->actions.get(); + // else if (expressions.prewhere_info) + // { + // if (expressions.prewhere_info->row_level_filter_actions) + // row_policy_dag = expressions.prewhere_info->row_level_filter_actions.get(); + // else if (expressions.prewhere_info->prewhere_actions) + // row_policy_dag = expressions.prewhere_info->prewhere_actions.get(); + // } - if (row_policy_dag) - { - auto required_columns_from_filter = row_policy_dag->getRequiredColumns(); + // if (row_policy_dag) + // { + // auto required_columns_from_filter = row_policy_dag->getRequiredColumns(); - for (const auto & column : required_columns_from_filter) - { - if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) - required_columns.push_back(column.name); - } - } - } + // for (const auto & column : required_columns_from_filter) + // { + // if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) + // required_columns.push_back(column.name); + // } + // } + // } /// Detect, if ALIAS columns are required for query execution auto alias_columns_required = false; @@ -1463,11 +1463,14 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (prewhere_info) { /// Get some columns directly from PREWHERE expression actions - auto prewhere_required_columns = ( - prewhere_info->row_level_filter_actions ? - prewhere_info->row_level_filter_actions : - prewhere_info->prewhere_actions)->getRequiredColumns().getNames(); + auto prewhere_required_columns = prewhere_info->prewhere_actions->getRequiredColumns().getNames(); required_columns_from_prewhere.insert(prewhere_required_columns.begin(), prewhere_required_columns.end()); + + if (prewhere_info->row_level_filter_actions) + { + auto row_level_required_columns = prewhere_info->row_level_filter_actions->getRequiredColumns().getNames(); + required_columns_from_prewhere.insert(row_level_required_columns.begin(), row_level_required_columns.end()); + } } /// Expression, that contains all raw required columns From 6cc2fb5e9f32517ebd29104c56f3ee07517f462d Mon Sep 17 00:00:00 2001 From: vdimir Date: Sat, 20 Feb 2021 18:00:59 +0300 Subject: [PATCH 279/510] Try to fix race in storage join: block parralel inserts --- src/Functions/FunctionJoinGet.cpp | 8 ++++---- src/Functions/FunctionJoinGet.h | 15 +++++++-------- src/Interpreters/HashJoin.cpp | 11 +---------- src/Interpreters/HashJoin.h | 9 ++++++++- src/Storages/StorageJoin.cpp | 10 +++++++--- src/Storages/StorageJoin.h | 18 ++++++++++++++---- 6 files changed, 41 insertions(+), 30 deletions(-) diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index 6b15bf821b2..3a2649c11a8 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -25,7 +25,7 @@ ColumnPtr ExecutableFunctionJoinGet::execute(const ColumnsWithTypeAndNa auto key = arguments[i]; keys.emplace_back(std::move(key)); } - return join->joinGet(keys, result_columns).column; + return join->join->joinGet(keys, result_columns).column; } template @@ -87,13 +87,13 @@ FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTyp + ", should be greater or equal to 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); - auto join = storage_join->getJoin(); + auto join_holder = storage_join->getJoin(); DataTypes data_types(arguments.size() - 2); for (size_t i = 2; i < arguments.size(); ++i) data_types[i - 2] = arguments[i].type; - auto return_type = join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); + auto return_type = join_holder->join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - return std::make_unique>(table_lock, storage_join, join, attr_name, data_types, return_type); + return std::make_unique>(table_lock, join_holder, attr_name, data_types, return_type); } void registerFunctionJoinGet(FunctionFactory & factory) diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index 27f348e9698..820c6cd3fa2 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -9,13 +9,14 @@ namespace DB class Context; class HashJoin; +class HashJoinHolder; using HashJoinPtr = std::shared_ptr; template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(HashJoinPtr join_, const DB::Block & result_columns_) + ExecutableFunctionJoinGet(std::shared_ptr join_, const DB::Block & result_columns_) : join(std::move(join_)), result_columns(result_columns_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; @@ -29,7 +30,7 @@ public: String getName() const override { return name; } private: - HashJoinPtr join; + std::shared_ptr join; DB::Block result_columns; }; @@ -39,12 +40,11 @@ class FunctionJoinGet final : public IFunctionBaseImpl public: static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; - FunctionJoinGet(TableLockHolder table_lock_, StoragePtr storage_join_, - HashJoinPtr join_, String attr_name_, + FunctionJoinGet(TableLockHolder table_lock_, + std::shared_ptr join_, String attr_name_, DataTypes argument_types_, DataTypePtr return_type_) : table_lock(std::move(table_lock_)) - , storage_join(std::move(storage_join_)) - , join(std::move(join_)) + , join(join_) , attr_name(std::move(attr_name_)) , argument_types(std::move(argument_types_)) , return_type(std::move(return_type_)) @@ -60,8 +60,7 @@ public: private: TableLockHolder table_lock; - StoragePtr storage_join; - HashJoinPtr join; + std::shared_ptr join; const String attr_name; DataTypes argument_types; DataTypePtr return_type; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 5c50b53e2ca..cd158241860 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -423,19 +423,16 @@ bool HashJoin::empty() const size_t HashJoin::getTotalByteCount() const { - std::shared_lock lock(data->rwlock); return getTotalByteCountLocked(); } size_t HashJoin::getTotalRowCount() const { - std::shared_lock lock(data->rwlock); return getTotalRowCountLocked(); } bool HashJoin::alwaysReturnsEmptySet() const { - std::shared_lock lock(data->rwlock); return isInnerOrRight(getKind()) && data->empty && !overDictionary(); } @@ -652,7 +649,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) size_t total_bytes = 0; { - std::unique_lock lock(data->rwlock); + assert(storage_join_lock.mutex() == nullptr); data->blocks.emplace_back(std::move(structured_block)); Block * stored_block = &data->blocks.back(); @@ -1219,8 +1216,6 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { - std::shared_lock lock(data->rwlock); - size_t num_keys = data_types.size(); if (right_table_keys.columns() != num_keys) throw Exception( @@ -1273,8 +1268,6 @@ ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & b // TODO: return array of values when strictness == ASTTableJoin::Strictness::All ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { - std::shared_lock lock(data->rwlock); - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && kind == ASTTableJoin::Kind::Left) { @@ -1287,8 +1280,6 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { - std::shared_lock lock(data->rwlock); - const Names & key_names_left = table_join->keyNamesLeft(); JoinCommon::checkTypesOfKeys(block, key_names_left, right_table_keys, key_names_right); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 06ce7559f31..06e07dc10dd 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -308,7 +308,7 @@ public: { /// Protect state for concurrent use in insertFromBlock and joinBlock. /// @note that these methods could be called simultaneously only while use of StorageJoin. - mutable std::shared_mutex rwlock; +// mutable std::shared_mutex rwlock; Type type = Type::EMPTY; bool empty = true; @@ -322,6 +322,11 @@ public: Arena pool; }; + void setLock(std::shared_mutex & rwlock) + { + storage_join_lock = std::shared_lock(rwlock); + } + void reuseJoinedData(const HashJoin & join); std::shared_ptr getJoinedData() const @@ -371,6 +376,8 @@ private: Block totals; + std::shared_lock storage_join_lock; + void init(Type type_); const Block & savedBlockSample() const { return data->sample_block; } diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index 8d4f0b3b3be..f130316566f 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -97,11 +97,17 @@ HashJoinPtr StorageJoin::getJoin(std::shared_ptr analyzed_join) const HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns()); join_clone->reuseJoinedData(*join); + join_clone->setLock(rwlock); + return join_clone; } -void StorageJoin::insertBlock(const Block & block) { join->addJoinedBlock(block, true); } +void StorageJoin::insertBlock(const Block & block) +{ + std::unique_lock lock(rwlock); + join->addJoinedBlock(block, true); +} size_t StorageJoin::getSize() const { return join->getTotalRowCount(); } std::optional StorageJoin::totalRows(const Settings &) const { return join->getTotalRowCount(); } @@ -267,7 +273,6 @@ public: JoinSource(const HashJoin & parent_, UInt64 max_block_size_, Block sample_block_) : SourceWithProgress(sample_block_) , parent(parent_) - , lock(parent.data->rwlock) , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) { @@ -312,7 +317,6 @@ protected: private: const HashJoin & parent; - std::shared_lock lock; UInt64 max_block_size; Block sample_block; Block restored_block; /// sample_block with parent column types diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index c453c036b65..6d3ec2710c9 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -14,6 +14,18 @@ class TableJoin; class HashJoin; using HashJoinPtr = std::shared_ptr; +class HashJoinHolder +{ + std::shared_lock lock; +public: + HashJoinPtr join; + + HashJoinHolder(std::shared_mutex & rwlock, HashJoinPtr join_) + : lock(rwlock) + , join(join_) + { + } +}; /** Allows you save the state for later use on the right side of the JOIN. * When inserted into a table, the data will be inserted into the state, @@ -31,12 +43,9 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; /// Access the innards. - HashJoinPtr & getJoin() { return join; } + std::shared_ptr getJoin() { return std::make_shared(rwlock, join); } HashJoinPtr getJoin(std::shared_ptr analyzed_join) const; - /// Verify that the data structure is suitable for implementing this type of JOIN. - void assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const; - Pipe read( const Names & column_names, const StorageMetadataPtr & /*metadata_snapshot*/, @@ -60,6 +69,7 @@ private: std::shared_ptr table_join; HashJoinPtr join; + mutable std::shared_mutex rwlock; void insertBlock(const Block & block) override; void finishInsert() override {} From 0c2cf3cf30b707fdf46c88760c931c194a086d2d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Feb 2021 18:36:56 +0300 Subject: [PATCH 280/510] Calculate checksum with siphash --- src/Coordination/Changelog.cpp | 51 ++++++++++++++++++++++------------ src/Coordination/Changelog.h | 9 +++--- 2 files changed, 38 insertions(+), 22 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index efb0f2798e2..adf367c565d 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include namespace DB @@ -56,13 +57,15 @@ LogEntryPtr makeClone(const LogEntryPtr & entry) Checksum computeRecordChecksum(const ChangelogRecord & record) { - const auto * header_start = reinterpret_cast(&record.header); - auto sum = CityHash_v1_0_2::CityHash128(header_start, sizeof(record.header)); - + SipHash hash; + hash.update(record.header.version); + hash.update(record.header.index); + hash.update(record.header.term); + hash.update(record.header.value_type); + hash.update(record.header.blob_size); if (record.header.blob_size != 0) - sum = CityHash_v1_0_2::CityHash128WithSeed(reinterpret_cast(record.blob->data_begin()), record.header.blob_size, sum); - - return sum; + hash.update(reinterpret_cast(record.blob->data_begin()), record.blob->size()); + return hash.get64(); } } @@ -82,7 +85,11 @@ public: off_t result = plain_buf.count(); writeIntBinary(computeRecordChecksum(record), plain_buf); - writePODBinary(record.header, plain_buf); + writeIntBinary(record.header.version, plain_buf); + writeIntBinary(record.header.index, plain_buf); + writeIntBinary(record.header.term, plain_buf); + writeIntBinary(record.header.value_type, plain_buf); + writeIntBinary(record.header.blob_size, plain_buf); if (record.header.blob_size != 0) plain_buf.write(reinterpret_cast(record.blob->data_begin()), record.blob->size()); @@ -160,8 +167,14 @@ public: Checksum record_checksum; readIntBinary(record_checksum, read_buf); + /// Initialization is required, otherwise checksums may fail ChangelogRecord record; - readPODBinary(record.header, read_buf); + readIntBinary(record.header.version, read_buf); + readIntBinary(record.header.index, read_buf); + readIntBinary(record.header.term, read_buf); + readIntBinary(record.header.value_type, read_buf); + readIntBinary(record.header.blob_size, read_buf); + if (record.header.version > CURRENT_CHANGELOG_VERSION) throw Exception(ErrorCodes::UNKNOWN_FORMAT_VERSION, "Unsupported changelog version {} on path {}", record.header.version, filepath); @@ -248,7 +261,7 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_index) size_t incomplete_log_index = 0; ChangelogReadResult result{}; - for (const auto & [start_index, changelog_description] : existing_changelogs) + for (const auto & [changelog_start_index, changelog_description] : existing_changelogs) { entries_in_last = changelog_description.to_log_index - changelog_description.from_log_index + 1; @@ -261,7 +274,7 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_index) /// May happen after truncate, crash or simply unfinished log if (result.entries_read < entries_in_last) { - incomplete_log_index = start_index; + incomplete_log_index = changelog_start_index; break; } } @@ -319,18 +332,20 @@ void Changelog::rotate(size_t new_start_log_index) ChangelogRecord Changelog::buildRecord(size_t index, const LogEntryPtr & log_entry) { - ChangelogRecordHeader header; - header.version = ChangelogVersion::V0; - header.index = index; - header.term = log_entry->get_term(); - header.value_type = log_entry->get_val_type(); + ChangelogRecord record; + record.header.version = ChangelogVersion::V0; + record.header.index = index; + record.header.term = log_entry->get_term(); + record.header.value_type = log_entry->get_val_type(); auto buffer = log_entry->get_buf_ptr(); if (buffer) - header.blob_size = buffer->size(); + record.header.blob_size = buffer->size(); else - header.blob_size = 0; + record.header.blob_size = 0; - return ChangelogRecord{header, buffer}; + record.blob = buffer; + + return record; } void Changelog::appendEntry(size_t index, const LogEntryPtr & log_entry, bool force_sync) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index f758edc27ed..0f67c2a9a7d 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -10,7 +10,7 @@ namespace DB { -using Checksum = CityHash_v1_0_2::uint128; +using Checksum = UInt64; using LogEntryPtr = nuraft::ptr; using LogEntries = std::vector; @@ -27,7 +27,7 @@ enum class ChangelogVersion : uint8_t static constexpr auto CURRENT_CHANGELOG_VERSION = ChangelogVersion::V0; -struct __attribute__((__packed__)) ChangelogRecordHeader +struct ChangelogRecordHeader { ChangelogVersion version = CURRENT_CHANGELOG_VERSION; size_t index; /// entry log number @@ -115,12 +115,13 @@ public: ~Changelog(); private: + /// Pack log_entry into changelog record + static ChangelogRecord buildRecord(size_t index, const LogEntryPtr & log_entry); /// Starts new file [new_start_log_index, new_start_log_index + rotate_interval] void rotate(size_t new_start_log_index); - /// Pack log_entry into changelog record - static ChangelogRecord buildRecord(size_t index, const LogEntryPtr & log_entry); + private: const std::string changelogs_dir; From 9f520f42c117e33ad107f9ea33465e11e2cf26e3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Feb 2021 18:56:55 +0300 Subject: [PATCH 281/510] Fix style --- src/Coordination/Changelog.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Coordination/Changelog.h b/src/Coordination/Changelog.h index 0f67c2a9a7d..be38915066d 100644 --- a/src/Coordination/Changelog.h +++ b/src/Coordination/Changelog.h @@ -121,8 +121,6 @@ private: /// Starts new file [new_start_log_index, new_start_log_index + rotate_interval] void rotate(size_t new_start_log_index); - - private: const std::string changelogs_dir; const size_t rotate_interval; From 48e188681c88b88c11924f98976993d500fbb1d4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 20 Feb 2021 16:05:33 +0300 Subject: [PATCH 282/510] do not start mutation for alters with wrong type conversion --- src/Storages/MergeTree/MergeTreeData.cpp | 23 ++++++++++++++----- .../01732_alters_bad_conversions.reference | 4 ++++ .../01732_alters_bad_conversions.sql | 17 ++++++++++++++ 3 files changed, 38 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/01732_alters_bad_conversions.reference create mode 100644 tests/queries/0_stateless/01732_alters_bad_conversions.sql diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index a0d23b8ab22..b09f068f509 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -1482,6 +1483,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S for (const auto & column : old_metadata.getColumns().getAllPhysical()) old_types.emplace(column.name, column.type.get()); + NamesAndTypesList columns_to_check_conversion; for (const AlterCommand & command : commands) { /// Just validate partition expression @@ -1571,9 +1573,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S throw Exception("ALTER of key column " + backQuoteIfNeed(command.column_name) + " is forbidden", ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); - if (columns_alter_type_check_safe_for_partition.count(command.column_name)) + if (command.type == AlterCommand::MODIFY_COLUMN) { - if (command.type == AlterCommand::MODIFY_COLUMN) + if (columns_alter_type_check_safe_for_partition.count(command.column_name)) { auto it = old_types.find(command.column_name); @@ -1584,11 +1586,8 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S + " is not safe because it can change the representation of partition key", ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); } - } - if (columns_alter_type_metadata_only.count(command.column_name)) - { - if (command.type == AlterCommand::MODIFY_COLUMN) + if (columns_alter_type_metadata_only.count(command.column_name)) { auto it = old_types.find(command.column_name); assert(it != old_types.end()); @@ -1598,6 +1597,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S + " is not safe because it can change the representation of primary key", ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); } + + if (old_metadata.getColumns().has(command.column_name)) + { + columns_to_check_conversion.push_back( + new_metadata.getColumns().getPhysical(command.column_name)); + } } } } @@ -1605,6 +1610,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S checkProperties(new_metadata, old_metadata); checkTTLExpressions(new_metadata, old_metadata); + if (!columns_to_check_conversion.empty()) + { + auto old_header = old_metadata.getSampleBlock(); + performRequiredConversions(old_header, columns_to_check_conversion, global_context); + } + if (old_metadata.hasSettingsChanges()) { const auto current_changes = old_metadata.getSettingsChanges()->as().changes; diff --git a/tests/queries/0_stateless/01732_alters_bad_conversions.reference b/tests/queries/0_stateless/01732_alters_bad_conversions.reference new file mode 100644 index 00000000000..5f570c78579 --- /dev/null +++ b/tests/queries/0_stateless/01732_alters_bad_conversions.reference @@ -0,0 +1,4 @@ +CREATE TABLE default.bad_conversions\n(\n `a` UInt32\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +0 +CREATE TABLE default.bad_conversions_2\n(\n `e` Enum8(\'foo\' = 1, \'bar\' = 2)\n)\nENGINE = MergeTree\nORDER BY tuple()\nSETTINGS index_granularity = 8192 +0 diff --git a/tests/queries/0_stateless/01732_alters_bad_conversions.sql b/tests/queries/0_stateless/01732_alters_bad_conversions.sql new file mode 100644 index 00000000000..27da5242368 --- /dev/null +++ b/tests/queries/0_stateless/01732_alters_bad_conversions.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS bad_conversions; +DROP TABLE IF EXISTS bad_conversions_2; + +CREATE TABLE bad_conversions (a UInt32) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO bad_conversions VALUES (1); +ALTER TABLE bad_conversions MODIFY COLUMN a Array(String); -- { serverError 53 } +SHOW CREATE TABLE bad_conversions; +SELECT count() FROM system.mutations WHERE table = 'bad_conversions' AND database = currentDatabase(); + +CREATE TABLE bad_conversions_2 (e Enum('foo' = 1, 'bar' = 2)) ENGINE = MergeTree ORDER BY tuple(); +INSERT INTO bad_conversions_2 VALUES (1); +ALTER TABLE bad_conversions_2 MODIFY COLUMN e Enum('bar' = 1, 'foo' = 2); -- { serverError 70 } +SHOW CREATE TABLE bad_conversions_2; +SELECT count() FROM system.mutations WHERE table = 'bad_conversions_2' AND database = currentDatabase(); + +DROP TABLE IF EXISTS bad_conversions; +DROP TABLE IF EXISTS bad_conversions_2; From f0396661b3cf74b98ea2b562d96edb18949e9df8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 20 Feb 2021 19:13:36 +0300 Subject: [PATCH 283/510] Refactor ActionsDAG::splitActionsForFilter --- src/Interpreters/ActionsDAG.cpp | 411 ++++++++++++++++++-------------- src/Interpreters/ActionsDAG.h | 2 + 2 files changed, 228 insertions(+), 185 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 8b6013a4365..b3f86313a1c 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1212,112 +1212,120 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co return split(split_nodes); } -ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs) +namespace { - std::unordered_map> inputs_map; - for (const auto & input : inputs) - inputs_map[input->result_name].emplace_back(input); - std::unordered_set allowed_nodes; - for (const auto & name : available_inputs) +struct ConjinctionNodes +{ + std::unordered_set allowed; + std::unordered_set rejected; +}; + +/// Take a node which result is predicate. +/// Assuming predicate is a conjunction (probably, trivial). +/// Find separate conjunctions nodes. Split nodes into allowed and rejected sets. +/// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set. +ConjinctionNodes getConjinctionNodes(ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) +{ + ConjinctionNodes conjunction; + + struct Frame { - auto & inputs_list = inputs_map[name]; - if (inputs_list.empty()) - continue; + ActionsDAG::Node * node; + bool is_predicate = false; + size_t next_child_to_visit = 0; + size_t num_allowed_children = 0; + }; - allowed_nodes.emplace(inputs_list.front()); - inputs_list.pop_front(); - } - - auto it = index.begin(); - for (; it != index.end(); ++it) - if ((*it)->result_name == filter_name) - break; - - if (it == index.end()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Index for ActionsDAG does not contain filter column name {}. DAG:\n{}", - filter_name, dumpDAG()); - - std::unordered_set selected_predicates; - std::unordered_set other_predicates; + std::stack stack; + std::unordered_set visited_nodes; + stack.push(Frame{.node = predicate, .is_predicate = true}); + visited_nodes.insert(predicate); + while (!stack.empty()) { - struct Frame + auto & cur = stack.top(); + bool is_conjunction = cur.is_predicate + && cur.node->type == ActionsDAG::ActionType::FUNCTION + && cur.node->function_base->getName() == "and"; + + /// At first, visit all children. + while (cur.next_child_to_visit < cur.node->children.size()) { - Node * node; - bool is_predicate = false; - size_t next_child_to_visit = 0; - size_t num_allowed_children = 0; - }; + auto * child = cur.node->children[cur.next_child_to_visit]; - std::stack stack; - std::unordered_set visited_nodes; - - stack.push(Frame{.node = *it, .is_predicate = true}); - visited_nodes.insert(*it); - while (!stack.empty()) - { - auto & cur = stack.top(); - bool is_conjunction = cur.is_predicate - && cur.node->type == ActionType::FUNCTION - && cur.node->function_base->getName() == "and"; - - /// At first, visit all children. - while (cur.next_child_to_visit < cur.node->children.size()) + if (visited_nodes.count(child) == 0) { - auto * child = cur.node->children[cur.next_child_to_visit]; - - if (visited_nodes.count(child) == 0) - { - visited_nodes.insert(child); - stack.push({.node = child, .is_predicate = is_conjunction}); - break; - } - - if (allowed_nodes.contains(child)) - ++cur.num_allowed_children; - ++cur.next_child_to_visit; + visited_nodes.insert(child); + stack.push({.node = child, .is_predicate = is_conjunction}); + break; } - if (cur.next_child_to_visit == cur.node->children.size()) - { - if (cur.num_allowed_children == cur.node->children.size()) - { - if (cur.node->type != ActionType::ARRAY_JOIN && cur.node->type != ActionType::INPUT) - allowed_nodes.emplace(cur.node); - } - else if (is_conjunction) - { - for (auto * child : cur.node->children) - if (allowed_nodes.count(child)) - selected_predicates.insert(child); - } - else if (cur.is_predicate) - { - other_predicates.insert(cur.node); - } + if (allowed_nodes.contains(child)) + ++cur.num_allowed_children; + ++cur.next_child_to_visit; + } - stack.pop(); + if (cur.next_child_to_visit == cur.node->children.size()) + { + if (cur.num_allowed_children == cur.node->children.size()) + { + if (cur.node->type != ActionsDAG::ActionType::ARRAY_JOIN && cur.node->type != ActionsDAG::ActionType::INPUT) + allowed_nodes.emplace(cur.node); } + else if (is_conjunction) + { + for (auto * child : cur.node->children) + if (allowed_nodes.count(child)) + conjunction.allowed.insert(child); + } + else if (cur.is_predicate) + { + conjunction.rejected.insert(cur.node); + } + + stack.pop(); } } - if (selected_predicates.empty()) + if (conjunction.allowed.empty()) { - if (allowed_nodes.count(*it)) - selected_predicates.insert(*it); - else - return nullptr; + if (allowed_nodes.count(predicate)) + conjunction.allowed.insert(predicate); } - // std::cerr << "************* Selectecd predicates\n"; - // for (const auto * p : selected_predicates) - // std::cerr << p->result_name << std::endl; + return conjunction; +} - // std::cerr << "............. Other predicates\n"; - // for (const auto * p : other_predicates) - // std::cerr << p->result_name << std::endl; +ColumnsWithTypeAndName prepareFunctionArguments(const std::vector nodes) +{ + ColumnsWithTypeAndName arguments; + arguments.reserve(nodes.size()); + + for (const auto * child : nodes) + { + ColumnWithTypeAndName argument; + argument.column = child->column; + argument.type = child->result_type; + argument.name = child->result_name; + + arguments.emplace_back(std::move(argument)); + } + + return arguments; +} + +} + +/// Create actions which calculate conjunction of selected nodes. +/// Assume conjunction nodes are predicates (and may be used as arguments of function AND). +/// +/// Result actions add single column with conjunction result (it is always last in index). +/// No other columns are added or removed. +ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(std::unordered_set conjunction) +{ + if (conjunction.empty()) + return nullptr; auto actions = cloneEmpty(); actions->settings.project_input = false; @@ -1327,82 +1335,128 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, std::make_unique( std::make_shared())); - std::unordered_map nodes_mapping; + std::unordered_map nodes_mapping; + struct Frame { - struct Frame + const ActionsDAG::Node * node; + size_t next_child_to_visit = 0; + }; + + std::stack stack; + + /// DFS. Clone actions. + for (const auto * predicate : conjunction) + { + if (nodes_mapping.count(predicate)) + continue; + + stack.push({.node = predicate}); + while (!stack.empty()) { - const Node * node; - size_t next_child_to_visit = 0; - }; - - std::stack stack; - - for (const auto * predicate : selected_predicates) - { - if (nodes_mapping.count(predicate)) - continue; - - stack.push({.node = predicate}); - while (!stack.empty()) + auto & cur = stack.top(); + /// At first, visit all children. + while (cur.next_child_to_visit < cur.node->children.size()) { - auto & cur = stack.top(); - /// At first, visit all children. - while (cur.next_child_to_visit < cur.node->children.size()) + auto * child = cur.node->children[cur.next_child_to_visit]; + + if (nodes_mapping.count(child) == 0) { - auto * child = cur.node->children[cur.next_child_to_visit]; - - if (nodes_mapping.count(child) == 0) - { - stack.push({.node = child}); - break; - } - - ++cur.next_child_to_visit; + stack.push({.node = child}); + break; } - if (cur.next_child_to_visit == cur.node->children.size()) + ++cur.next_child_to_visit; + } + + if (cur.next_child_to_visit == cur.node->children.size()) + { + auto & node = actions->nodes.emplace_back(*cur.node); + nodes_mapping[cur.node] = &node; + + for (auto & child : node.children) + child = nodes_mapping[child]; + + if (node.type == ActionType::INPUT) { - auto & node = actions->nodes.emplace_back(*cur.node); - nodes_mapping[cur.node] = &node; - - for (auto & child : node.children) - child = nodes_mapping[child]; - - if (node.type == ActionType::INPUT) - { - actions->inputs.emplace_back(&node); - actions->index.insert(&node); - } - - stack.pop(); + actions->inputs.emplace_back(&node); + actions->index.insert(&node); } + + stack.pop(); } } - - Node * result_predicate = nodes_mapping[*selected_predicates.begin()]; - - if (selected_predicates.size() > 1) - { - std::vector args; - args.reserve(selected_predicates.size()); - for (const auto * predicate : selected_predicates) - args.emplace_back(nodes_mapping[predicate]); - - result_predicate = &actions->addFunction(func_builder_and, args, {}, true, false); - } - - actions->index.insert(result_predicate); } - if (selected_predicates.count(*it)) + Node * result_predicate = nodes_mapping[*conjunction.begin()]; + + if (conjunction.size() > 1) + { + std::vector args; + args.reserve(conjunction.size()); + for (const auto * predicate : conjunction) + args.emplace_back(nodes_mapping[predicate]); + + result_predicate = &actions->addFunction(func_builder_and, args, {}, true, false); + } + + actions->index.insert(result_predicate); + return actions; +} + +ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, bool can_remove_filter, const Names & available_inputs) +{ + Node * predicate; + + { + auto it = index.begin(); + for (; it != index.end(); ++it) + if ((*it)->result_name == filter_name) + break; + + if (it == index.end()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Index for ActionsDAG does not contain filter column name {}. DAG:\n{}", + filter_name, dumpDAG()); + + predicate = *it; + } + + std::unordered_set allowed_nodes; + + /// Get input nodes from available_inputs names. + { + std::unordered_map> inputs_map; + for (const auto & input : inputs) + inputs_map[input->result_name].emplace_back(input); + + for (const auto & name : available_inputs) + { + auto & inputs_list = inputs_map[name]; + if (inputs_list.empty()) + continue; + + allowed_nodes.emplace(inputs_list.front()); + inputs_list.pop_front(); + } + } + + auto conjunction = getConjinctionNodes(predicate, allowed_nodes); + auto actions = cloneActionsForConjunction(conjunction.allowed); + if (!actions) + return nullptr; + + /// Now, when actions are created, update current DAG. + + if (conjunction.allowed.count(predicate)) { /// The whole predicate was split. if (can_remove_filter) { + /// If filter column is not needed, remove it from index. for (auto i = index.begin(); i != index.end(); ++i) { - if (*i == *it) + if (*i == predicate) { index.remove(i); break; @@ -1411,84 +1465,71 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, } else { + /// Replace predicate result to constant 1. Node node; node.type = ActionType::COLUMN; - node.result_name = std::move((*it)->result_name); - node.result_type = std::move((*it)->result_type); + node.result_name = std::move(predicate->result_name); + node.result_type = std::move(predicate->result_type); node.column = node.result_type->createColumnConst(0, 1); - *(*it) = std::move(node); + *predicate = std::move(node); } removeUnusedActions(false); } - else if ((*it)->type == ActionType::FUNCTION && (*it)->function_base->getName() == "and") + else { - std::vector new_children(other_predicates.begin(), other_predicates.end()); + /// Predicate is conjunction, where both allowed and rejected sets are not empty. + /// Replace this node to conjunction of rejected predicates. + + std::vector new_children(conjunction.rejected.begin(), conjunction.rejected.end()); if (new_children.size() == 1) { - if (new_children.front()->result_type->equals(*((*it)->result_type))) + /// Rejected set has only one predicate. + if (new_children.front()->result_type->equals(*predicate->result_type)) { + /// If it's type is same, just add alias. Node node; node.type = ActionType::ALIAS; - node.result_name = (*it)->result_name; - node.result_type = (*it)->result_type; + node.result_name = predicate->result_name; + node.result_type = predicate->result_type; node.children.swap(new_children); - *(*it) = std::move(node); + *predicate = std::move(node); } else { + /// If type is different, cast column. + /// This case is possible, cause AND can use any numeric type as argument. Node node; node.type = ActionType::COLUMN; - node.result_name = (*it)->result_type->getName(); + node.result_name = predicate->result_type->getName(); node.column = DataTypeString().createColumnConst(0, node.result_name); node.result_type = std::make_shared(); auto * right_arg = &nodes.emplace_back(std::move(node)); auto * left_arg = new_children.front(); - - (*it)->children = {left_arg, right_arg}; - ColumnsWithTypeAndName arguments; - arguments.reserve((*it)->children.size()); - - for (const auto * child : (*it)->children) - { - ColumnWithTypeAndName argument; - argument.column = child->column; - argument.type = child->result_type; - argument.name = child->result_name; - - arguments.emplace_back(std::move(argument)); - } + predicate->children = {left_arg, right_arg}; + auto arguments = prepareFunctionArguments(predicate->children); FunctionOverloadResolverPtr func_builder_cast = std::make_shared( CastOverloadResolver::createImpl(false)); - (*it)->function_builder = func_builder_cast; - (*it)->function_base = (*it)->function_builder->build(arguments); - (*it)->function = (*it)->function_base->prepare(arguments); + predicate->function_builder = func_builder_cast; + predicate->function_base = predicate->function_builder->build(arguments); + predicate->function = predicate->function_base->prepare(arguments); } } else { - (*it)->children.swap(new_children); - ColumnsWithTypeAndName arguments; - arguments.reserve((*it)->children.size()); + /// Predicate is function AND, which still have more then one argument. + /// Just update children and rebuild it. + predicate->children.swap(new_children); + auto arguments = prepareFunctionArguments(predicate->children); - for (const auto * child : (*it)->children) - { - ColumnWithTypeAndName argument; - argument.column = child->column; - argument.type = child->result_type; - argument.name = child->result_name; - - arguments.emplace_back(std::move(argument)); - } - - (*it)->function_base = (*it)->function_builder->build(arguments); - (*it)->function = (*it)->function_base->prepare(arguments); + predicate->function_base = predicate->function_builder->build(arguments); + predicate->function = predicate->function_base->prepare(arguments); } removeUnusedActions(false); diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index bd1dcd347df..87cf03f6edd 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -313,6 +313,8 @@ private: void addAliases(const NamesWithAliases & aliases, std::vector & result_nodes); void compileFunctions(); + + ActionsDAGPtr cloneActionsForConjunction(std::unordered_set conjunction); }; From 2ae0b47edbf1b01d45461e64c1c8df59ed2a7361 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 20 Feb 2021 19:25:47 +0300 Subject: [PATCH 284/510] Refactor tryPushDownFilter optimization. --- .../Optimizations/filterPushDown.cpp | 25 +++++++++++++------ 1 file changed, 18 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 1b84fee4857..01e38e81092 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -56,19 +56,30 @@ static size_t tryAddNewFilterStep( if ((*it)->result_name == filter_column_name) break; + const bool found_filter_column = it != expression->getIndex().end(); + + if (!found_filter_column && removes_filter) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", + filter_column_name, expression->dumpDAG()); + + const bool filter_is_constant = found_filter_column && (*it)->column && isColumnConst(*(*it)->column); + + if (!found_filter_column || filter_is_constant) + /// This means that all predicates of filter were pused down. + /// Replace current actions to expression, as we don't need to filter anything. + parent = std::make_unique(child->getOutputStream(), expression); + if (it == expression->getIndex().end()) { - if (!removes_filter) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", - filter_column_name, expression->dumpDAG()); + /// Filter was removed after split. + + - // std::cerr << "replacing to expr because filter " << filter_column_name << " was removed\n"; - parent = std::make_unique(child->getOutputStream(), expression); } else if ((*it)->column && isColumnConst(*(*it)->column)) { - // std::cerr << "replacing to expr because filter is const\n"; + /// Filter column was replaced to constant. parent = std::make_unique(child->getOutputStream(), expression); } From 00e0dbc3e5d39bb8bd0ff79b5001d69866c3a9cf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 20 Feb 2021 20:42:06 +0300 Subject: [PATCH 285/510] Fix test. --- src/Interpreters/ActionsDAG.cpp | 23 +++++++++----- src/Interpreters/ActionsDAG.h | 2 +- .../Optimizations/filterPushDown.cpp | 30 ++----------------- .../01655_plan_optimizations.reference | 4 +-- .../0_stateless/01655_plan_optimizations.sh | 4 +-- 5 files changed, 23 insertions(+), 40 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index b3f86313a1c..1406eecc5c0 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1217,8 +1217,8 @@ namespace struct ConjinctionNodes { - std::unordered_set allowed; - std::unordered_set rejected; + std::vector allowed; + std::vector rejected; }; /// Take a node which result is predicate. @@ -1228,6 +1228,8 @@ struct ConjinctionNodes ConjinctionNodes getConjinctionNodes(ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) { ConjinctionNodes conjunction; + std::unordered_set allowed; + std::unordered_set rejected; struct Frame { @@ -1276,12 +1278,19 @@ ConjinctionNodes getConjinctionNodes(ActionsDAG::Node * predicate, std::unordere else if (is_conjunction) { for (auto * child : cur.node->children) + { if (allowed_nodes.count(child)) - conjunction.allowed.insert(child); + { + if (allowed.insert(child).second) + conjunction.allowed.push_back(child); + + } + } } else if (cur.is_predicate) { - conjunction.rejected.insert(cur.node); + if (rejected.insert(cur.node).second) + conjunction.rejected.push_back(cur.node); } stack.pop(); @@ -1291,7 +1300,7 @@ ConjinctionNodes getConjinctionNodes(ActionsDAG::Node * predicate, std::unordere if (conjunction.allowed.empty()) { if (allowed_nodes.count(predicate)) - conjunction.allowed.insert(predicate); + conjunction.allowed.push_back(predicate); } return conjunction; @@ -1322,7 +1331,7 @@ ColumnsWithTypeAndName prepareFunctionArguments(const std::vector conjunction) +ActionsDAGPtr ActionsDAG::cloneActionsForConjunction(std::vector conjunction) { if (conjunction.empty()) return nullptr; @@ -1448,7 +1457,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, /// Now, when actions are created, update current DAG. - if (conjunction.allowed.count(predicate)) + if (conjunction.rejected.empty()) { /// The whole predicate was split. if (can_remove_filter) diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 87cf03f6edd..2e3baa181fd 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -314,7 +314,7 @@ private: void compileFunctions(); - ActionsDAGPtr cloneActionsForConjunction(std::unordered_set conjunction); + ActionsDAGPtr cloneActionsForConjunction(std::vector conjunction); }; diff --git a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp index 01e38e81092..d64f082b7ee 100644 --- a/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp +++ b/src/Processors/QueryPlan/Optimizations/filterPushDown.cpp @@ -58,11 +58,12 @@ static size_t tryAddNewFilterStep( const bool found_filter_column = it != expression->getIndex().end(); - if (!found_filter_column && removes_filter) + if (!found_filter_column && !removes_filter) throw Exception(ErrorCodes::LOGICAL_ERROR, "Filter column {} was removed from ActionsDAG but it is needed in result. DAG:\n{}", filter_column_name, expression->dumpDAG()); + /// Filter column was replaced to constant. const bool filter_is_constant = found_filter_column && (*it)->column && isColumnConst(*(*it)->column); if (!found_filter_column || filter_is_constant) @@ -70,19 +71,6 @@ static size_t tryAddNewFilterStep( /// Replace current actions to expression, as we don't need to filter anything. parent = std::make_unique(child->getOutputStream(), expression); - if (it == expression->getIndex().end()) - { - /// Filter was removed after split. - - - - } - else if ((*it)->column && isColumnConst(*(*it)->column)) - { - /// Filter column was replaced to constant. - parent = std::make_unique(child->getOutputStream(), expression); - } - /// Add new Filter step before Aggregating. /// Expression/Filter -> Aggregating -> Something auto & node = nodes.emplace_back(); @@ -109,20 +97,6 @@ static Names getAggregatinKeys(const Aggregator::Params & params) return keys; } -// static NameSet getColumnNamesFromSortDescription(const SortDescription & sort_desc, const Block & header) -// { -// NameSet names; -// for (const auto & column : sort_desc) -// { -// if (!column.column_name.empty()) -// names.insert(column.column_name); -// else -// names.insert(header.safeGetByPosition(column.column_number).name); -// } - -// return names; -// } - size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) { if (parent_node->children.size() != 1) diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index fa83c098412..f261e134494 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -68,7 +68,7 @@ Filter column: notEquals(y, 0) 9 10 > one condition of filter should be pushed down after aggregating, other two conditions are ANDed Filter column -FUNCTION and(minus(s, 4) :: 2, minus(s, 8) :: 1) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) +FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4)) Aggregating Filter column: notEquals(y, 0) 0 1 @@ -83,7 +83,7 @@ Filter column: notEquals(y, 0) Filter column ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4)) Aggregating -Filter column: and(minus(y, 4), notEquals(y, 0)) +Filter column: and(notEquals(y, 0), minus(y, 4)) 0 1 1 2 2 3 diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index e47b03661e4..84452fe651f 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -66,7 +66,7 @@ $CLICKHOUSE_CLIENT -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s - 8 and s - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 2, minus(s, 8) :: 1) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" + grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 1, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y @@ -79,7 +79,7 @@ $CLICKHOUSE_CLIENT -q " select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y ) where y != 0 and s != 8 and y - 4 settings enable_optimize_predicate_expression=0" | - grep -o "Aggregating\|Filter column\|Filter column: and(minus(y, 4), notEquals(y, 0))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" + grep -o "Aggregating\|Filter column\|Filter column: and(notEquals(y, 0), minus(y, 4))\|ALIAS notEquals(s, 8) :: 1 -> and(notEquals(y, 0), notEquals(s, 8), minus(y, 4))" $CLICKHOUSE_CLIENT -q " select s, y from ( select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y From ce6d2ff7244ad7fe05673cc67ff82e6bc06477e5 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 20 Feb 2021 22:13:30 +0400 Subject: [PATCH 286/510] Fix the test --- tests/integration/test_row_policy/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 22da7b10b0b..5781686067f 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -124,11 +124,11 @@ def test_cannot_trick_row_policy_with_keyword_with(): assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 1], [1, 0], [0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 1], [1, 0], [0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0], [0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) def test_policy_from_users_xml_affects_only_user_assigned(): From dfde9de0e9c31dc57ee0ba4cc0105a1195570e39 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sat, 20 Feb 2021 23:30:21 +0400 Subject: [PATCH 287/510] Remove commented-out code --- src/Interpreters/ExpressionAnalyzer.cpp | 9 ----- src/Interpreters/InterpreterSelectQuery.cpp | 40 ------------------- .../MergeTree/MergeTreeBlockReadUtils.cpp | 7 ---- 3 files changed, 56 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2dc8d137abe..b174265fbbd 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1406,7 +1406,6 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( { filter_info = filter_info_; filter_info->do_remove_column = true; - //query_analyzer.appendPreliminaryFilter(chain, filter_info->actions, filter_info->column_name); } if (auto actions = query_analyzer.appendPrewhere(chain, !first_stage, additional_required_columns_after_prewhere)) @@ -1575,12 +1574,6 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si { size_t next_step_i = 0; - // if (hasFilter()) - // { - // const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); - // filter_info->do_remove_column = step.can_remove_required_output.at(0); - // } - if (hasPrewhere()) { const ExpressionActionsChain::Step & step = *chain.steps.at(next_step_i++); @@ -1606,8 +1599,6 @@ void ExpressionAnalysisResult::finalize(const ExpressionActionsChain & chain, si void ExpressionAnalysisResult::removeExtraColumns() const { - // if (hasFilter()) - // filter_info->actions->projectInput(); if (hasWhere()) before_where->projectInput(); if (hasHaving()) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 835c0c0e50f..b3ad2d2af0d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -860,20 +860,6 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, const BlockInpu expressions.prewhere_info->row_level_filter_actions = std::move(expressions.filter_info->actions); expressions.prewhere_info->row_level_column_name = std::move(expressions.filter_info->column_name); expressions.prewhere_info->row_level_filter_actions->projectInput(false); - // if (expressions.filter_info->do_remove_column) - // { - // /// Instead of removing column, add it to prewhere_actions input (but not in index). - // /// It will be removed at prewhere_actions execution. - // const auto & index = expressions.prewhere_info->row_level_filter_actions->getIndex(); - // auto it = index.find(expressions.prewhere_info->row_level_column_name); - // if (it == index.end()) - // throw Exception(ErrorCodes::LOGICAL_ERROR, "Not found column {} in row level security filter {}", - // expressions.prewhere_info->row_level_column_name, expressions.prewhere_info->row_level_filter_actions->dumpDAG()); - // const auto & node = *it; - - // expressions.prewhere_info->prewhere_actions->addInput(node->result_name, node->result_type, true, false); - // } - expressions.filter_info = nullptr; } } @@ -1409,32 +1395,6 @@ void InterpreterSelectQuery::executeFetchColumns(QueryProcessingStage::Enum proc if (storage) { - /// Append columns from the table filter to required - // if (row_policy_filter) - // { - // ActionsDAG * row_policy_dag = nullptr; - // if (expressions.filter_info) - // row_policy_dag = expressions.filter_info->actions.get(); - // else if (expressions.prewhere_info) - // { - // if (expressions.prewhere_info->row_level_filter_actions) - // row_policy_dag = expressions.prewhere_info->row_level_filter_actions.get(); - // else if (expressions.prewhere_info->prewhere_actions) - // row_policy_dag = expressions.prewhere_info->prewhere_actions.get(); - // } - - // if (row_policy_dag) - // { - // auto required_columns_from_filter = row_policy_dag->getRequiredColumns(); - - // for (const auto & column : required_columns_from_filter) - // { - // if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column.name)) - // required_columns.push_back(column.name); - // } - // } - // } - /// Detect, if ALIAS columns are required for query execution auto alias_columns_required = false; const ColumnsDescription & storage_columns = metadata_snapshot->getColumns(); diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index bf5fd307b1d..10ce061a864 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -304,13 +304,6 @@ MergeTreeReadTaskColumns getReadTaskColumns( column_names = post_column_names; } - // std::cerr << "---------- Pre column names\n"; - // for (const auto & col : pre_column_names) - // std::cerr << col << std::endl; - // std::cerr << "----------- Post column names\n"; - // for (const auto & col : column_names) - // std::cerr << col << std::endl; - MergeTreeReadTaskColumns result; if (check_columns) From 2eecaee08dc1f52786527674d3e9263df846cdb1 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Fri, 19 Feb 2021 16:24:57 +0300 Subject: [PATCH 288/510] Better tests for protobuf format. --- src/Columns/ColumnFixedString.cpp | 15 - src/Columns/ColumnFixedString.h | 2 - src/DataTypes/DataTypeFixedString.cpp | 19 +- src/DataTypes/DataTypeFixedString.h | 6 + src/Formats/ProtobufSerializer.cpp | 17 +- .../00825_protobuf_format_array_3dim.sh | 12 +- .../00825_protobuf_format_array_of_arrays.sh | 18 +- .../00825_protobuf_format_enum_mapping.sh | 10 +- .../00825_protobuf_format_input.insh | 5 - .../00825_protobuf_format_input.reference | 18 - .../00825_protobuf_format_input.sh | 77 --- .../00825_protobuf_format_input_single.insh | 12 - .../0_stateless/00825_protobuf_format_map.sh | 18 +- .../00825_protobuf_format_nested_optional.sh | 10 +- ...tobuf_format_no_length_delimiter.reference | 13 + ...825_protobuf_format_no_length_delimiter.sh | 52 ++ .../00825_protobuf_format_output.reference | Bin 3726 -> 0 bytes .../00825_protobuf_format_output.sh | 79 --- .../00825_protobuf_format_persons.reference | 569 ++++++++++++++++++ .../00825_protobuf_format_persons.sh | 118 ++++ .../00825_protobuf_format_squares.reference | 24 + .../00825_protobuf_format_squares.sh | 34 ++ .../00825_protobuf_format_table_default.sh | 10 +- .../00825_protobuf_format_array_3dim.proto | 0 ...0825_protobuf_format_array_of_arrays.proto | 0 .../00825_protobuf_format_enum_mapping.proto | 0 .../00825_protobuf_format_map.proto | 0 ...0825_protobuf_format_nested_optional.proto | 0 ..._protobuf_format_no_length_delimiter.proto | 6 + .../00825_protobuf_format_persons.proto} | 6 - ...825_protobuf_format_persons_syntax2.proto} | 0 .../00825_protobuf_format_squares.proto | 6 + .../00825_protobuf_format_syntax2.proto | 63 ++ .../00825_protobuf_format_table_default.proto | 0 .../protobuf_length_delimited_encoder.py | 8 +- 35 files changed, 964 insertions(+), 263 deletions(-) delete mode 100644 tests/queries/0_stateless/00825_protobuf_format_input.insh delete mode 100644 tests/queries/0_stateless/00825_protobuf_format_input.reference delete mode 100755 tests/queries/0_stateless/00825_protobuf_format_input.sh delete mode 100644 tests/queries/0_stateless/00825_protobuf_format_input_single.insh create mode 100644 tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.reference create mode 100755 tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh delete mode 100644 tests/queries/0_stateless/00825_protobuf_format_output.reference delete mode 100755 tests/queries/0_stateless/00825_protobuf_format_output.sh create mode 100644 tests/queries/0_stateless/00825_protobuf_format_persons.reference create mode 100755 tests/queries/0_stateless/00825_protobuf_format_persons.sh create mode 100644 tests/queries/0_stateless/00825_protobuf_format_squares.reference create mode 100755 tests/queries/0_stateless/00825_protobuf_format_squares.sh rename tests/queries/0_stateless/{ => format_schemas}/00825_protobuf_format_array_3dim.proto (100%) rename tests/queries/0_stateless/{ => format_schemas}/00825_protobuf_format_array_of_arrays.proto (100%) rename tests/queries/0_stateless/{ => format_schemas}/00825_protobuf_format_enum_mapping.proto (100%) rename tests/queries/0_stateless/{ => format_schemas}/00825_protobuf_format_map.proto (100%) rename tests/queries/0_stateless/{ => format_schemas}/00825_protobuf_format_nested_optional.proto (100%) create mode 100644 tests/queries/0_stateless/format_schemas/00825_protobuf_format_no_length_delimiter.proto rename tests/queries/0_stateless/{00825_protobuf_format.proto => format_schemas/00825_protobuf_format_persons.proto} (97%) rename tests/queries/0_stateless/{00825_protobuf_format_syntax2.proto => format_schemas/00825_protobuf_format_persons_syntax2.proto} (100%) create mode 100644 tests/queries/0_stateless/format_schemas/00825_protobuf_format_squares.proto create mode 100644 tests/queries/0_stateless/format_schemas/00825_protobuf_format_syntax2.proto rename tests/queries/0_stateless/{ => format_schemas}/00825_protobuf_format_table_default.proto (100%) diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index c4a7f923867..84bd0561f01 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -474,19 +474,4 @@ ColumnPtr ColumnFixedString::compress() const }); } - -void ColumnFixedString::alignStringLength(ColumnFixedString::Chars & data, size_t n, size_t old_size) -{ - size_t length = data.size() - old_size; - if (length < n) - { - data.resize_fill(old_size + n); - } - else if (length > n) - { - data.resize_assume_reserved(old_size); - throw Exception("Too large value for FixedString(" + std::to_string(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE); - } -} - } diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index d9f6619b2d1..5e7b7f360f0 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -184,8 +184,6 @@ public: const Chars & getChars() const { return chars; } size_t getN() const { return n; } - - static void alignStringLength(ColumnFixedString::Chars & data, size_t n, size_t old_size); }; } diff --git a/src/DataTypes/DataTypeFixedString.cpp b/src/DataTypes/DataTypeFixedString.cpp index 21cfe855169..87e989d1dd2 100644 --- a/src/DataTypes/DataTypeFixedString.cpp +++ b/src/DataTypes/DataTypeFixedString.cpp @@ -25,6 +25,7 @@ namespace ErrorCodes extern const int CANNOT_READ_ALL_DATA; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int UNEXPECTED_AST_STRUCTURE; + extern const int TOO_LARGE_STRING_SIZE; } @@ -120,13 +121,21 @@ void DataTypeFixedString::serializeTextEscaped(const IColumn & column, size_t ro } -static inline void alignStringLength(const DataTypeFixedString & type, - ColumnFixedString::Chars & data, - size_t string_start) +void DataTypeFixedString::alignStringLength(PaddedPODArray & chars, size_t old_size) const { - ColumnFixedString::alignStringLength(data, type.getN(), string_start); + size_t length = chars.size() - old_size; + if (length < n) + { + chars.resize_fill(old_size + n); + } + else if (length > n) + { + chars.resize_assume_reserved(old_size); + throw Exception("Too large value for FixedString(" + std::to_string(n) + ")", ErrorCodes::TOO_LARGE_STRING_SIZE); + } } + template static inline void read(const DataTypeFixedString & self, IColumn & column, Reader && reader) { @@ -135,7 +144,7 @@ static inline void read(const DataTypeFixedString & self, IColumn & column, Read try { reader(data); - alignStringLength(self, data, prev_size); + self.alignStringLength(data, prev_size); } catch (...) { diff --git a/src/DataTypes/DataTypeFixedString.h b/src/DataTypes/DataTypeFixedString.h index af82e4b5d11..5c80a0e346a 100644 --- a/src/DataTypes/DataTypeFixedString.h +++ b/src/DataTypes/DataTypeFixedString.h @@ -1,6 +1,7 @@ #pragma once #include +#include #define MAX_FIXEDSTRING_SIZE 0xFFFFFF @@ -82,6 +83,11 @@ public: bool isCategorial() const override { return true; } bool canBeInsideNullable() const override { return true; } bool canBeInsideLowCardinality() const override { return true; } + + /// Makes sure that the length of a newly inserted string to `chars` is equal to getN(). + /// If the length is less than getN() the function will add zero characters up to getN(). + /// If the length is greater than getN() the function will throw an exception. + void alignStringLength(PaddedPODArray & chars, size_t old_size) const; }; } diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 82149460773..66efa76fe97 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -525,16 +525,16 @@ namespace { public: using ColumnType = std::conditional_t; - using StringDataType = std::conditional_t; ProtobufSerializerString( - const StringDataType & string_data_type_, + const std::shared_ptr & fixed_string_data_type_, const google::protobuf::FieldDescriptor & field_descriptor_, const ProtobufReaderOrWriter & reader_or_writer_) : ProtobufSerializerSingleValue(field_descriptor_, reader_or_writer_) + , fixed_string_data_type(fixed_string_data_type_) + , n(fixed_string_data_type->getN()) { static_assert(is_fixed_string, "This constructor for FixedString only"); - n = string_data_type_.getN(); setFunctions(); prepareEnumMapping(); } @@ -583,11 +583,11 @@ namespace { if (row_num < old_size) { - ColumnFixedString::alignStringLength(text_buffer, n, 0); + fixed_string_data_type->alignStringLength(text_buffer, 0); memcpy(data.data() + row_num * n, text_buffer.data(), n); } else - ColumnFixedString::alignStringLength(data, n, old_data_size); + fixed_string_data_type->alignStringLength(data, old_data_size); } else { @@ -817,7 +817,7 @@ namespace auto str = default_function(); arr.insert(str.data(), str.data() + str.size()); if constexpr (is_fixed_string) - ColumnFixedString::alignStringLength(arr, n, 0); + fixed_string_data_type->alignStringLength(arr, 0); default_string = std::move(arr); } return *default_string; @@ -865,7 +865,8 @@ namespace str.insert(name.data(), name.data() + name.length()); } - size_t n = 0; + const std::shared_ptr fixed_string_data_type; + const size_t n = 0; std::function write_function; std::function &)> read_function; std::function default_function; @@ -2765,7 +2766,7 @@ namespace case TypeIndex::DateTime: return std::make_unique(field_descriptor, reader_or_writer); case TypeIndex::DateTime64: return std::make_unique(assert_cast(*data_type), field_descriptor, reader_or_writer); case TypeIndex::String: return std::make_unique>(field_descriptor, reader_or_writer); - case TypeIndex::FixedString: return std::make_unique>(assert_cast(*data_type), field_descriptor, reader_or_writer); + case TypeIndex::FixedString: return std::make_unique>(typeid_cast>(data_type), field_descriptor, reader_or_writer); case TypeIndex::Enum8: return std::make_unique>(typeid_cast>(data_type), field_descriptor, reader_or_writer); case TypeIndex::Enum16: return std::make_unique>(typeid_cast>(data_type), field_descriptor, reader_or_writer); case TypeIndex::Decimal32: return std::make_unique>(assert_cast &>(*data_type), field_descriptor, reader_or_writer); diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh index 903217ca939..762f8a937e4 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_array_3dim.sh @@ -1,18 +1,19 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM array_3dim_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_array_3dim:ABC'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo -$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$CURDIR/00825_protobuf_format_array_3dim:ABC" --input "$BINARY_FILE_PATH" +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_array_3dim:ABC" --input "$BINARY_FILE_PATH" # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO array_3dim_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$CURDIR/00825_protobuf_format_array_3dim:ABC'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO array_3dim_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_3dim:ABC'" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM array_3dim_protobuf_00825" rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE array_3dim_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh b/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh index 0b386723091..243446f9438 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.sh @@ -3,18 +3,21 @@ # https://github.com/ClickHouse/ClickHouse/issues/9069 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM array_of_arrays_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo -$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$CURDIR/00825_protobuf_format_array_of_arrays:AA" --input "$BINARY_FILE_PATH" +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA" --input "$BINARY_FILE_PATH" # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO array_of_arrays_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$CURDIR/00825_protobuf_format_array_of_arrays:AA'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO array_of_arrays_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_array_of_arrays:AA'" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM array_of_arrays_protobuf_00825" rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE array_of_arrays_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh index cbb387a62a5..300f82e5ca2 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.sh @@ -3,13 +3,14 @@ # https://github.com/ClickHouse/ClickHouse/issues/7438 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_enum_mapping:Message'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo -$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$CURDIR/00825_protobuf_format_enum_mapping:Message" --input "$BINARY_FILE_PATH" +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_enum_mapping:Message" --input "$BINARY_FILE_PATH" # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$CURDIR/00825_protobuf_format_enum_mapping:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO enum_mapping_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_enum_mapping:Message'" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM enum_mapping_protobuf_00825" rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE enum_mapping_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_input.insh b/tests/queries/0_stateless/00825_protobuf_format_input.insh deleted file mode 100644 index 39a2f17c98f..00000000000 --- a/tests/queries/0_stateless/00825_protobuf_format_input.insh +++ /dev/null @@ -1,5 +0,0 @@ -echo -ne '\xf3\x01\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\xb2\x01\x10\x0a\x0e\xa2\x06\x0b\x0a\x09\x08\xf4\x03\x12\x04\xf5\x03\xf6\x03\x7e\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09\xc0\x01\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41\xb2\x01\x0a\x0a\x08\xa2\x06\x05\x0a\x03\x08\xf7\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" -echo -ne '\xb3\x01\x12\x05\x46\x72\x69\x64\x61\x28\x99\xe1\xf3\xd1\x0b\x52\x08\x45\x72\x6d\x61\x6b\x6f\x76\x61\x72\x0c\x00\x00\xdc\x42\x00\x00\x52\x43\x00\x00\x94\x42\x79\x48\xce\x3d\x51\x00\x00\x00\x00\xc8\x02\x14\xc2\x05\x08\x00\x00\x80\x44\x00\x00\x80\x49\x9a\x06\x02\x4b\x42\x9a\x06\x02\x4d\x42\xa1\x06\x00\x00\x00\x00\x00\x00\xe0\x3f\xa8\x06\x2a\xa8\x06\xa8\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\x25\x06\x49\x40\xfa\x06\x02\x34\x30\x90\x08\xe2\x08\xe1\x08\x89\xe6\x6e\xdd\x01\x00\x00\x00\xb0\x09\xc3\x19\xd0\x0c\xb7\x02\xe2\x12\x24\x32\x30\x66\x63\x64\x39\x35\x61\x2d\x33\x33\x32\x64\x2d\x34\x31\x64\x62\x2d\x61\x39\x65\x63\x2d\x31\x36\x31\x66\x36\x34\x34\x64\x30\x35\x39\x63\xa0\x38\xbc\x05\xaa\x38\x02\xbd\x05\xb4\x01\x08\x01\x12\x06\x49\x73\x6f\x6c\x64\x65\x52\x07\x4c\x61\x76\x72\x6f\x76\x61\x72\x0c\x00\x00\x7f\x43\x00\x00\x00\x00\x00\x00\x7f\x43\xaa\x01\x03\x61\x62\x63\xc8\x02\x32\xc2\x05\x08\x00\x00\x00\x41\x00\x00\x80\x3f\x9a\x06\x04\x42\x79\x74\x65\x9a\x06\x03\x42\x69\x74\xa1\x06\x00\x00\x00\x00\x00\x00\x12\x40\xa8\x06\x1a\xa8\x06\xb0\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\xf9\x0f\x49\x40\xc2\x06\x01\x2c\xfa\x06\x02\x33\x32\x90\x08\x78\xe1\x08\x39\x4e\x2b\xfe\xe4\xf5\xff\xff\xb0\x09\xe8\x30\xd8\x12\x01\xe2\x12\x24\x37\x63\x66\x61\x36\x38\x35\x36\x2d\x61\x35\x34\x61\x2d\x34\x37\x38\x36\x2d\x62\x38\x65\x35\x2d\x37\x34\x35\x31\x35\x39\x64\x35\x32\x32\x37\x38\xa0\x38\xbe\x05\xc2\x3e\x05\x15\x00\x00\xb6\x42' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'" -echo -ne '\xa5\x02\x0a\x24\x61\x61\x30\x65\x35\x61\x30\x36\x2d\x63\x61\x62\x32\x2d\x34\x30\x33\x34\x2d\x61\x36\x61\x32\x2d\x34\x38\x65\x38\x32\x62\x39\x31\x36\x36\x34\x65\x12\x06\x4c\x65\x6f\x6e\x69\x64\x1a\x08\x4b\x69\x72\x69\x6c\x6c\x6f\x76\x22\x04\x6d\x61\x6c\x65\x2a\x0a\x31\x39\x38\x33\x2d\x30\x36\x2d\x32\x34\x3a\x0c\x2b\x37\x34\x39\x35\x30\x32\x37\x35\x38\x36\x34\x42\x01\x31\x4a\x13\x32\x30\x31\x39\x2d\x30\x32\x2d\x30\x34\x20\x30\x39\x3a\x34\x35\x3a\x30\x30\x52\x02\x33\x35\x5a\x06\x63\x61\x6e\x63\x65\x72\x62\x07\x37\x20\x72\x69\x6e\x67\x73\x62\x08\x45\x61\x73\x74\x73\x69\x64\x65\x62\x0b\x4c\x61\x73\x74\x20\x48\x75\x72\x72\x61\x68\x6a\x01\x30\x6a\x01\x30\x6a\x03\x32\x35\x35\x72\x09\x53\x61\x6e\x20\x44\x69\x65\x67\x6f\x7a\x09\x33\x32\x2e\x38\x32\x33\x39\x34\x33\x7a\x0b\x2d\x31\x31\x37\x2e\x30\x38\x31\x33\x32\x37\x82\x01\x09\x33\x2e\x31\x34\x31\x35\x39\x32\x37\x8a\x01\x08\x31\x35\x30\x30\x30\x30\x30\x30\x92\x01\x06\x31\x38\x36\x2e\x37\x35\x9a\x01\x04\x2d\x32\x2e\x31\xa2\x01\x0b\x32\x30\x36\x35\x39\x38\x32\x39\x33\x33\x31\xaa\x01\x18\x0a\x06\x6d\x69\x6e\x75\x74\x65\x0a\x04\x68\x6f\x75\x72\x12\x02\x36\x30\x12\x04\x33\x36\x30\x30\xb2\x01\x08\x0a\x06\x12\x04\x31\x38\x30\x30' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'" -echo -ne '\xdd\x01\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01\xb3\x01\x0b\xa2\x06\x05\x0b\x08\x96\x4a\x0c\x0c\xb4\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'" -echo -ne '\x04\x08\x02\x10\x04\x00\x04\x08\x03\x10\x09' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" diff --git a/tests/queries/0_stateless/00825_protobuf_format_input.reference b/tests/queries/0_stateless/00825_protobuf_format_input.reference deleted file mode 100644 index 75fef3f8ac3..00000000000 --- a/tests/queries/0_stateless/00825_protobuf_format_input.reference +++ /dev/null @@ -1,18 +0,0 @@ -a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] -c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] -aa0e5a06-cab2-4034-a6a2-48e82b91664e Leonid Kirillov male 1983-06-24 \N +74950275864\0 1 2019-02-04 09:45:00 35 cancer ['7 rings','Eastside','Last Hurrah'] [0,0,255] San Diego [32.823943,-117.081327] 3.1415927 15000000.00 186.75 -2.1 20659829331 ['minute','hour'] [60,3600] \N [1800] -20fcd95a-332d-41db-a9ec-161f644d059c Frida Ermakova female 1978-12-12 \N 3124555929\0\0\0 0 2013-03-11 16:30:00 40 sagittarius [] [110,210,74] [42.000000,-88.000000] 3.1410000324249268 311.00 0.5 10.0 8010000009 ['KB','MB'] [1024,1048576] 700 [701] -a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] -3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1] 9494 [] -7cfa6856-a54a-4786-b8e5-745159d52278 Isolde Lavrova female 1987-02-09 \N \N 1 \N 32 aquarius [] [255,0,255] [26.000000,-80.000000] 3.1415998935699463 \N 4.5 25.0 -11111111111111 ['Byte','Bit'] [8,1] 702 [] -0 0 -2 4 -3 9 -a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] -c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] -a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] -3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1] 9494 [] -2 4 -3 9 -ok -ok diff --git a/tests/queries/0_stateless/00825_protobuf_format_input.sh b/tests/queries/0_stateless/00825_protobuf_format_input.sh deleted file mode 100755 index 5a85a852cb1..00000000000 --- a/tests/queries/0_stateless/00825_protobuf_format_input.sh +++ /dev/null @@ -1,77 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -# shellcheck source=../shell_config.sh -. "$CURDIR"/../shell_config.sh - -set -eo pipefail - -# Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' -DROP TABLE IF EXISTS in_persons_00825; -DROP TABLE IF EXISTS in_squares_00825; - -CREATE TABLE in_persons_00825 (uuid UUID, - name String, - surname String, - gender Enum8('male'=1, 'female'=0), - birthDate Date, - photo Nullable(String), - phoneNumber Nullable(FixedString(13)), - isOnline UInt8, - visitTime Nullable(DateTime), - age UInt8, - zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, - 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, - 'pisces'=219), - songs Array(String), - color Array(UInt8), - hometown LowCardinality(String), - location Array(Decimal32(6)), - pi Nullable(Float64), - lotteryWin Nullable(Decimal64(2)), - someRatio Float32, - temperature Decimal32(1), - randomBigNumber Int64, - measureUnits Nested (unit String, coef Float32), - nestiness_a_b_c_d Nullable(UInt32), - `nestiness_a_B.c_E` Array(UInt32) - ) ENGINE = MergeTree ORDER BY tuple(); - -CREATE TABLE in_squares_00825 (number UInt32, square UInt32) ENGINE = MergeTree ORDER BY tuple(); -EOF - -# To generate the file 00825_protobuf_format_input.insh use the following commands: -# ninja ProtobufDelimitedMessagesSerializer -# build/utils/test-data-generator/ProtobufDelimitedMessagesSerializer -# shellcheck source=./00825_protobuf_format_input.insh -source "$CURDIR"/00825_protobuf_format_input.insh - -$CLICKHOUSE_CLIENT --query "SELECT * FROM in_persons_00825 ORDER BY uuid;" -$CLICKHOUSE_CLIENT --query "SELECT * FROM in_squares_00825 ORDER BY number;" - -$CLICKHOUSE_CLIENT --query "TRUNCATE TABLE in_persons_00825;" -$CLICKHOUSE_CLIENT --query "TRUNCATE TABLE in_squares_00825;" - -# shellcheck source=./00825_protobuf_format_input_single.insh -source "$CURDIR"/00825_protobuf_format_input_single.insh - -$CLICKHOUSE_CLIENT --query "SELECT * FROM in_persons_00825 ORDER BY uuid;" -$CLICKHOUSE_CLIENT --query "SELECT * FROM in_squares_00825 ORDER BY number;" - -# Try to input malformed data. -set +eo pipefail -echo -ne '\xe0\x80\x3f\x0b' \ - | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" 2>&1 \ - | grep -qF "Protobuf messages are corrupted" && echo "ok" || echo "fail" -set -eo pipefail - -# Try to input malformed data for ProtobufSingle -set +eo pipefail -echo -ne '\xff\xff\x3f\x0b' \ - | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" 2>&1 \ - | grep -qF "Protobuf messages are corrupted" && echo "ok" || echo "fail" -set -eo pipefail - -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS in_persons_00825;" -$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS in_squares_00825;" diff --git a/tests/queries/0_stateless/00825_protobuf_format_input_single.insh b/tests/queries/0_stateless/00825_protobuf_format_input_single.insh deleted file mode 100644 index 6c4dfec05aa..00000000000 --- a/tests/queries/0_stateless/00825_protobuf_format_input_single.insh +++ /dev/null @@ -1,12 +0,0 @@ -echo -ne '\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\xb2\x01\x10\x0a\x0e\xa2\x06\x0b\x0a\x09\x08\xf4\x03\x12\x04\xf5\x03\xf6\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" -echo -ne '\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" -echo -ne '\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41\xb2\x01\x0a\x0a\x08\xa2\x06\x05\x0a\x03\x08\xf7\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" -echo -ne '\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01\xb3\x01\x0b\xa2\x06\x05\x0b\x08\x96\x4a\x0c\x0c\xb4\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'" - -echo -ne '\x08\x02\x10\x04' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" -echo -ne '\x08\x03\x10\x09' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" - -### Actually empty Protobuf message is a valid message (with all values default). -### It will work in Kafka but clickhouse-client forbids that: -### Code: 108. DB::Exception: No data to insert -## echo -ne '' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'" \ No newline at end of file diff --git a/tests/queries/0_stateless/00825_protobuf_format_map.sh b/tests/queries/0_stateless/00825_protobuf_format_map.sh index 5df25c41750..da7cacf76b8 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_map.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_map.sh @@ -3,29 +3,30 @@ # https://github.com/ClickHouse/ClickHouse/issues/6497 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM map_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_map:Message'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo @@ -34,7 +35,8 @@ hexdump -C $BINARY_FILE_PATH # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO map_00825 FORMAT Protobuf SETTINGS format_schema='$CURDIR/00825_protobuf_format_map:Message'" < "$BINARY_FILE_PATH" -$CLICKHOUSE_CLIENT --query "SELECT * FROM map_00825" +$CLICKHOUSE_CLIENT --query "INSERT INTO map_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_map:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM map_protobuf_00825" rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE map_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh b/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh index 58ded92f2c1..b33db75b5c9 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_nested_optional.sh @@ -3,13 +3,14 @@ # https://github.com/ClickHouse/ClickHouse/issues/6497 CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM nested_optional_protobuf_00825 FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_nested_optional:Message'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo -$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$CURDIR/00825_protobuf_format_nested_optional:Message" --input "$BINARY_FILE_PATH" +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_nested_optional:Message" --input "$BINARY_FILE_PATH" # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO nested_optional_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$CURDIR/00825_protobuf_format_nested_optional:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO nested_optional_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_optional:Message'" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM nested_optional_protobuf_00825" rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE nested_optional_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.reference b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.reference new file mode 100644 index 00000000000..ba84d41eb51 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.reference @@ -0,0 +1,13 @@ +1000 1K +2000 2K +3000 3K + +Binary representation: +00000000 08 e8 07 12 02 31 4b |.....1K| +00000007 + +x: 1000 +str: "1K" + +Roundtrip: +1000 1K diff --git a/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh new file mode 100755 index 00000000000..b95d35e8256 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_no_length_delimiter.sh @@ -0,0 +1,52 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -eo pipefail + +# Run the client. +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" + +# Check the output in the ProtobufSingle format +echo +echo "Binary representation:" +hexdump -C $BINARY_FILE_PATH + +echo +(cd $SCHEMADIR && protoc --decode Message 00825_protobuf_format_no_length_delimiter.proto) < $BINARY_FILE_PATH + +# Check the input in the ProtobufSingle format. +echo +echo "Roundtrip:" +$CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_no_length_delimiter_protobuf_00825 AS no_length_delimiter_protobuf_00825" +$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_no_length_delimiter_protobuf_00825 FORMAT ProtobufSingle SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_no_length_delimiter:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_no_length_delimiter_protobuf_00825" +rm "$BINARY_FILE_PATH" + +# The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. +$CLICKHOUSE_CLIENT --multiquery --testmode > /dev/null <c7uVQdUpWmqfBS_dn2f4DyeWgSvvZUx1G`$q*q5M_gQ(`{0Jmyf0nVI#tr z`jWg{?!Axa_dL%%zso!l*P0q7M5el=L{(W*!i_(Z{c-_qdW~!QN zwffTbKn=7X_j1*_gBm#%gm*>tsa}F8Tn&4AoDdm7*B2Qho8er+J(b1jZ?7e zzA>&`6!>}8>t0~5vagG4BYs0Qqq-?|X|gJ*mZ?Yv)}m5Zl&W1}LZe2QddphOJ~J6N ztND<4`2R$ zwE$HFU-|K=qfpCnsA6&PvvCdwm#J14tqlVsnlS?a~;hiG0ku4 zPj{QC-W?hKkITcs*^O>sd!^Laegn`AgBr|F15H}+Fl;q7Iyg5@1KVG}e+XsS?y>t* zQHSk@DI|*g80QgL{uIdzbc8D^`2h##jUG92s(zf)#<|B%b2AG~ zf$N-0(GT#S3*Yn1m2ZDz;h0I`#HzW9Z_p3kJM#6YKF?0*SySvqOM160>Frc78w75k zil!g{&jJbX$6o~?WSqOXm>+18L#Mg9%4P=?@gLwXApkX99l7w!;^L%mtR4W}iRV6a zxXkNbd;W|03%^#h736ooWB^DXd}|s89mR8Wdd_&eJk% zJTT<||CI-TEhvI>>ZEX^X|7^u?$XS>E6dpJO*yJxV?*t2%`MM|K>CkdrR2vNcek0e z>O|Slu_P0T7`Nm|5Xmpx5B4@BEK8Wy$oe{!uEV$u^xh^!%${^SVy9Don=ve*G=VUD zmV!p0)TdDeLM|gHn$J%N#p)j9R(!I$4{=0ReKJ0XSnQKW5KDdf7%G=F?8mt=fuwxy zEYF)|IpamL*NuG`W*JVIf?%o?S&Fe2CfS+DNqFY(ME@lZ7BgoJ*JTy>GKho~3F|=w zToC^@t|#6bv3e6M+&94zm|PSuizIN`3YL#gzO3@ zWnnAR=*^|9cUj1Kiy3'; -SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'; -SELECT 'STRINGS->'; -SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'; -SELECT 'SYNTAX2->'; -SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'; -SELECT 'SQUARES->'; -SELECT * FROM out_squares_00825 ORDER BY number FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'; - -SELECT '\n\n** ProtobufSingle **\n\n'; - -SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -SELECT 'ALTERNATIVE->'; -SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'; -SELECT 'STRINGS->'; -SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'; -SELECT 'SYNTAX2->'; -SELECT * FROM out_persons_00825 ORDER BY name LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'; -SELECT 'SQUARES->'; -SELECT * FROM out_squares_00825 ORDER BY number LIMIT 1 FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'; - --- Code: 546, e.displayText() = DB::Exception: The ProtobufSingle format can't be used to write multiple rows because this format doesn't have any row delimiter. -SELECT * FROM out_persons_00825 ORDER BY name FORMAT ProtobufSingle SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; -- { clientError 546 } - -DROP TABLE IF EXISTS out_persons_00825; -DROP TABLE IF EXISTS out_squares_00825; -EOF diff --git a/tests/queries/0_stateless/00825_protobuf_format_persons.reference b/tests/queries/0_stateless/00825_protobuf_format_persons.reference new file mode 100644 index 00000000000..711980b3592 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_persons.reference @@ -0,0 +1,569 @@ +a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] +c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] +a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] + +Schema 00825_protobuf_format_persons:Person + +Binary representation: +00000000 f4 01 0a 24 61 37 35 32 32 31 35 38 2d 33 64 34 |...$a7522158-3d4| +00000010 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 63 35 39 |1-4b77-ad69-6c59| +00000020 38 65 65 35 35 63 34 39 12 04 49 76 61 6e 1a 06 |8ee55c49..Ivan..| +00000030 50 65 74 72 6f 76 20 01 28 af 1f 32 03 70 6e 67 |Petrov .(..2.png| +00000040 3a 0d 2b 37 34 39 35 31 32 33 34 35 36 37 00 40 |:.+74951234567.@| +00000050 01 4d fc d0 30 5c 50 26 58 09 62 09 59 65 73 74 |.M..0\P&X.b.Yest| +00000060 65 72 64 61 79 62 07 46 6c 6f 77 65 72 73 6a 04 |erdayb.Flowersj.| +00000070 ff 01 00 00 72 06 4d 6f 73 63 6f 77 7a 08 4b 03 |....r.Moscowz.K.| +00000080 5f 42 72 7d 16 42 81 01 1f 85 eb 51 b8 1e 09 40 |_Br}.B.....Q...@| +00000090 89 01 33 33 33 33 33 c3 6a 40 95 01 cd cc cc 3d |..33333.j@.....=| +000000a0 9d 01 9a 99 b9 40 a0 01 80 c4 d7 8d 7f aa 01 0c |.....@..........| +000000b0 0a 05 6d 65 74 65 72 15 00 00 80 3f aa 01 11 0a |..meter....?....| +000000c0 0a 63 65 6e 74 69 6d 65 74 65 72 15 0a d7 23 3c |.centimeter...#<| +000000d0 aa 01 10 0a 09 6b 69 6c 6f 6d 65 74 65 72 15 00 |.....kilometer..| +000000e0 00 7a 44 b2 01 10 0a 0e a2 06 0b 0a 09 08 f4 03 |.zD.............| +000000f0 12 04 f5 03 f6 03 7e 0a 24 63 36 39 34 61 64 38 |......~.$c694ad8| +00000100 61 2d 66 37 31 34 2d 34 65 61 33 2d 39 30 37 64 |a-f714-4ea3-907d| +00000110 2d 66 64 35 34 66 62 32 35 64 39 62 35 12 07 4e |-fd54fb25d9b5..N| +00000120 61 74 61 6c 69 61 1a 08 53 6f 6b 6f 6c 6f 76 61 |atalia..Sokolova| +00000130 28 a6 3f 32 03 6a 70 67 50 1a 58 0b 6a 04 64 c8 |(.?2.jpgP.X.j.d.| +00000140 01 32 72 08 50 6c 79 6d 6f 75 74 68 7a 08 6a 9d |.2r.Plymouthz.j.| +00000150 49 42 46 8c 84 c0 81 01 6e 86 1b f0 f9 21 09 40 |IBF.....n....!.@| +00000160 95 01 42 60 e5 3b 9d 01 cd cc ac 40 a0 01 ff ff |..B`.;.....@....| +00000170 a9 ce 93 8c 09 c0 01 0a 24 61 37 64 61 31 61 61 |........$a7da1aa| +00000180 36 2d 66 34 32 35 2d 34 37 38 39 2d 38 39 34 37 |6-f425-4789-8947| +00000190 2d 62 30 33 34 37 38 36 65 64 33 37 34 12 06 56 |-b034786ed374..V| +000001a0 61 73 69 6c 79 1a 07 53 69 64 6f 72 6f 76 20 01 |asily..Sidorov .| +000001b0 28 fb 48 32 03 62 6d 70 3a 0d 2b 34 34 32 30 31 |(.H2.bmp:.+44201| +000001c0 32 33 34 35 36 37 38 40 01 4d 50 e0 27 5c 50 17 |2345678@.MP.'\P.| +000001d0 58 04 62 05 53 75 6e 6e 79 6a 05 fa 01 f4 01 0a |X.b.Sunnyj......| +000001e0 72 08 4d 75 72 6d 61 6e 73 6b 7a 08 fd f0 89 42 |r.Murmanskz....B| +000001f0 c8 4c 04 42 81 01 11 2d 44 54 fb 21 09 40 89 01 |.L.B...-DT.!.@..| +00000200 00 00 00 e8 76 48 37 42 95 01 00 00 48 44 9d 01 |....vH7B....HD..| +00000210 cd cc 4c c0 a0 01 80 d4 9f 93 01 aa 01 0c 0a 05 |..L.............| +00000220 70 6f 75 6e 64 15 00 00 80 41 b2 01 0a 0a 08 a2 |pound....A......| +00000230 06 05 0a 03 08 f7 03 |.......| +00000237 + +MESSAGE #1 AT 0x00000002 +uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49" +name: "Ivan" +surname: "Petrov" +gender: male +birthDate: 4015 +photo: "png" +phoneNumber: "+74951234567\000" +isOnline: true +visitTime: 1546703100 +age: 38 +zodiacSign: capricorn +songs: "Yesterday" +songs: "Flowers" +color: 255 +color: 0 +color: 0 +hometown: "Moscow" +location: 55.7532158 +location: 37.6225052 +pi: 3.14 +lotteryWin: 214.1 +someRatio: 0.1 +temperature: 5.8 +randomBigNumber: 17060000000 +measureUnits { + unit: "meter" + coef: 1 +} +measureUnits { + unit: "centimeter" + coef: 0.01 +} +measureUnits { + unit: "kilometer" + coef: 1000 +} +nestiness { + a { + b { + c { + d: 500 + e: 501 + e: 502 + } + } + } +} +MESSAGE #2 AT 0x000000F7 +uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5" +name: "Natalia" +surname: "Sokolova" +birthDate: 8102 +photo: "jpg" +age: 26 +zodiacSign: pisces +color: 100 +color: 200 +color: 50 +hometown: "Plymouth" +location: 50.4037247 +location: -4.14212322 +pi: 3.14159 +someRatio: 0.007 +temperature: 5.4 +randomBigNumber: -20000000000000 +MESSAGE #3 AT 0x00000177 +uuid: "a7da1aa6-f425-4789-8947-b034786ed374" +name: "Vasily" +surname: "Sidorov" +gender: male +birthDate: 9339 +photo: "bmp" +phoneNumber: "+442012345678" +isOnline: true +visitTime: 1546117200 +age: 23 +zodiacSign: leo +songs: "Sunny" +color: 250 +color: 244 +color: 10 +hometown: "Murmansk" +location: 68.9706802 +location: 33.0749817 +pi: 3.14159265358979 +lotteryWin: 100000000000 +someRatio: 800 +temperature: -3.2 +randomBigNumber: 154400000 +measureUnits { + unit: "pound" + coef: 16 +} +nestiness { + a { + b { + c { + d: 503 + } + } + } +} + +Binary representation is as expected + +Roundtrip: +a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] +c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] +a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] + +Schema 00825_protobuf_format_persons:AltPerson + +Binary representation: +00000000 c4 01 08 01 12 04 49 76 61 6e 28 87 a8 c4 9b 97 |......Ivan(.....| +00000010 02 52 06 50 65 74 72 6f 76 72 0c 00 00 7f 43 00 |.R.Petrovr....C.| +00000020 00 00 00 00 00 00 00 79 fc d0 30 5c 00 00 00 00 |.......y..0\....| +00000030 c8 02 0a c2 05 0c 00 00 80 3f 0a d7 23 3c 00 00 |.........?..#<..| +00000040 7a 44 9a 06 05 6d 65 74 65 72 9a 06 0a 63 65 6e |zD...meter...cen| +00000050 74 69 6d 65 74 65 72 9a 06 09 6b 69 6c 6f 6d 65 |timeter...kilome| +00000060 74 65 72 a1 06 00 00 00 a0 99 99 b9 3f a8 06 37 |ter.........?..7| +00000070 a8 06 25 bd 06 c3 f5 48 40 fa 06 02 33 38 90 08 |..%....H@...38..| +00000080 c6 09 e1 08 00 f1 da f8 03 00 00 00 b0 09 af 1f |................| +00000090 d0 0c d6 01 e2 12 24 61 37 35 32 32 31 35 38 2d |......$a7522158-| +000000a0 33 64 34 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 |3d41-4b77-ad69-6| +000000b0 63 35 39 38 65 65 35 35 63 34 39 a0 38 f4 03 aa |c598ee55c49.8...| +000000c0 38 04 f5 03 f6 03 84 01 12 07 4e 61 74 61 6c 69 |8.........Natali| +000000d0 61 52 08 53 6f 6b 6f 6c 6f 76 61 72 0c 00 00 c8 |aR.Sokolovar....| +000000e0 42 00 00 48 43 00 00 48 42 c8 02 0a a1 06 00 00 |B..HC..HB.......| +000000f0 00 40 08 ac 7c 3f a8 06 32 a8 06 fc ff ff ff ff |.@..|?..2.......| +00000100 ff ff ff ff 01 b0 06 01 bd 06 d0 0f 49 40 fa 06 |............I@..| +00000110 02 32 36 90 08 db 01 e1 08 00 c0 1a 63 cf ed ff |.26.........c...| +00000120 ff b0 09 a6 3f e2 12 24 63 36 39 34 61 64 38 61 |....?..$c694ad8a| +00000130 2d 66 37 31 34 2d 34 65 61 33 2d 39 30 37 64 2d |-f714-4ea3-907d-| +00000140 66 64 35 34 66 62 32 35 64 39 62 35 a3 01 08 01 |fd54fb25d9b5....| +00000150 12 06 56 61 73 69 6c 79 28 ce ca f4 cf ee 0c 52 |..Vasily(......R| +00000160 07 53 69 64 6f 72 6f 76 72 0c 00 00 7a 43 00 00 |.Sidorovr...zC..| +00000170 74 43 00 00 20 41 79 50 e0 27 5c 00 00 00 00 c8 |tC.. AyP.'\.....| +00000180 02 05 c2 05 04 00 00 80 41 9a 06 05 70 6f 75 6e |........A...poun| +00000190 64 a1 06 00 00 00 00 00 00 89 40 a8 06 44 a8 06 |d.........@..D..| +000001a0 21 bd 06 db 0f 49 40 fa 06 02 32 33 90 08 d3 05 |!....I@...23....| +000001b0 e1 08 00 f5 33 09 00 00 00 00 b0 09 fb 48 d0 0c |....3........H..| +000001c0 80 d0 db c3 f4 02 e2 12 24 61 37 64 61 31 61 61 |........$a7da1aa| +000001d0 36 2d 66 34 32 35 2d 34 37 38 39 2d 38 39 34 37 |6-f425-4789-8947| +000001e0 2d 62 30 33 34 37 38 36 65 64 33 37 34 a0 38 f7 |-b034786ed374.8.| +000001f0 03 |.| +000001f1 + +MESSAGE #1 AT 0x00000002 +isOnline: online +name: "Ivan" +phoneNumber: 74951234567 +surname: "Petrov" +color: 255 +color: 0 +color: 0 +visitTime: 1546703100 +temperature: 5 +measureUnits_coef: 1 +measureUnits_coef: 0.01 +measureUnits_coef: 1000 +measureUnits_unit: "meter" +measureUnits_unit: "centimeter" +measureUnits_unit: "kilometer" +someRatio: 0.10000000149011612 +location: 55 +location: 37 +pi: 3.14 +age: "38" +zodiacSign: 1222 +randomBigNumber: 17060000000 +birthDate: 4015 +lotteryWin: 214 +uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49" +nestiness_a_b_c_d: 500 +nestiness_a_b_c_e: 501 +nestiness_a_b_c_e: 502 +MESSAGE #2 AT 0x000000C8 +name: "Natalia" +surname: "Sokolova" +color: 100 +color: 200 +color: 50 +temperature: 5 +someRatio: 0.0070000002160668373 +location: 50 +location: -4 +gender: female +pi: 3.14159 +age: "26" +zodiacSign: 219 +randomBigNumber: -20000000000000 +birthDate: 8102 +uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5" +MESSAGE #3 AT 0x0000014E +isOnline: online +name: "Vasily" +phoneNumber: 442012345678 +surname: "Sidorov" +color: 250 +color: 244 +color: 10 +visitTime: 1546117200 +temperature: -3 +measureUnits_coef: 16 +measureUnits_unit: "pound" +someRatio: 800 +location: 68 +location: 33 +pi: 3.14159274 +age: "23" +zodiacSign: 723 +randomBigNumber: 154400000 +birthDate: 9339 +lotteryWin: 100000000000 +uuid: "a7da1aa6-f425-4789-8947-b034786ed374" +nestiness_a_b_c_d: 503 + +Binary representation is as expected + +Roundtrip: +a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N 74951234567\0\0 1 2019-01-05 18:45:00 38 capricorn [] [255,0,0] [55.000000,37.000000] 3.140000104904175 214.00 0.1 5.0 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] +c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] [50.000000,-4.000000] 3.141590118408203 \N 0.007 5.0 -20000000000000 [] [] \N [] +a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N 442012345678\0 1 2018-12-30 00:00:00 23 leo [] [250,244,10] [68.000000,33.000000] 3.1415927410125732 100000000000.00 800 -3.0 154400000 ['pound'] [16] 503 [] + +Schema 00825_protobuf_format_persons:StrPerson + +Binary representation: +00000000 a7 02 0a 24 61 37 35 32 32 31 35 38 2d 33 64 34 |...$a7522158-3d4| +00000010 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 63 35 39 |1-4b77-ad69-6c59| +00000020 38 65 65 35 35 63 34 39 12 04 49 76 61 6e 1a 06 |8ee55c49..Ivan..| +00000030 50 65 74 72 6f 76 22 04 6d 61 6c 65 2a 0a 31 39 |Petrov".male*.19| +00000040 38 30 2d 31 32 2d 32 39 3a 0d 2b 37 34 39 35 31 |80-12-29:.+74951| +00000050 32 33 34 35 36 37 00 42 01 31 4a 13 32 30 31 39 |234567.B.1J.2019| +00000060 2d 30 31 2d 30 35 20 31 38 3a 34 35 3a 30 30 52 |-01-05 18:45:00R| +00000070 02 33 38 5a 09 63 61 70 72 69 63 6f 72 6e 62 09 |.38Z.capricornb.| +00000080 59 65 73 74 65 72 64 61 79 62 07 46 6c 6f 77 65 |Yesterdayb.Flowe| +00000090 72 73 6a 03 32 35 35 6a 01 30 6a 01 30 72 06 4d |rsj.255j.0j.0r.M| +000000a0 6f 73 63 6f 77 7a 09 35 35 2e 37 35 33 32 31 35 |oscowz.55.753215| +000000b0 7a 09 33 37 2e 36 32 32 35 30 34 82 01 04 33 2e |z.37.622504...3.| +000000c0 31 34 8a 01 06 32 31 34 2e 31 30 92 01 03 30 2e |14...214.10...0.| +000000d0 31 9a 01 03 35 2e 38 a2 01 0b 31 37 30 36 30 30 |1...5.8...170600| +000000e0 30 30 30 30 30 aa 01 2d 0a 05 6d 65 74 65 72 0a |00000..-..meter.| +000000f0 0a 63 65 6e 74 69 6d 65 74 65 72 0a 09 6b 69 6c |.centimeter..kil| +00000100 6f 6d 65 74 65 72 12 01 31 12 04 30 2e 30 31 12 |ometer..1..0.01.| +00000110 04 31 30 30 30 b2 01 11 0a 0f 0a 03 35 30 30 12 |.1000.......500.| +00000120 03 35 30 31 12 03 35 30 32 b4 01 0a 24 63 36 39 |.501..502...$c69| +00000130 34 61 64 38 61 2d 66 37 31 34 2d 34 65 61 33 2d |4ad8a-f714-4ea3-| +00000140 39 30 37 64 2d 66 64 35 34 66 62 32 35 64 39 62 |907d-fd54fb25d9b| +00000150 35 12 07 4e 61 74 61 6c 69 61 1a 08 53 6f 6b 6f |5..Natalia..Soko| +00000160 6c 6f 76 61 22 06 66 65 6d 61 6c 65 2a 0a 31 39 |lova".female*.19| +00000170 39 32 2d 30 33 2d 30 38 42 01 30 52 02 32 36 5a |92-03-08B.0R.26Z| +00000180 06 70 69 73 63 65 73 6a 03 31 30 30 6a 03 32 30 |.piscesj.100j.20| +00000190 30 6a 02 35 30 72 08 50 6c 79 6d 6f 75 74 68 7a |0j.50r.Plymouthz| +000001a0 09 35 30 2e 34 30 33 37 32 34 7a 09 2d 34 2e 31 |.50.403724z.-4.1| +000001b0 34 32 31 32 33 82 01 07 33 2e 31 34 31 35 39 92 |42123...3.14159.| +000001c0 01 05 30 2e 30 30 37 9a 01 03 35 2e 34 a2 01 0f |..0.007...5.4...| +000001d0 2d 32 30 30 30 30 30 30 30 30 30 30 30 30 30 84 |-20000000000000.| +000001e0 02 0a 24 61 37 64 61 31 61 61 36 2d 66 34 32 35 |..$a7da1aa6-f425| +000001f0 2d 34 37 38 39 2d 38 39 34 37 2d 62 30 33 34 37 |-4789-8947-b0347| +00000200 38 36 65 64 33 37 34 12 06 56 61 73 69 6c 79 1a |86ed374..Vasily.| +00000210 07 53 69 64 6f 72 6f 76 22 04 6d 61 6c 65 2a 0a |.Sidorov".male*.| +00000220 31 39 39 35 2d 30 37 2d 32 38 3a 0d 2b 34 34 32 |1995-07-28:.+442| +00000230 30 31 32 33 34 35 36 37 38 42 01 31 4a 13 32 30 |012345678B.1J.20| +00000240 31 38 2d 31 32 2d 33 30 20 30 30 3a 30 30 3a 30 |18-12-30 00:00:0| +00000250 30 52 02 32 33 5a 03 6c 65 6f 62 05 53 75 6e 6e |0R.23Z.leob.Sunn| +00000260 79 6a 03 32 35 30 6a 03 32 34 34 6a 02 31 30 72 |yj.250j.244j.10r| +00000270 08 4d 75 72 6d 61 6e 73 6b 7a 09 36 38 2e 39 37 |.Murmanskz.68.97| +00000280 30 36 38 32 7a 09 33 33 2e 30 37 34 39 38 31 82 |0682z.33.074981.| +00000290 01 10 33 2e 31 34 31 35 39 32 36 35 33 35 38 39 |..3.141592653589| +000002a0 37 39 8a 01 0f 31 30 30 30 30 30 30 30 30 30 30 |79...10000000000| +000002b0 30 2e 30 30 92 01 03 38 30 30 9a 01 04 2d 33 2e |0.00...800...-3.| +000002c0 32 a2 01 09 31 35 34 34 30 30 30 30 30 aa 01 0b |2...154400000...| +000002d0 0a 05 70 6f 75 6e 64 12 02 31 36 b2 01 07 0a 05 |..pound..16.....| +000002e0 0a 03 35 30 33 |..503| +000002e5 + +MESSAGE #1 AT 0x00000002 +uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49" +name: "Ivan" +surname: "Petrov" +gender: "male" +birthDate: "1980-12-29" +phoneNumber: "+74951234567\000" +isOnline: "1" +visitTime: "2019-01-05 18:45:00" +age: "38" +zodiacSign: "capricorn" +songs: "Yesterday" +songs: "Flowers" +color: "255" +color: "0" +color: "0" +hometown: "Moscow" +location: "55.753215" +location: "37.622504" +pi: "3.14" +lotteryWin: "214.10" +someRatio: "0.1" +temperature: "5.8" +randomBigNumber: "17060000000" +measureUnits { + unit: "meter" + unit: "centimeter" + unit: "kilometer" + coef: "1" + coef: "0.01" + coef: "1000" +} +nestiness_a { + b_c { + d: "500" + e: "501" + e: "502" + } +} +MESSAGE #2 AT 0x0000012B +uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5" +name: "Natalia" +surname: "Sokolova" +gender: "female" +birthDate: "1992-03-08" +isOnline: "0" +age: "26" +zodiacSign: "pisces" +color: "100" +color: "200" +color: "50" +hometown: "Plymouth" +location: "50.403724" +location: "-4.142123" +pi: "3.14159" +someRatio: "0.007" +temperature: "5.4" +randomBigNumber: "-20000000000000" +MESSAGE #3 AT 0x000001E1 +uuid: "a7da1aa6-f425-4789-8947-b034786ed374" +name: "Vasily" +surname: "Sidorov" +gender: "male" +birthDate: "1995-07-28" +phoneNumber: "+442012345678" +isOnline: "1" +visitTime: "2018-12-30 00:00:00" +age: "23" +zodiacSign: "leo" +songs: "Sunny" +color: "250" +color: "244" +color: "10" +hometown: "Murmansk" +location: "68.970682" +location: "33.074981" +pi: "3.14159265358979" +lotteryWin: "100000000000.00" +someRatio: "800" +temperature: "-3.2" +randomBigNumber: "154400000" +measureUnits { + unit: "pound" + coef: "16" +} +nestiness_a { + b_c { + d: "503" + } +} + +Binary representation is as expected + +Roundtrip: +a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 \N +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753215,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] +c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 \N \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] +a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 \N +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970682,33.074981] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] + +Schema 00825_protobuf_format_syntax2:Syntax2Person + +Binary representation: +00000000 f1 01 0a 24 61 37 35 32 32 31 35 38 2d 33 64 34 |...$a7522158-3d4| +00000010 31 2d 34 62 37 37 2d 61 64 36 39 2d 36 63 35 39 |1-4b77-ad69-6c59| +00000020 38 65 65 35 35 63 34 39 12 04 49 76 61 6e 1a 06 |8ee55c49..Ivan..| +00000030 50 65 74 72 6f 76 20 01 28 af 1f 32 03 70 6e 67 |Petrov .(..2.png| +00000040 3a 0d 2b 37 34 39 35 31 32 33 34 35 36 37 00 40 |:.+74951234567.@| +00000050 01 4d fc d0 30 5c 50 26 58 09 62 09 59 65 73 74 |.M..0\P&X.b.Yest| +00000060 65 72 64 61 79 62 07 46 6c 6f 77 65 72 73 68 ff |erdayb.Flowersh.| +00000070 01 68 00 68 00 72 06 4d 6f 73 63 6f 77 7a 08 4b |.h.h.r.Moscowz.K| +00000080 03 5f 42 72 7d 16 42 81 01 1f 85 eb 51 b8 1e 09 |._Br}.B.....Q...| +00000090 40 89 01 33 33 33 33 33 c3 6a 40 95 01 cd cc cc |@..33333.j@.....| +000000a0 3d 9d 01 9a 99 b9 40 a0 01 80 c4 d7 8d 7f ab 01 |=.....@.........| +000000b0 0d 00 00 80 3f 0d 0a d7 23 3c 0d 00 00 7a 44 12 |....?...#<...zD.| +000000c0 05 6d 65 74 65 72 12 0a 63 65 6e 74 69 6d 65 74 |.meter..centimet| +000000d0 65 72 12 09 6b 69 6c 6f 6d 65 74 65 72 ac 01 b3 |er..kilometer...| +000000e0 01 0b a2 06 0b 0b 08 f4 03 10 f5 03 10 f6 03 0c |................| +000000f0 0c b4 01 83 01 0a 24 63 36 39 34 61 64 38 61 2d |......$c694ad8a-| +00000100 66 37 31 34 2d 34 65 61 33 2d 39 30 37 64 2d 66 |f714-4ea3-907d-f| +00000110 64 35 34 66 62 32 35 64 39 62 35 12 07 4e 61 74 |d54fb25d9b5..Nat| +00000120 61 6c 69 61 1a 08 53 6f 6b 6f 6c 6f 76 61 20 00 |alia..Sokolova .| +00000130 28 a6 3f 32 03 6a 70 67 40 00 50 1a 58 0b 68 64 |(.?2.jpg@.P.X.hd| +00000140 68 c8 01 68 32 72 08 50 6c 79 6d 6f 75 74 68 7a |h..h2r.Plymouthz| +00000150 08 6a 9d 49 42 46 8c 84 c0 81 01 6e 86 1b f0 f9 |.j.IBF.....n....| +00000160 21 09 40 95 01 42 60 e5 3b 9d 01 cd cc ac 40 a0 |!.@..B`.;.....@.| +00000170 01 ff ff a9 ce 93 8c 09 c3 01 0a 24 61 37 64 61 |...........$a7da| +00000180 31 61 61 36 2d 66 34 32 35 2d 34 37 38 39 2d 38 |1aa6-f425-4789-8| +00000190 39 34 37 2d 62 30 33 34 37 38 36 65 64 33 37 34 |947-b034786ed374| +000001a0 12 06 56 61 73 69 6c 79 1a 07 53 69 64 6f 72 6f |..Vasily..Sidoro| +000001b0 76 20 01 28 fb 48 32 03 62 6d 70 3a 0d 2b 34 34 |v .(.H2.bmp:.+44| +000001c0 32 30 31 32 33 34 35 36 37 38 40 01 4d 50 e0 27 |2012345678@.MP.'| +000001d0 5c 50 17 58 04 62 05 53 75 6e 6e 79 68 fa 01 68 |\P.X.b.Sunnyh..h| +000001e0 f4 01 68 0a 72 08 4d 75 72 6d 61 6e 73 6b 7a 08 |..h.r.Murmanskz.| +000001f0 fd f0 89 42 c8 4c 04 42 81 01 11 2d 44 54 fb 21 |...B.L.B...-DT.!| +00000200 09 40 89 01 00 00 00 e8 76 48 37 42 95 01 00 00 |.@......vH7B....| +00000210 48 44 9d 01 cd cc 4c c0 a0 01 80 d4 9f 93 01 ab |HD....L.........| +00000220 01 0d 00 00 80 41 12 05 70 6f 75 6e 64 ac 01 b3 |.....A..pound...| +00000230 01 0b a2 06 05 0b 08 f7 03 0c 0c b4 01 |.............| +0000023d + +MESSAGE #1 AT 0x00000002 +uuid: "a7522158-3d41-4b77-ad69-6c598ee55c49" +name: "Ivan" +surname: "Petrov" +gender: male +birthDate: 4015 +photo: "png" +phoneNumber: "+74951234567\000" +isOnline: true +visitTime: 1546703100 +age: 38 +zodiacSign: capricorn +songs: "Yesterday" +songs: "Flowers" +color: 255 +color: 0 +color: 0 +hometown: "Moscow" +location: 55.7532158 +location: 37.6225052 +pi: 3.14 +lotteryWin: 214.1 +someRatio: 0.1 +temperature: 5.8 +randomBigNumber: 17060000000 +MeasureUnits { + coef: 1 + coef: 0.01 + coef: 1000 + unit: "meter" + unit: "centimeter" + unit: "kilometer" +} +Nestiness { + A { + b { + C { + d: 500 + e: 501 + e: 502 + } + } + } +} +MESSAGE #2 AT 0x000000F5 +uuid: "c694ad8a-f714-4ea3-907d-fd54fb25d9b5" +name: "Natalia" +surname: "Sokolova" +gender: female +birthDate: 8102 +photo: "jpg" +isOnline: false +age: 26 +zodiacSign: pisces +color: 100 +color: 200 +color: 50 +hometown: "Plymouth" +location: 50.4037247 +location: -4.14212322 +pi: 3.14159 +someRatio: 0.007 +temperature: 5.4 +randomBigNumber: -20000000000000 +MESSAGE #3 AT 0x0000017A +uuid: "a7da1aa6-f425-4789-8947-b034786ed374" +name: "Vasily" +surname: "Sidorov" +gender: male +birthDate: 9339 +photo: "bmp" +phoneNumber: "+442012345678" +isOnline: true +visitTime: 1546117200 +age: 23 +zodiacSign: leo +songs: "Sunny" +color: 250 +color: 244 +color: 10 +hometown: "Murmansk" +location: 68.9706802 +location: 33.0749817 +pi: 3.14159265358979 +lotteryWin: 100000000000 +someRatio: 800 +temperature: -3.2 +randomBigNumber: 154400000 +MeasureUnits { + coef: 16 + unit: "pound" +} +Nestiness { + A { + b { + C { + d: 503 + } + } + } +} + +Binary representation is as expected + +Roundtrip: +a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] +c694ad8a-f714-4ea3-907d-fd54fb25d9b5 Natalia Sokolova female 1992-03-08 jpg \N 0 \N 26 pisces [] [100,200,50] Plymouth [50.403724,-4.142123] 3.14159 \N 0.007 5.4 -20000000000000 [] [] \N [] +a7da1aa6-f425-4789-8947-b034786ed374 Vasily Sidorov male 1995-07-28 bmp +442012345678 1 2018-12-30 00:00:00 23 leo ['Sunny'] [250,244,10] Murmansk [68.970680,33.074982] 3.14159265358979 100000000000.00 800 -3.2 154400000 ['pound'] [16] 503 [] diff --git a/tests/queries/0_stateless/00825_protobuf_format_persons.sh b/tests/queries/0_stateless/00825_protobuf_format_persons.sh new file mode 100755 index 00000000000..957ed738d99 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_persons.sh @@ -0,0 +1,118 @@ +#!/usr/bin/env bash + +# To generate reference file for this test use the following commands: +# ninja ProtobufDelimitedMessagesSerializer +# build/utils/test-data-generator/ProtobufDelimitedMessagesSerializer + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -eo pipefail + +# Run the client. +$CLICKHOUSE_CLIENT --multiquery < $BINARY_FILE_PATH +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_persons:Person" --input "$BINARY_FILE_PATH" +echo +echo "Roundtrip:" +$CLICKHOUSE_CLIENT --query "CREATE TABLE roundtrip_persons_00825 AS persons_00825" +$CLICKHOUSE_CLIENT --query "INSERT INTO roundtrip_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:Person'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM roundtrip_persons_00825 ORDER BY name" +rm "$BINARY_FILE_PATH" + +# Use schema 00825_protobuf_format_persons:AltPerson +echo +echo "Schema 00825_protobuf_format_persons:AltPerson" +BINARY_FILE_PATH=$(mktemp "$CURDIR/00825_protobuf_format_persons.XXXXXX.binary") +$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_persons:AltPerson'" > $BINARY_FILE_PATH +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_persons:AltPerson" --input "$BINARY_FILE_PATH" +echo +echo "Roundtrip:" +$CLICKHOUSE_CLIENT --query "CREATE TABLE alt_persons_00825 AS persons_00825" +$CLICKHOUSE_CLIENT --query "INSERT INTO alt_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:AltPerson'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM alt_persons_00825 ORDER BY name" +rm "$BINARY_FILE_PATH" + +# Use schema 00825_protobuf_format_persons:StrPerson +echo +echo "Schema 00825_protobuf_format_persons:StrPerson" +BINARY_FILE_PATH=$(mktemp "$CURDIR/00825_protobuf_format_persons.XXXXXX.binary") +$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_persons:StrPerson'" > $BINARY_FILE_PATH +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_persons:StrPerson" --input "$BINARY_FILE_PATH" +echo +echo "Roundtrip:" +$CLICKHOUSE_CLIENT --query "CREATE TABLE str_persons_00825 AS persons_00825" +$CLICKHOUSE_CLIENT --query "INSERT INTO str_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons:StrPerson'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM str_persons_00825 ORDER BY name" +rm "$BINARY_FILE_PATH" + +# Use schema 00825_protobuf_format_syntax2:Syntax2Person +echo +echo "Schema 00825_protobuf_format_syntax2:Syntax2Person" +BINARY_FILE_PATH=$(mktemp "$CURDIR/00825_protobuf_format_persons.XXXXXX.binary") +$CLICKHOUSE_CLIENT --query "SELECT * FROM persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person'" > $BINARY_FILE_PATH +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person" --input "$BINARY_FILE_PATH" +echo +echo "Roundtrip:" +$CLICKHOUSE_CLIENT --query "CREATE TABLE syntax2_persons_00825 AS persons_00825" +$CLICKHOUSE_CLIENT --query "INSERT INTO syntax2_persons_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_persons_syntax2:Syntax2Person'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM syntax2_persons_00825 ORDER BY name" +rm "$BINARY_FILE_PATH" + +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" + +# Check the output in the protobuf format +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_squares:NumberAndSquare" --input "$BINARY_FILE_PATH" + +# Check the input in the protobuf format (now the table contains the same data twice). +echo +$CLICKHOUSE_CLIENT --query "INSERT INTO squares_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_squares:NumberAndSquare'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM squares_protobuf_00825" + +rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE squares_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_table_default.sh b/tests/queries/0_stateless/00825_protobuf_format_table_default.sh index 97f7769269a..96664ca7f20 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_table_default.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_table_default.sh @@ -1,13 +1,14 @@ #!/usr/bin/env bash CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh set -eo pipefail # Run the client. -$CLICKHOUSE_CLIENT --multiquery <<'EOF' +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM table_default_protobuf_00825 ORDER BY x,y,z FORMAT Protobuf SETTINGS format_schema = '$SCHEMADIR/00825_protobuf_format_table_default:Message'" > "$BINARY_FILE_PATH" # Check the output in the protobuf format echo -$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$CURDIR/00825_protobuf_format_table_default:Message" --input "$BINARY_FILE_PATH" +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_table_default:Message" --input "$BINARY_FILE_PATH" # Check the input in the protobuf format (now the table contains the same data twice). echo -$CLICKHOUSE_CLIENT --query "INSERT INTO table_default_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$CURDIR/00825_protobuf_format_table_default:Message'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "INSERT INTO table_default_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_table_default:Message'" < "$BINARY_FILE_PATH" $CLICKHOUSE_CLIENT --query "SELECT * FROM table_default_protobuf_00825 ORDER BY x,y,z" rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE table_default_protobuf_00825" diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_3dim.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_array_3dim.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_array_3dim.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_array_3dim.proto diff --git a/tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_array_of_arrays.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_array_of_arrays.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_array_of_arrays.proto diff --git a/tests/queries/0_stateless/00825_protobuf_format_enum_mapping.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_enum_mapping.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_enum_mapping.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_enum_mapping.proto diff --git a/tests/queries/0_stateless/00825_protobuf_format_map.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_map.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_map.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_map.proto diff --git a/tests/queries/0_stateless/00825_protobuf_format_nested_optional.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_nested_optional.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_nested_optional.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_nested_optional.proto diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_no_length_delimiter.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_no_length_delimiter.proto new file mode 100644 index 00000000000..9726d0dede1 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_no_length_delimiter.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message Message { + int32 x = 1; + string str = 2; +}; diff --git a/tests/queries/0_stateless/00825_protobuf_format.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto similarity index 97% rename from tests/queries/0_stateless/00825_protobuf_format.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto index 0d9bdd83ccd..b588619f488 100644 --- a/tests/queries/0_stateless/00825_protobuf_format.proto +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons.proto @@ -143,9 +143,3 @@ message StrPerson { MeasureUnits measureUnits = 21; NestinessA nestiness_a = 22; }; - -message NumberAndSquare -{ - uint32 number = 1; - uint64 square = 2; -}; diff --git a/tests/queries/0_stateless/00825_protobuf_format_syntax2.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons_syntax2.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_syntax2.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_persons_syntax2.proto diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_squares.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_squares.proto new file mode 100644 index 00000000000..bca321568f3 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_squares.proto @@ -0,0 +1,6 @@ +syntax = "proto3"; + +message NumberAndSquare { + uint32 number = 1; + uint64 square = 2; +}; diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_syntax2.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_syntax2.proto new file mode 100644 index 00000000000..d4abea38d4e --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_syntax2.proto @@ -0,0 +1,63 @@ +syntax = "proto2"; + +message Syntax2Person { + enum Gender { + female = 0; + male = 1; + }; + + enum ZodiacSign { + aries = 0; + taurus = 1; + gemini = 2; + cancer = 3; + leo = 4; + virgo = 5; + libra = 6; + scorpius = 7; + sagittarius = 8; + capricorn = 9; + aquarius = 10; + pisces = 11; + }; + + required string uuid = 1; + required string name = 2; + required string surname = 3; + required Gender gender = 4; + required uint32 birthDate = 5; + optional bytes photo = 6; + optional string phoneNumber = 7; + optional bool isOnline = 8; + optional fixed32 visitTime = 9; + optional uint32 age = 10; + optional ZodiacSign zodiacSign = 11; + repeated string songs = 12; + repeated uint32 color = 13; + optional string hometown = 14 [default='Moscow']; + repeated float location = 15 [packed=true]; + optional double pi = 16; + optional double lotteryWin = 17; + optional float someRatio = 18; + optional float temperature = 19; + optional sint64 randomBigNumber = 20; + optional group MeasureUnits = 21 { + repeated float coef = 1; + repeated string unit = 2; + }; + optional group Nestiness = 22 + { + optional group A = 1 { + message SubB { + optional group C = 1 { + optional uint32 d = 1; + repeated uint32 e = 2; + }; + }; + optional SubB b = 100; + }; + }; + optional string newFieldStr = 23 [default='abc']; + optional int32 newFieldInt = 24 [default=-11]; + optional bool newBool = 25 [default=true]; +}; diff --git a/tests/queries/0_stateless/00825_protobuf_format_table_default.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_table_default.proto similarity index 100% rename from tests/queries/0_stateless/00825_protobuf_format_table_default.proto rename to tests/queries/0_stateless/format_schemas/00825_protobuf_format_table_default.proto diff --git a/tests/queries/0_stateless/helpers/protobuf_length_delimited_encoder.py b/tests/queries/0_stateless/helpers/protobuf_length_delimited_encoder.py index 3ed42f1c820..86c5048c8a3 100755 --- a/tests/queries/0_stateless/helpers/protobuf_length_delimited_encoder.py +++ b/tests/queries/0_stateless/helpers/protobuf_length_delimited_encoder.py @@ -12,18 +12,18 @@ import tempfile def read_varint(input): res = 0 - shift = 0 + multiplier = 1 while True: c = input.read(1) if len(c) == 0: return None b = c[0] if b < 0x80: - res += b << shift + res += b * multiplier break b -= 0x80 - res += b << shift - shift = shift << 7 + res += b * multiplier + multiplier *= 0x80 return res def write_varint(output, value): From 48d5d75282cecc1f7412ccf594e9b7e8d6f0425a Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 20 Feb 2021 23:47:18 +0300 Subject: [PATCH 289/510] Make unit test more stable --- src/Coordination/tests/gtest_for_build.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index f871f39a906..350d60f64c2 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -204,7 +204,7 @@ TEST(CoordinationTest, TestSummingRaft3) ChangelogDirTest test3("./logs3"); SummingRaftServer s3(3, "localhost", 44446, "./logs3"); - nuraft::srv_config first_config(1, "localhost:44444"); + nuraft::srv_config first_config(1, 0, "localhost:44444", "", false, 0); auto ret1 = s2.raft_instance->add_srv(first_config); if (!ret1->get_accepted()) { @@ -219,7 +219,7 @@ TEST(CoordinationTest, TestSummingRaft3) std::this_thread::sleep_for(std::chrono::milliseconds(100)); } - nuraft::srv_config third_config(3, "localhost:44446"); + nuraft::srv_config third_config(3, 0, "localhost:44446", "", false, 0); auto ret3 = s2.raft_instance->add_srv(third_config); if (!ret3->get_accepted()) { From 331de3d6539490ebe25ba04b0a089d5db2dd803d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Feb 2021 00:08:19 +0300 Subject: [PATCH 290/510] Trying to add tests more stable --- src/Coordination/Changelog.cpp | 3 +++ src/Coordination/CoordinationSettings.h | 6 +++--- src/Coordination/NuKeeperStateManager.cpp | 2 +- tests/config/config.d/test_keeper_port.xml | 3 ++- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index adf367c565d..6ffd6a6f04b 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -213,6 +213,8 @@ public: logs.emplace(record.header.index, log_entry); index_to_offset[record.header.index] = result.last_position; + if (result.entries_read % 50000 == 0) + LOG_TRACE(log, "Reading changelog from path {}, entries {}", filepath, result.entries_read); } } catch (const Exception & ex) @@ -228,6 +230,7 @@ public: result.error = true; tryLogCurrentException(log); } + LOG_TRACE(log, "Totally read from changelog {} {} entries", filepath, result.entries_read); return result; } diff --git a/src/Coordination/CoordinationSettings.h b/src/Coordination/CoordinationSettings.h index ba3d3a7141a..34a97f82399 100644 --- a/src/Coordination/CoordinationSettings.h +++ b/src/Coordination/CoordinationSettings.h @@ -22,14 +22,14 @@ struct Settings; M(Milliseconds, heart_beat_interval_ms, 500, "Heartbeat interval between quorum nodes", 0) \ M(Milliseconds, election_timeout_lower_bound_ms, 1000, "Lower bound of election timer (avoid too often leader elections)", 0) \ M(Milliseconds, election_timeout_upper_bound_ms, 2000, "Lower bound of election timer (avoid too often leader elections)", 0) \ - M(UInt64, reserved_log_items, 5000, "How many log items to store (don't remove during compaction)", 0) \ - M(UInt64, snapshot_distance, 5000, "How many log items we have to collect to write new snapshot", 0) \ + M(UInt64, reserved_log_items, 50000, "How many log items to store (don't remove during compaction)", 0) \ + M(UInt64, snapshot_distance, 100000, "How many log items we have to collect to write new snapshot", 0) \ M(UInt64, max_stored_snapshots, 3, "How many snapshots we want to store", 0) \ M(Bool, auto_forwarding, true, "Allow to forward write requests from followers to leader", 0) \ M(Milliseconds, shutdown_timeout, 5000, "How many time we will until RAFT shutdown", 0) \ M(Milliseconds, startup_timeout, 30000, "How many time we will until RAFT to start", 0) \ M(LogsLevel, raft_logs_level, LogsLevel::information, "Log internal RAFT logs into main server log level. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \ - M(UInt64, rotate_log_storage_interval, 500000, "How many records will be stored in one log storage file", 0) \ + M(UInt64, rotate_log_storage_interval, 100000, "How many records will be stored in one log storage file", 0) \ M(Bool, force_sync, true, " Call fsync on each change in RAFT changelog", 0) DECLARE_SETTINGS_TRAITS(CoordinationSettingsTraits, LIST_OF_COORDINATION_SETTINGS) diff --git a/src/Coordination/NuKeeperStateManager.cpp b/src/Coordination/NuKeeperStateManager.cpp index 14e8badd92f..83cb18e27fe 100644 --- a/src/Coordination/NuKeeperStateManager.cpp +++ b/src/Coordination/NuKeeperStateManager.cpp @@ -12,7 +12,7 @@ namespace ErrorCodes NuKeeperStateManager::NuKeeperStateManager(int server_id_, const std::string & host, int port, const std::string & logs_path) : my_server_id(server_id_) , my_port(port) - , log_store(nuraft::cs_new(logs_path, 5000, true)) + , log_store(nuraft::cs_new(logs_path, 5000, false)) , cluster_config(nuraft::cs_new()) { auto peer_config = nuraft::cs_new(my_server_id, host + ":" + std::to_string(port)); diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index 88fbf027ce7..1e627bb8b0e 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -7,9 +7,10 @@ 10000 30000 - 0 + 200000 0 false + 60 From 5bf8c42e04724fb55bde25cd4dfc5412e35fb5c5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Feb 2021 10:18:39 +0300 Subject: [PATCH 291/510] Fix config --- tests/config/config.d/test_keeper_port.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index 1e627bb8b0e..6579cac7173 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -10,7 +10,7 @@ 200000 0 false - 60 + 60000 From dda1eb1f005e092738923be1504f83327c4ccddc Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 21 Feb 2021 13:17:58 +0400 Subject: [PATCH 292/510] Fix the test --- tests/integration/test_row_policy/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index 5781686067f..c963e87ea2a 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -124,11 +124,11 @@ def test_cannot_trick_row_policy_with_keyword_with(): assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1, 0], [1, 0, 0], [0, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) def test_policy_from_users_xml_affects_only_user_assigned(): From 4f57b685634e4c164678f92dd82fbbb1b7e06084 Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Sun, 21 Feb 2021 13:22:41 +0400 Subject: [PATCH 293/510] Fix the test --- tests/integration/test_row_policy/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_row_policy/test.py b/tests/integration/test_row_policy/test.py index c963e87ea2a..ffb6dcb0588 100644 --- a/tests/integration/test_row_policy/test.py +++ b/tests/integration/test_row_policy/test.py @@ -124,11 +124,11 @@ def test_cannot_trick_row_policy_with_keyword_with(): assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1], [1, 0]]) assert node.query("WITH 0 AS c SELECT * FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1], [1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) - assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 0, 0], [0, 1, 0], [1, 0, 0], [1, 1, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3") == TSV([[0, 1, 0], [1, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 WHERE c >= 0 AND a >= 0 SETTINGS optimize_move_to_prewhere = 0") == TSV([[0, 1, 0], [1, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 AND a >= 0") == TSV([[0, 1, 0], [1, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE c >= 0 WHERE a >= 0") == TSV([[0, 1, 0], [1, 0, 0]]) + assert node.query("WITH 0 AS c SELECT a, b, c FROM mydb.filtered_table3 PREWHERE a >= 0 WHERE c >= 0") == TSV([[0, 1, 0], [1, 0, 0]]) def test_policy_from_users_xml_affects_only_user_assigned(): From cfb0dbb9e5f069342fe096542d888cbb2a01bc02 Mon Sep 17 00:00:00 2001 From: George Date: Sun, 21 Feb 2021 15:01:10 +0300 Subject: [PATCH 294/510] updated and added links --- .../functions/tuple-map-functions.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index f8755f1e2a9..ac8d92750fd 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -114,7 +114,7 @@ Result: ## mapContains {#mapcontains} -Determines whether `map.keys` contains the `key` parameter. +Determines whether the `map` contains the `key` parameter. **Syntax** @@ -124,12 +124,12 @@ mapContains(map, key) **Parameters** -- `map` — Map. -- `key` — Key. Type matches the type of `map.keys`. +- `map` — Map. [Map](../../sql-reference/data-types/map.md). +- `key` — Key. Type matches the type of `key` parameter. **Returned value** -- `1` if `map.keys` contains `key`, `0` if not. +- `1` if `map` contains `key`, `0` if not. Type: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -157,7 +157,7 @@ Result: ## mapKeys {#mapkeys} -Returns all the keys from `map` parameter. +Returns all keys from the `map` parameter. **Syntax** @@ -167,11 +167,11 @@ mapKeys(map) **Parameters** -- `map` — Map. +- `map` — Map. [Map](../../sql-reference/data-types/map.md). **Returned value** -- Array containing all the keys from `map`. +- Array containing all keys from the `map`. Type: [Array](../../sql-reference/data-types/array.md). @@ -198,7 +198,7 @@ Result: ## mapValues {#mapvalues} -Returns all the values from `map` parameter. +Returns all values from the `map` parameter. **Syntax** @@ -208,7 +208,7 @@ mapKeys(map) **Parameters** -- `map` — Map. +- `map` — Map. [Map](../../sql-reference/data-types/map.md). **Returned value** From bff08e32bfb00e1e73be8ab23c8f1263dc8ab455 Mon Sep 17 00:00:00 2001 From: George Date: Sun, 21 Feb 2021 15:22:48 +0300 Subject: [PATCH 295/510] updates --- .../sql-reference/functions/tuple-map-functions.md | 2 +- .../sql-reference/functions/tuple-map-functions.md | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 8bc36d19ed7..1d4839cbbf9 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -187,7 +187,7 @@ mapContains(map, key) **Parameters** - `map` — Map. [Map](../../sql-reference/data-types/map.md). -- `key` — Key. Type matches the type of `key` parameter. +- `key` — Key. Type matches the type of keys of `map` parameter. **Returned value** diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 7c69f2ca4ef..9516deb243a 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -178,7 +178,7 @@ select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type ## mapContains {#mapcontains} -Определяет, включает ли в себя `map.keys` параметр `key`. +Определяет, включает ли в себя `map` параметр `key`. **Синтаксис** @@ -188,12 +188,12 @@ mapContains(map, key) **Параметры** -- `map` — контейнер map. -- `key` — ключ. Тип соответстует типу `map.keys`. +- `map` — контейнер Map. [Map](../../sql-reference/data-types/map.md). +- `key` — ключ. Тип соответстует типу ключей параметра `map`. **Возвращаемое значение** -- `1` если `map.keys` включает `key`, иначе `0`. +- `1` если `map` включает `key`, иначе `0`. Тип: [UInt8](../../sql-reference/data-types/int-uint.md). @@ -231,7 +231,7 @@ mapKeys(map) **Параметры** -- `map` — контейнер map. +- `map` — контейнер Map. [Map](../../sql-reference/data-types/map.md). **Возвращаемое значение** @@ -272,7 +272,7 @@ mapKeys(map) **Параметры** -- `map` — контейнер map. +- `map` — контейнер Map. [Map](../../sql-reference/data-types/map.md). **Возвращаемое значение** From e447dd72f473700f0803bd8c26a8748f81fb3c2d Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Feb 2021 17:00:10 +0300 Subject: [PATCH 296/510] Disable snapshots --- tests/config/config.d/test_keeper_port.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/config.d/test_keeper_port.xml b/tests/config/config.d/test_keeper_port.xml index 6579cac7173..d66183514d5 100644 --- a/tests/config/config.d/test_keeper_port.xml +++ b/tests/config/config.d/test_keeper_port.xml @@ -7,7 +7,7 @@ 10000 30000 - 200000 + 0 0 false 60000 From dd5185c77942f81dedea139a1c837ba0cd7a3f75 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:03:24 +0300 Subject: [PATCH 297/510] Refactor 4.0 --- src/Client/ConnectionEstablisher.cpp | 285 ++++++++-------- src/Client/ConnectionEstablisher.h | 131 +++---- src/Client/ConnectionPoolWithFailover.cpp | 8 +- src/Client/ConnectionPoolWithFailover.h | 2 +- src/Client/HedgedConnections.cpp | 183 +++++----- src/Client/HedgedConnections.h | 17 +- src/Client/HedgedConnectionsFactory.cpp | 323 ++++++++++-------- src/Client/HedgedConnectionsFactory.h | 43 ++- src/Client/PacketReceiver.h | 119 +++++-- src/Common/Epoll.cpp | 7 +- src/Common/Epoll.h | 9 +- src/IO/ReadBufferFromPocoSocket.cpp | 6 + src/Interpreters/tests/CMakeLists.txt | 3 + .../test.py | 1 + 14 files changed, 639 insertions(+), 498 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index f92d878b670..fb8e977b569 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -16,153 +16,24 @@ namespace ErrorCodes extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; + extern const int LOGICAL_ERROR; } - ConnectionEstablisher::ConnectionEstablisher( IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings * settings_, - const QualifiedTableName * table_to_check_) - : pool(pool_), timeouts(timeouts_), settings(settings_), table_to_check(table_to_check_), - stage(Stage::INITIAL), log(&Poco::Logger::get("ConnectionEstablisher")) + Poco::Logger * log_, + const QualifiedTableName * table_to_check_) : pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_) { -#if defined(OS_LINUX) - epoll.add(receive_timeout.getDescriptor()); -#endif } -void ConnectionEstablisher::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &) -{ -#if defined(OS_LINUX) - if (connection_establisher.socket_fd != fd) - { - if (connection_establisher.socket_fd != -1) - connection_establisher.epoll.remove(connection_establisher.socket_fd); - - connection_establisher.epoll.add(fd); - connection_establisher.socket_fd = fd; - } - - connection_establisher.receive_timeout.setRelative(timeout); - fiber = std::move(fiber).resume(); - connection_establisher.receive_timeout.reset(); -#else - (void) fd; - (void) timeout; -#endif -} - -Fiber ConnectionEstablisher::Routine::operator()(Fiber && sink) +void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std::string & fail_message) { + is_finished = false; + SCOPE_EXIT(is_finished = true); try { - connection_establisher.establishConnection(ReadCallback{connection_establisher, sink}); - } - catch (const boost::context::detail::forced_unwind &) - { - /// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited - /// It should not be caught or it will segfault. - /// Other exceptions must be caught - throw; - } - catch (...) - { - connection_establisher.exception = std::current_exception(); - } - - return std::move(sink); -} - -void ConnectionEstablisher::resume() -{ - if (!fiber_created) - { - reset(); - fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); - fiber_created = true; - resumeFiber(); - return; - } - -#if defined(OS_LINUX) - bool is_socket_ready = false; - bool is_receive_timeout_alarmed = false; - - epoll_event events[2]; - events[0].data.fd = events[1].data.fd = -1; - size_t ready_count = epoll.getManyReady(2, events, true); - for (size_t i = 0; i != ready_count; ++i) - { - if (events[i].data.fd == socket_fd) - is_socket_ready = true; - if (events[i].data.fd == receive_timeout.getDescriptor()) - is_receive_timeout_alarmed = true; - } - - if (is_receive_timeout_alarmed && !is_socket_ready) - processReceiveTimeout(); -#endif - - resumeFiber(); -} - -void ConnectionEstablisher::cancel() -{ - destroyFiber(); - reset(); -} - -void ConnectionEstablisher::processReceiveTimeout() -{ -#if defined(OS_LINUX) - destroyFiber(); - stage = Stage::FAILED; - fail_message = "Code: 209, e.displayText() = DB::NetException: Timeout exceeded while reading from socket (" + result.entry->getDescription() + ")"; - epoll.remove(socket_fd); - resetResult(); -#endif -} - -void ConnectionEstablisher::resetResult() -{ - if (!result.entry.isNull()) - { - result.entry->disconnect(); - result.reset(); - } -} - -void ConnectionEstablisher::reset() -{ - stage = Stage::INITIAL; - resetResult(); - fail_message.clear(); - socket_fd = -1; -} - -void ConnectionEstablisher::resumeFiber() -{ - fiber = std::move(fiber).resume(); - - if (exception) - std::rethrow_exception(std::move(exception)); - - if (stage == Stage::FAILED) - destroyFiber(); -} - -void ConnectionEstablisher::destroyFiber() -{ - Fiber to_destroy = std::move(fiber); - fiber_created = false; -} - -void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) -{ - try - { - stage = Stage::IN_PROCESS; result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); @@ -175,7 +46,6 @@ void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) result.entry->forceConnected(*timeouts); result.is_usable = true; result.is_up_to_date = true; - stage = Stage::FINISHED; return; } @@ -192,8 +62,6 @@ void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) fail_message = fmt::format(message_pattern, backQuote(table_to_check->database), backQuote(table_to_check->table), result.entry->getDescription()); LOG_WARNING(log, fail_message); ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); - - stage = Stage::FINISHED; return; } @@ -203,7 +71,6 @@ void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) if (!max_allowed_delay) { result.is_up_to_date = true; - stage = Stage::FINISHED; return; } @@ -219,7 +86,6 @@ void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) LOG_TRACE(log, "Server {} has unacceptable replica delay for table {}.{}: {}", result.entry->getDescription(), table_to_check->database, table_to_check->table, delay); ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); } - stage = Stage::FINISHED; } catch (const Exception & e) { @@ -228,9 +94,144 @@ void ConnectionEstablisher::establishConnection(AsyncCallback async_callback) throw; fail_message = getCurrentExceptionMessage(/* with_stacktrace = */ false); - resetResult(); - stage = Stage::FAILED; + + if (!result.entry.isNull()) + { + result.entry->disconnect(); + result.reset(); + } } } +ConnectionEstablisherAsync::ConnectionEstablisherAsync( + IConnectionPool * pool_, + const ConnectionTimeouts * timeouts_, + const Settings * settings_, + Poco::Logger * log_, + const QualifiedTableName * table_to_check_) + : connection_establisher(pool_, timeouts_, settings_, log_, table_to_check_) +{ + epoll.add(receive_timeout.getDescriptor()); +} + +void ConnectionEstablisherAsync::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &) +{ + if (connection_establisher_async.socket_fd != fd) + { + if (connection_establisher_async.socket_fd != -1) + connection_establisher_async.epoll.remove(connection_establisher_async.socket_fd); + + connection_establisher_async.epoll.add(fd); + connection_establisher_async.socket_fd = fd; + } + + connection_establisher_async.receive_timeout.setRelative(timeout); + fiber = std::move(fiber).resume(); + connection_establisher_async.receive_timeout.reset(); +} + +Fiber ConnectionEstablisherAsync::Routine::operator()(Fiber && sink) +{ + try + { + connection_establisher_async.connection_establisher.setAsyncCallback(ReadCallback{connection_establisher_async, sink}); + connection_establisher_async.connection_establisher.run(connection_establisher_async.result, connection_establisher_async.fail_message); + } + catch (const boost::context::detail::forced_unwind &) + { + /// This exception is thrown by fiber implementation in case if fiber is being deleted but hasn't exited + /// It should not be caught or it will segfault. + /// Other exceptions must be caught + throw; + } + catch (...) + { + connection_establisher_async.exception = std::current_exception(); + } + + return std::move(sink); +} + +std::variant ConnectionEstablisherAsync::resume() +{ + if (!fiber_created) + { + reset(); + fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); + fiber_created = true; + } else if (!checkReceiveTimeout()) + return result; + + fiber = std::move(fiber).resume(); + + if (exception) + std::rethrow_exception(std::move(exception)); + + if (connection_establisher.isFinished()) + { + destroyFiber(); + return result; + } + + return epoll.getFileDescriptor(); +} + +bool ConnectionEstablisherAsync::checkReceiveTimeout() +{ + bool is_socket_ready = false; + bool is_receive_timeout_alarmed = false; + + epoll_event events[2]; + events[0].data.fd = events[1].data.fd = -1; + size_t ready_count = epoll.getManyReady(2, events, false); + for (size_t i = 0; i != ready_count; ++i) + { + if (events[i].data.fd == socket_fd) + is_socket_ready = true; + if (events[i].data.fd == receive_timeout.getDescriptor()) + is_receive_timeout_alarmed = true; + } + + if (is_receive_timeout_alarmed && !is_socket_ready) + { + destroyFiber(); + /// In not async case this exception would be thrown and caught in ConnectionEstablisher::run, + /// but in async case we process timeout outside and cannot throw exception. So, we just save fail message. + fail_message = "Timeout exceeded while reading from socket (" + result.entry->getDescription() + ")"; + epoll.remove(socket_fd); + resetResult(); + return false; + } + + return true; +} + +void ConnectionEstablisherAsync::cancel() +{ + destroyFiber(); + reset(); +} + +void ConnectionEstablisherAsync::reset() +{ + resetResult(); + fail_message.clear(); + socket_fd = -1; +} + +void ConnectionEstablisherAsync::resetResult() +{ + if (!result.entry.isNull()) + { + result.entry->disconnect(); + result.reset(); + } +} + +void ConnectionEstablisherAsync::destroyFiber() +{ + Fiber to_destroy = std::move(fiber); + fiber_created = false; +} + } diff --git a/src/Client/ConnectionEstablisher.h b/src/Client/ConnectionEstablisher.h index 8d10126b3da..1096452ebce 100644 --- a/src/Client/ConnectionEstablisher.h +++ b/src/Client/ConnectionEstablisher.h @@ -1,5 +1,7 @@ #pragma once +#include + #include #include #include @@ -10,12 +12,8 @@ namespace DB { -/// Class for nonblocking establishing connection to the replica. -/// It runs establishing connection process in fiber and sets special -/// read callback which is called when reading from socket blocks. -/// When read callback is called, socket and receive timeout are added in epoll -/// and execution returns to the main program. -/// So, you can poll this epoll file descriptor to determine when to resume. +/// Class for establishing connection to the replica. It supports setting up +/// an async callback that will be called when reading from socket blocks. class ConnectionEstablisher { public: @@ -24,60 +22,74 @@ public: ConnectionEstablisher(IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, const Settings * settings_, + Poco::Logger * log, const QualifiedTableName * table_to_check = nullptr); - /// Establish connection with replica, call async_callbeck when - /// reading from socket blocks. - void establishConnection(AsyncCallback async_callback = {}); + /// Establish connection and save it in result, write possible exception message in fail_message. + void run(TryResult & result, std::string & fail_message); - /// In the first call create fiber with establishConnection function, - /// in the next - check timeout and resume fiber. - void resume(); + /// Set async callback that will be called when reading from socket blocks. + void setAsyncCallback(AsyncCallback async_callback_) { async_callback = std::move(async_callback_); } + + bool isFinished() const { return is_finished; } + +private: + IConnectionPool * pool; + const ConnectionTimeouts * timeouts; + const Settings * settings; + Poco::Logger * log; + const QualifiedTableName * table_to_check; + + bool is_finished; + AsyncCallback async_callback = {}; + +}; + +#if defined(OS_LINUX) + +/// Class for nonblocking establishing connection to the replica. +/// It runs establishing connection process in fiber and sets special +/// read callback which is called when reading from socket blocks. +/// When read callback is called, socket and receive timeout are added in epoll +/// and execution returns to the main program. +/// So, you can poll this epoll file descriptor to determine when to resume. +class ConnectionEstablisherAsync +{ +public: + using TryResult = PoolWithFailoverBase::TryResult; + + ConnectionEstablisherAsync(IConnectionPool * pool_, + const ConnectionTimeouts * timeouts_, + const Settings * settings_, + Poco::Logger * log_, + const QualifiedTableName * table_to_check = nullptr); + + /// Resume establishing connection. If the process was not finished, + /// return file descriptor (you can add it in epoll and poll it, + /// when this fd become ready, call resume again), + /// if the process was failed or finished, return it's result, + std::variant resume(); /// Cancel establishing connections. Fiber will be destroyed, /// class will be set in initial stage. void cancel(); - bool isInProcess() const { return stage == Stage::IN_PROCESS; } - - bool isFinished() const { return stage == Stage::FINISHED; } - - bool isFailed() const { return stage == Stage::FAILED; } - - int getFileDescriptor() const - { - int fd = -1; -#if defined(OS_LINUX) - fd = epoll.getFileDescriptor(); -#endif - return fd; - } + TryResult getResult() const { return result; } const std::string & getFailMessage() const { return fail_message; } - TryResult getResult() { return result; } - - Connection * getConnection() { return &*result.entry; } - - private: - void processReceiveTimeout(); - - enum class Stage - { - INITIAL, - IN_PROCESS, - FINISHED, - FAILED, - }; + /// When epoll file descriptor is ready, check if it's an expired timeout. + /// Return false if receive timeout expired and socket is not ready, return true otherwise. + bool checkReceiveTimeout(); struct Routine { - ConnectionEstablisher & connection_establisher; + ConnectionEstablisherAsync & connection_establisher_async; struct ReadCallback { - ConnectionEstablisher & connection_establisher; + ConnectionEstablisherAsync & connection_establisher_async; Fiber & fiber; void operator()(int fd, const Poco::Timespan & timeout, const std::string &); @@ -86,31 +98,34 @@ private: Fiber operator()(Fiber && sink); }; - void resetResult(); - void reset(); + void resetResult(); + void destroyFiber(); - void resumeFiber(); - - IConnectionPool * pool; - const ConnectionTimeouts * timeouts; - std::string fail_message; - const Settings * settings; - const QualifiedTableName * table_to_check; + ConnectionEstablisher connection_establisher; TryResult result; - Stage stage; - Poco::Logger * log; + std::string fail_message; + Fiber fiber; FiberStack fiber_stack; - std::exception_ptr exception; - int socket_fd = -1; - bool fiber_created = false; -#if defined(OS_LINUX) + + /// We use timer descriptor for checking socket receive timeout. TimerDescriptor receive_timeout; + + /// In read callback we add socket file descriptor and timer descriptor with receive timeout + /// in epoll, so we can return epoll file descriptor outside for polling. Epoll epoll; -#endif + int socket_fd = -1; + std::string socket_description; + + /// If and exception occurred in fiber resume, we save it and rethrow. + std::exception_ptr exception; + + bool fiber_created = false; }; +#endif + } diff --git a/src/Client/ConnectionPoolWithFailover.cpp b/src/Client/ConnectionPoolWithFailover.cpp index acbb678d870..0c61b2bb49d 100644 --- a/src/Client/ConnectionPoolWithFailover.cpp +++ b/src/Client/ConnectionPoolWithFailover.cpp @@ -247,10 +247,10 @@ ConnectionPoolWithFailover::tryGetEntry( const Settings * settings, const QualifiedTableName * table_to_check) { - ConnectionEstablisher connection_establisher(&pool, &timeouts, settings, table_to_check); - connection_establisher.establishConnection(); - fail_message = connection_establisher.getFailMessage(); - return connection_establisher.getResult(); + ConnectionEstablisher connection_establisher(&pool, &timeouts, settings, log, table_to_check); + TryResult result; + connection_establisher.run(result, fail_message); + return result; } std::vector ConnectionPoolWithFailover::getShuffledPools(const Settings * settings) diff --git a/src/Client/ConnectionPoolWithFailover.h b/src/Client/ConnectionPoolWithFailover.h index 3bd39977566..023ef863bdf 100644 --- a/src/Client/ConnectionPoolWithFailover.h +++ b/src/Client/ConnectionPoolWithFailover.h @@ -99,7 +99,7 @@ private: /// Try to get a connection from the pool and check that it is good. /// If table_to_check is not null and the check is enabled in settings, check that replication delay /// for this table is not too large. - static TryResult tryGetEntry( + TryResult tryGetEntry( IConnectionPool & pool, const ConnectionTimeouts & timeouts, std::string & fail_message, diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 61d6d317c6e..9601c035f07 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -39,8 +39,8 @@ HedgedConnections::HedgedConnections( ReplicaState & replica = offset_states[i].replicas.back(); replica.connection->setThrottler(throttler_); - epoll.add(replica.packet_receiver.getFileDescriptor()); - fd_to_replica_location[replica.packet_receiver.getFileDescriptor()] = ReplicaLocation{i, 0}; + epoll.add(replica.packet_receiver->getFileDescriptor()); + fd_to_replica_location[replica.packet_receiver->getFileDescriptor()] = ReplicaLocation{i, 0}; epoll.add(replica.change_replica_timeout.getDescriptor()); timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{i, 0}; @@ -143,6 +143,16 @@ void HedgedConnections::sendQuery( break; } + if (!disable_two_level_aggregation) + { + /// Tell hedged_connections_factory to skip replicas that doesn't support two-level aggregation. + hedged_connections_factory.setSkipPredicate( + [timeouts](Connection * connection) + { + return connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; + }); + } + auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) { Settings modified_settings = settings; @@ -157,7 +167,7 @@ void HedgedConnections::sendQuery( if (offset_states.size() > 1) { modified_settings.parallel_replicas_count = offset_states.size(); - modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver.getFileDescriptor()].offset; + modified_settings.parallel_replica_offset = fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset; } replica.connection->sendQuery(timeouts, query, query_id, stage, &modified_settings, &client_info, with_pending_data); @@ -183,11 +193,8 @@ void HedgedConnections::disconnect() if (hedged_connections_factory.hasEventsInProcess()) { - if (next_replica_in_process) - { + if (hedged_connections_factory.numberOfProcessingReplicas() > 0) epoll.remove(hedged_connections_factory.getFileDescriptor()); - next_replica_in_process = false; - } hedged_connections_factory.stopChoosingReplicas(); } @@ -291,33 +298,23 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As int event_fd; while (true) { - /// Check connections for pending data in buffer. - ReplicaLocation location; - if (checkPendingData(location)) - { - ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; - - replica_state.packet_receiver.resume(); - if (replica_state.packet_receiver.isPacketReady()) - return location; - continue; - } - /// Get ready file descriptor from epoll and process it. event_fd = getReadyFileDescriptor(async_callback); if (event_fd == hedged_connections_factory.getFileDescriptor()) - tryGetNewReplica(false); + checkNewReplica(); else if (fd_to_replica_location.contains(event_fd)) { - location = fd_to_replica_location[event_fd]; + ReplicaLocation location = fd_to_replica_location[event_fd]; ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; - replica_state.packet_receiver.resume(); + auto res = replica_state.packet_receiver->resume(); - if (replica_state.packet_receiver.isPacketReady()) + if (std::holds_alternative(res)) + { + last_received_packet = std::move(std::get(res)); return location; - - if (replica_state.packet_receiver.isReceiveTimeoutExpired()) + } + else if (std::holds_alternative(res)) { finishProcessReplica(replica_state, true); @@ -328,11 +325,11 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As } else if (timeout_fd_to_replica_location.contains(event_fd)) { - location = timeout_fd_to_replica_location[event_fd]; + ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; offset_states[location.offset].replicas[location.index].change_replica_timeout.reset(); offset_states[location.offset].next_replica_in_process = true; offsets_queue.push(location.offset); - tryGetNewReplica(true); + startNewReplica(); } else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); @@ -343,28 +340,20 @@ int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { epoll_event event; event.data.fd = -1; - epoll.getManyReady(1, &event, true, std::move(async_callback)); - return event.data.fd; -} - -bool HedgedConnections::checkPendingData(ReplicaLocation & location_out) -{ - for (auto & [fd, location] : fd_to_replica_location) + size_t events_count = 0; + while (events_count == 0) { - if (offset_states[location.offset].replicas[location.index].connection->hasReadPendingData()) - { - location_out = location; - return true; - } + events_count = epoll.getManyReady(1, &event, false); + if (!events_count && async_callback) + async_callback(epoll.getFileDescriptor(), 0, epoll.getDescription()); } - - return false; + return event.data.fd; } Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & replica_location) { ReplicaState & replica = offset_states[replica_location.offset].replicas[replica_location.index]; - Packet packet = replica.packet_receiver.getPacket(); + Packet packet = std::move(last_received_packet); switch (packet.type) { case Protocol::Server::Data: @@ -413,78 +402,84 @@ void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & r /// If we received data from replicas with all offsets, we need to stop choosing new replicas. if (hedged_connections_factory.hasEventsInProcess() && offsets_with_received_first_data_packet == offset_states.size()) { - if (next_replica_in_process) - { + if (hedged_connections_factory.numberOfProcessingReplicas() > 0) epoll.remove(hedged_connections_factory.getFileDescriptor()); - next_replica_in_process = false; - } hedged_connections_factory.stopChoosingReplicas(); } } -void HedgedConnections::tryGetNewReplica(bool start_new_connection) +void HedgedConnections::startNewReplica() { Connection * connection = nullptr; - HedgedConnectionsFactory::State state = hedged_connections_factory.getNextConnection(start_new_connection, false, connection); + HedgedConnectionsFactory::State state = hedged_connections_factory.startNewConnection(connection); - /// Skip replicas that doesn't support two-level aggregation if we didn't disable it in sendQuery. - while (state == HedgedConnectionsFactory::State::READY && !disable_two_level_aggregation - && connection->getServerRevision(hedged_connections_factory.getConnectionTimeouts()) - < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD) - state = hedged_connections_factory.getNextConnection(true, false, connection); - - if (state == HedgedConnectionsFactory::State::READY) - { - size_t offset = offsets_queue.front(); - offsets_queue.pop(); - - offset_states[offset].replicas.emplace_back(connection); - ++offset_states[offset].active_connection_count; - offset_states[offset].next_replica_in_process = false; - ++active_connection_count; - - ReplicaState & replica = offset_states[offset].replicas.back(); - epoll.add(replica.packet_receiver.getFileDescriptor()); - fd_to_replica_location[replica.packet_receiver.getFileDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1}; - epoll.add(replica.change_replica_timeout.getDescriptor()); - timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1}; - - pipeline_for_new_replicas.run(replica); - } - else if (state == HedgedConnectionsFactory::State::NOT_READY && !next_replica_in_process) - { + /// Check if we need to add hedged_connections_factory file descriptor to epoll. + if (state == HedgedConnectionsFactory::State::NOT_READY && hedged_connections_factory.numberOfProcessingReplicas() == 1) epoll.add(hedged_connections_factory.getFileDescriptor()); - next_replica_in_process = true; - } - /// Check if we cannot get new replica and there is no active replica with needed offsets. - else if (state == HedgedConnectionsFactory::State::CANNOT_CHOOSE) - { - while (!offsets_queue.empty()) - { - if (offset_states[offsets_queue.front()].active_connection_count == 0) - throw Exception("Cannot find enough connections to replicas", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); - offset_states[offsets_queue.front()].next_replica_in_process = false; - offsets_queue.pop(); - } - } + processNewReplicaState(state, connection); +} + +void HedgedConnections::checkNewReplica() +{ + Connection * connection = nullptr; + HedgedConnectionsFactory::State state = hedged_connections_factory.waitForReadyConnections(/*blocking = */false, connection); + + processNewReplicaState(state, connection); /// Check if we don't need to listen hedged_connections_factory file descriptor in epoll anymore. - if (next_replica_in_process && (state == HedgedConnectionsFactory::State::CANNOT_CHOOSE || offsets_queue.empty())) - { + if (hedged_connections_factory.numberOfProcessingReplicas() == 0) epoll.remove(hedged_connections_factory.getFileDescriptor()); - next_replica_in_process = false; +} + +void HedgedConnections::processNewReplicaState(HedgedConnectionsFactory::State state, Connection * connection) +{ + switch (state) + { + case HedgedConnectionsFactory::State::READY: + { + size_t offset = offsets_queue.front(); + offsets_queue.pop(); + + offset_states[offset].replicas.emplace_back(connection); + ++offset_states[offset].active_connection_count; + offset_states[offset].next_replica_in_process = false; + ++active_connection_count; + + ReplicaState & replica = offset_states[offset].replicas.back(); + epoll.add(replica.packet_receiver->getFileDescriptor()); + fd_to_replica_location[replica.packet_receiver->getFileDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1}; + epoll.add(replica.change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_location[replica.change_replica_timeout.getDescriptor()] = ReplicaLocation{offset, offset_states[offset].replicas.size() - 1}; + + pipeline_for_new_replicas.run(replica); + break; + } + case HedgedConnectionsFactory::State::CANNOT_CHOOSE: + { + while (!offsets_queue.empty()) + { + /// Check if there is no active replica with needed offsets. + if (offset_states[offsets_queue.front()].active_connection_count == 0) + throw Exception("Cannot find enough connections to replicas", ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + offset_states[offsets_queue.front()].next_replica_in_process = false; + offsets_queue.pop(); + } + break; + } + case HedgedConnectionsFactory::State::NOT_READY: + break; } } void HedgedConnections::finishProcessReplica(ReplicaState & replica, bool disconnect) { - replica.packet_receiver.cancel(); + replica.packet_receiver->cancel(); replica.change_replica_timeout.reset(); - epoll.remove(replica.packet_receiver.getFileDescriptor()); - --offset_states[fd_to_replica_location[replica.packet_receiver.getFileDescriptor()].offset].active_connection_count; - fd_to_replica_location.erase(replica.packet_receiver.getFileDescriptor()); + epoll.remove(replica.packet_receiver->getFileDescriptor()); + --offset_states[fd_to_replica_location[replica.packet_receiver->getFileDescriptor()].offset].active_connection_count; + fd_to_replica_location.erase(replica.packet_receiver->getFileDescriptor()); epoll.remove(replica.change_replica_timeout.getDescriptor()); timeout_fd_to_replica_location.erase(replica.change_replica_timeout.getDescriptor()); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index bfd5a36c500..555f601b901 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -21,14 +21,15 @@ namespace DB class HedgedConnections : public IConnections { public: + using PacketReceiverPtr = std::unique_ptr; struct ReplicaState { - ReplicaState(Connection * connection_) : connection(connection_), packet_receiver(connection_) + explicit ReplicaState(Connection * connection_) : connection(connection_), packet_receiver(std::make_unique(connection_)) { } Connection * connection = nullptr; - PacketReceiver packet_receiver; + PacketReceiverPtr packet_receiver; TimerDescriptor change_replica_timeout; }; @@ -119,14 +120,16 @@ private: void processReceivedFirstDataPacket(const ReplicaLocation & replica_location); - void tryGetNewReplica(bool start_new_connection); + void startNewReplica(); + + void checkNewReplica(); + + void processNewReplicaState(HedgedConnectionsFactory::State state, Connection * connection); void finishProcessReplica(ReplicaState & replica, bool disconnect); int getReadyFileDescriptor(AsyncCallback async_callback = {}); - bool checkPendingData(ReplicaLocation & location_out); - HedgedConnectionsFactory hedged_connections_factory; /// All replicas in offset_states[offset] is responsible for process query @@ -159,9 +162,7 @@ private: /// If we didn't disabled it, we need to skip this replica. bool disable_two_level_aggregation = false; - /// This flag means we need to get connection with new replica, but no replica is ready. - /// When it's true, hedged_connections_factory.getFileDescriptor() is in epoll. - bool next_replica_in_process = false; + Packet last_received_packet; Epoll epoll; const Settings & settings; diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 6519e0c9a94..681b1e3cd3d 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -22,15 +22,12 @@ HedgedConnectionsFactory::HedgedConnectionsFactory( { shuffled_pools = pool->getShuffledPools(settings); for (size_t i = 0; i != shuffled_pools.size(); ++i) - replicas.emplace_back(ConnectionEstablisher(shuffled_pools[i].pool, &timeouts, settings, table_to_check.get())); + replicas.emplace_back(ConnectionEstablisherAsync(shuffled_pools[i].pool, &timeouts, settings, log, table_to_check.get())); max_tries = (settings ? size_t{settings->connections_with_failover_max_tries} : size_t{DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES}); fallback_to_stale_replicas = settings && settings->fallback_to_stale_replicas_for_distributed_queries; - entries_count = 0; - usable_count = 0; - failed_pools_count = 0; } HedgedConnectionsFactory::~HedgedConnectionsFactory() @@ -62,18 +59,22 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode /// Try to start establishing connections with max_entries replicas. for (size_t i = 0; i != max_entries; ++i) { - int index = startEstablishingNewConnection(connection); - if (index == -1) - break; - if (replicas[index].is_ready) + ++requested_connections_count; + State state = startNewConnectionImpl(connection); + if (state == State::READY) connections.push_back(connection); + if (state == State::CANNOT_CHOOSE) + break; } /// Process connections until we get enough READY connections /// (work asynchronously with all connections we started). + /// TODO: when we get GET_ALL mode we can start reading packets from ready + /// TODO: connection as soon as we got it, not even waiting for the others. while (connections.size() < max_entries) { - auto state = getNextConnection(false, true, connection); + /// Set blocking = true to avoid busy-waiting here. + auto state = waitForReadyConnections(/*blocking = */true, connection); if (state == State::READY) connections.push_back(connection); else if (state == State::CANNOT_CHOOSE) @@ -82,29 +83,38 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode break; /// Determine the reason of not enough replicas. - if (!fallback_to_stale_replicas && usable_count >= min_entries) - throw DB::Exception( + if (!fallback_to_stale_replicas && up_to_date_count < min_entries) + throw Exception( "Could not find enough connections to up-to-date replicas. Got: " + std::to_string(connections.size()) + ", needed: " + std::to_string(min_entries), DB::ErrorCodes::ALL_REPLICAS_ARE_STALE); + if (usable_count < min_entries) + throw NetException( + "All connection tries failed. Log: \n\n" + fail_messages + "\n", + DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); - throw DB::NetException( - "All connection tries failed. Log: \n\n" + fail_messages + "\n", - DB::ErrorCodes::ALL_CONNECTION_TRIES_FAILED); + throw Exception("Unknown reason of not enough replicas.", ErrorCodes::LOGICAL_ERROR); } } return connections; } -HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out) +HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnection(Connection *& connection_out) { - if (start_new_connection) - { - int index = startEstablishingNewConnection(connection_out); - if (index != -1 && replicas[index].is_ready) - return State::READY; - } + LOG_DEBUG(log, "startNewConnection"); + ++requested_connections_count; + State state = startNewConnectionImpl(connection_out); + /// If we cannot start new connection but there are connections in epoll, return NOT_READY. + if (state == State::CANNOT_CHOOSE && !epoll.empty()) + state = State::NOT_READY; + + return state; +} + +HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnections(bool blocking, Connection *& connection_out) +{ + LOG_DEBUG(log, "waitForReadyConnections"); State state = processEpollEvents(blocking, connection_out); if (state != State::CANNOT_CHOOSE) @@ -120,24 +130,6 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::getNextConnection(bool return setBestUsableReplica(connection_out); } -void HedgedConnectionsFactory::stopChoosingReplicas() -{ - for (auto & [fd, index] : fd_to_replica_index) - { - epoll.remove(fd); - replicas[index].connection_establisher.cancel(); - } - - for (auto & [fd, index] : timeout_fd_to_replica_index) - { - replicas[index].change_replica_timeout.reset(); - epoll.remove(fd); - } - - fd_to_replica_index.clear(); - timeout_fd_to_replica_index.clear(); -} - int HedgedConnectionsFactory::getNextIndex() { /// Check if there is no free replica. @@ -158,8 +150,7 @@ int HedgedConnectionsFactory::getNextIndex() next_index = (next_index + 1) % shuffled_pools.size(); /// Check if we can try this replica. - if (!replicas[next_index].connection_establisher.isInProcess() - && !replicas[next_index].connection_establisher.isFinished() + if (replicas[next_index].connection_establisher.getResult().entry.isNull() && (max_tries == 0 || shuffled_pools[next_index].error_count < max_tries)) finish = true; @@ -172,100 +163,34 @@ int HedgedConnectionsFactory::getNextIndex() return next_index; } -int HedgedConnectionsFactory::startEstablishingNewConnection(Connection *& connection_out) +HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnectionImpl(Connection *& connection_out) { + LOG_DEBUG(log, "startNewConnectionImpl"); + int index; + State state; do { index = getNextIndex(); if (index == -1) - return -1; + return State::CANNOT_CHOOSE; - ReplicaStatus & replica = replicas[index]; - ++replicas_in_process_count; - replica.connection_establisher.resume(); - - processConnectionEstablisherStage(index); - - if (replica.connection_establisher.isInProcess()) - { - epoll.add(replica.connection_establisher.getFileDescriptor()); - fd_to_replica_index[replica.connection_establisher.getFileDescriptor()] = index; - - /// Add timeout for changing replica. - replica.change_replica_timeout.setRelative(timeouts.hedged_connection_timeout); - epoll.add(replica.change_replica_timeout.getDescriptor()); - timeout_fd_to_replica_index[replica.change_replica_timeout.getDescriptor()] = index; - } + state = resumeConnectionEstablisher(index, connection_out); } - while (!replicas[index].connection_establisher.isInProcess() && !replicas[index].is_ready); + while (state == State::CANNOT_CHOOSE); - if (replicas[index].is_ready) - connection_out = replicas[index].connection_establisher.getConnection(); - - return index; -} - -void HedgedConnectionsFactory::processConnectionEstablisherStage(int index, bool remove_from_epoll) -{ - ReplicaStatus & replica = replicas[index]; - - if (replica.connection_establisher.isFinished()) - { - --replicas_in_process_count; - ++entries_count; - - if (remove_from_epoll) - removeReplicaFromEpoll(index); - - if (replica.connection_establisher.getResult().is_usable) - { - ++usable_count; - if (replica.connection_establisher.getResult().is_up_to_date) - replica.is_ready = true; - - return; - } - - /// If replica is not usable, we need to save fail message. - if (!replica.connection_establisher.getFailMessage().empty()) - fail_messages += replica.connection_establisher.getFailMessage() + "\n"; - } - else if (replica.connection_establisher.isFailed()) - processFailedConnection(index, remove_from_epoll); -} - -void HedgedConnectionsFactory::processFailedConnection(int index, bool remove_from_epoll) -{ - ConnectionEstablisher & connection_establisher = replicas[index].connection_establisher; - - if (remove_from_epoll) - removeReplicaFromEpoll(index); - - if (!connection_establisher.getFailMessage().empty()) - fail_messages += connection_establisher.getFailMessage() + "\n"; - - ShuffledPool & shuffled_pool = shuffled_pools[index]; - LOG_WARNING( - log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), connection_establisher.getFailMessage()); - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); - - shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); - - if (shuffled_pool.error_count >= max_tries) - { - ++failed_pools_count; - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); - } - - --replicas_in_process_count; + return state; } HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out) { + LOG_DEBUG(log, "processEpollEvents"); + int event_fd; while (!epoll.empty()) { + LOG_DEBUG(log, "loop"); + event_fd = getReadyFileDescriptor(blocking); if (event_fd == -1) @@ -274,24 +199,29 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(boo if (fd_to_replica_index.contains(event_fd)) { int index = fd_to_replica_index[event_fd]; - processConnectionEstablisherEvent(index, connection_out); - - if (replicas[index].is_ready) - return State::READY; - if (replicas[index].connection_establisher.isInProcess()) + State state = resumeConnectionEstablisher(index, connection_out); + if (state == State::NOT_READY) continue; + + /// Connection establishing not in process now, remove all + /// information about it from epoll. + removeReplicaFromEpoll(index, event_fd); + + if (state == State::READY) + return state; } else if (timeout_fd_to_replica_index.contains(event_fd)) + { + LOG_DEBUG(log, "change_replica_timeout"); replicas[timeout_fd_to_replica_index[event_fd]].change_replica_timeout.reset(); + } else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); /// We reach this point only if we need to start new connection /// (Special timeout expired or one of the previous connections failed). - int index = startEstablishingNewConnection(connection_out); - /// Return only if replica is ready. - if (index != -1 && replicas[index].is_ready) + if (startNewConnectionImpl(connection_out) == State::READY) return State::READY; } @@ -306,35 +236,138 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) return event.data.fd; } -void HedgedConnectionsFactory::removeReplicaFromEpoll(int index) +HedgedConnectionsFactory::State HedgedConnectionsFactory::resumeConnectionEstablisher(int index, Connection *& connection_out) { - ReplicaStatus & replica = replicas[index]; - epoll.remove(replica.connection_establisher.getFileDescriptor()); - fd_to_replica_index.erase(replica.connection_establisher.getFileDescriptor()); + LOG_DEBUG(log, "resumeConnectionEstablisher"); - replica.change_replica_timeout.reset(); - epoll.remove(replica.change_replica_timeout.getDescriptor()); - timeout_fd_to_replica_index.erase(replica.change_replica_timeout.getDescriptor()); + auto res = replicas[index].connection_establisher.resume(); + + if (std::holds_alternative(res)) + return processFinishedConnection(index, std::get(res), connection_out); + + int fd = std::get(res); + if (!fd_to_replica_index.contains(fd)) + addNewReplicaToEpoll(index, fd); + + return State::NOT_READY; } -void HedgedConnectionsFactory::processConnectionEstablisherEvent(int index, Connection *& connection_out) +HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnection(int index, TryResult result, Connection *& connection_out) { - replicas[index].connection_establisher.resume(); - processConnectionEstablisherStage(index, true); - if (replicas[index].is_ready) - connection_out = replicas[index].connection_establisher.getConnection(); + LOG_DEBUG(log, "processFinishedConnection"); + + const std::string & fail_message = replicas[index].connection_establisher.getFailMessage(); + if (!fail_message.empty()) + fail_messages += fail_message + "\n"; + + if (!result.entry.isNull()) + { + ++entries_count; + + if (result.is_usable) + { + ++usable_count; + if (result.is_up_to_date) + { + ++up_to_date_count; + if (!skip_predicate || !skip_predicate(&*result.entry)) + { + replicas[index].is_ready = true; + ++ready_replicas_count; + connection_out = &*result.entry; + return State::READY; + } + } + } + } + else + { + ShuffledPool & shuffled_pool = shuffled_pools[index]; + LOG_WARNING( + log, "Connection failed at try №{}, reason: {}", (shuffled_pool.error_count + 1), fail_message); + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); + + shuffled_pool.error_count = std::min(pool->getMaxErrorCup(), shuffled_pool.error_count + 1); + + if (shuffled_pool.error_count >= max_tries) + { + ++failed_pools_count; + ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); + } + } + + return State::CANNOT_CHOOSE; +} + +void HedgedConnectionsFactory::stopChoosingReplicas() +{ + LOG_DEBUG(log, "stopChoosingReplicas"); + + for (auto & [fd, index] : fd_to_replica_index) + { + --replicas_in_process_count; + epoll.remove(fd); + replicas[index].connection_establisher.cancel(); + } + + for (auto & [timeout_fd, index] : timeout_fd_to_replica_index) + { + replicas[index].change_replica_timeout.reset(); + epoll.remove(timeout_fd); + } + + fd_to_replica_index.clear(); + timeout_fd_to_replica_index.clear(); +} + +void HedgedConnectionsFactory::addNewReplicaToEpoll(int index, int fd) +{ + LOG_DEBUG(log, "addNewReplicaToEpoll"); + + ++replicas_in_process_count; + epoll.add(fd); + fd_to_replica_index[fd] = index; + + /// Add timeout for changing replica. + replicas[index].change_replica_timeout.setRelative(timeouts.hedged_connection_timeout); + epoll.add(replicas[index].change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_index[replicas[index].change_replica_timeout.getDescriptor()] = index; +} + +void HedgedConnectionsFactory::removeReplicaFromEpoll(int index, int fd) +{ + LOG_DEBUG(log, "removeReplicaFromEpoll"); + + --replicas_in_process_count; + epoll.remove(fd); + fd_to_replica_index.erase(fd); + + replicas[index].change_replica_timeout.reset(); + epoll.remove(replicas[index].change_replica_timeout.getDescriptor()); + timeout_fd_to_replica_index.erase(replicas[index].change_replica_timeout.getDescriptor()); +} + +int HedgedConnectionsFactory::numberOfProcessingReplicas() const +{ + if (epoll.empty()) + return 0; + + return requested_connections_count - ready_replicas_count; } HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(Connection *& connection_out) { + LOG_DEBUG(log, "setBestUsableReplica"); + std::vector indexes; for (size_t i = 0; i != replicas.size(); ++i) { /// Don't add unusable, failed replicas and replicas that are ready or in process. - if (!replicas[i].connection_establisher.getResult().entry.isNull() - && replicas[i].connection_establisher.getResult().is_usable - && !replicas[i].connection_establisher.isInProcess() - && !replicas[i].is_ready) + TryResult result = replicas[i].connection_establisher.getResult(); + if (!result.entry.isNull() + && result.is_usable + && !replicas[i].is_ready + && (!skip_predicate || !skip_predicate(&*result.entry))) indexes.push_back(i); } @@ -350,9 +383,9 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C return replicas[lhs].connection_establisher.getResult().staleness < replicas[rhs].connection_establisher.getResult().staleness; }); - ++ready_replicas_count; replicas[indexes[0]].is_ready = true; - connection_out = replicas[indexes[0]].connection_establisher.getConnection(); + TryResult result = replicas[indexes[0]].connection_establisher.getResult(); + connection_out = &*result.entry; return State::READY; } diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index 896774daae3..f49b368bc4b 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -25,6 +25,7 @@ class HedgedConnectionsFactory { public: using ShuffledPool = ConnectionPoolWithFailover::Base::ShuffledPool; + using TryResult = PoolWithFailoverBase::TryResult; enum class State { @@ -35,11 +36,11 @@ public: struct ReplicaStatus { - ReplicaStatus(ConnectionEstablisher connection_stablisher_) : connection_establisher(std::move(connection_stablisher_)) + explicit ReplicaStatus(ConnectionEstablisherAsync connection_stablisher_) : connection_establisher(std::move(connection_stablisher_)) { } - ConnectionEstablisher connection_establisher; + ConnectionEstablisherAsync connection_establisher; TimerDescriptor change_replica_timeout; bool is_ready = false; }; @@ -57,10 +58,9 @@ public: /// if there is no events in epoll and blocking is false, return NOT_READY. /// Returned state might be READY, NOT_READY and CANNOT_CHOOSE. /// If state is READY, replica connection will be written in connection_out. - State getNextConnection(bool start_new_connection, bool blocking, Connection *& connection_out); + State waitForReadyConnections(bool blocking, Connection *& connection_out); - /// Check if we can try to produce new READY replica. -// bool canGetNewConnection() const { return ready_replicas_count + failed_pools_count < shuffled_pools.size(); } + State startNewConnection(Connection *& connection_out); /// Stop working with all replicas that are not READY. void stopChoosingReplicas(); @@ -71,14 +71,16 @@ public: const ConnectionTimeouts & getConnectionTimeouts() const { return timeouts; } + int numberOfProcessingReplicas() const; + + void setSkipPredicate(std::function pred) { skip_predicate = std::move(pred); } + ~HedgedConnectionsFactory(); private: /// Try to start establishing connection to the new replica. Return /// the index of the new replica or -1 if cannot start new connection. - int startEstablishingNewConnection(Connection *& connection_out); - - void processConnectionEstablisherStage(int replica_index, bool remove_from_epoll = false); + State startNewConnectionImpl(Connection *& connection_out); /// Find an index of the next free replica to start connection. /// Return -1 if there is no free replica. @@ -86,11 +88,15 @@ private: int getReadyFileDescriptor(bool blocking); - void processFailedConnection(int replica_index, bool remove_from_epoll); + void processFailedConnection(int index, const std::string & fail_message); - void processConnectionEstablisherEvent(int replica_index, Connection *& connection_out); + State resumeConnectionEstablisher(int index, Connection *& connection_out); - void removeReplicaFromEpoll(int index); + State processFinishedConnection(int index, TryResult result, Connection *& connection_out); + + void removeReplicaFromEpoll(int index, int fd); + + void addNewReplicaToEpoll(int index, int fd); /// Return NOT_READY state if there is no ready events, READY if replica is ready /// and CANNOT_CHOOSE if there is no more events in epoll. @@ -111,10 +117,7 @@ private: /// Map timeout for changing replica to replica index. std::unordered_map timeout_fd_to_replica_index; - /// Indexes of replicas, that are in process of connection. - size_t replicas_in_process_count = 0; - /// Indexes of ready replicas. - size_t ready_replicas_count = 0; + std::function skip_predicate; std::shared_ptr table_to_check; int last_used_index = -1; @@ -122,10 +125,14 @@ private: Epoll epoll; Poco::Logger * log; std::string fail_messages; - size_t entries_count; - size_t usable_count; - size_t failed_pools_count; size_t max_tries; + size_t entries_count = 0; + size_t usable_count = 0; + size_t up_to_date_count = 0; + size_t failed_pools_count= 0; + size_t replicas_in_process_count = 0; + size_t requested_connections_count = 0; + size_t ready_replicas_count = 0; }; } diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index 79d805be877..665488ca30f 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -2,41 +2,86 @@ #if defined(OS_LINUX) +#include + #include #include #include +#include +#include namespace DB { +namespace ErrorCodes +{ + extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_READ_FROM_SOCKET; +} + /// Class for nonblocking packet receiving. It runs connection->receivePacket /// in fiber and sets special read callback which is called when /// reading from socket blocks. When read callback is called, /// socket and receive timeout are added in epoll and execution returns to the main program. /// So, you can poll this epoll file descriptor to determine when to resume -/// packet receiving (beside polling epoll descriptor, you also need to check connection->hasPendingData(), -/// because small packet can be read in buffer with the previous one, so new packet will be ready in buffer, -/// but there is no data socket to poll). +/// packet receiving. class PacketReceiver { public: - PacketReceiver(Connection * connection_) : connection(connection_) + explicit PacketReceiver(Connection * connection_) : connection(connection_) { epoll.add(receive_timeout.getDescriptor()); epoll.add(connection->getSocket()->impl()->sockfd()); + + if (-1 == pipe2(pipe_fd, O_NONBLOCK)) + throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE); + epoll.add(pipe_fd[0]); + fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); } + ~PacketReceiver() + { + close(pipe_fd[0]); + close(pipe_fd[1]); + } + /// Resume packet receiving. - void resume() + std::variant resume() { /// If there is no pending data, check receive timeout. if (!connection->hasReadPendingData() && !checkReceiveTimeout()) - return; + { + /// Receive timeout expired. + return Poco::Timespan(); + } + /// Resume fiber. fiber = std::move(fiber).resume(); if (exception) std::rethrow_exception(std::move(exception)); + + if (is_read_in_process) + return epoll.getFileDescriptor(); + + /// Write something in pipe when buffer has pending data, because + /// in this case socket won't be ready in epoll but we need to tell + /// outside that there is more data in buffer. + if (connection->hasReadPendingData()) + { + uint64_t buf = 0; + while (-1 == write(pipe_fd[1], &buf, sizeof(buf))) + { + if (errno == EAGAIN) + break; + + if (errno != EINTR) + throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET); + } + } + + /// Receiving packet was finished. + return std::move(packet); } void cancel() @@ -45,20 +90,16 @@ public: connection = nullptr; } - Packet getPacket() { return std::move(packet); } - int getFileDescriptor() const { return epoll.getFileDescriptor(); } - bool isPacketReady() const { return !is_read_in_process; } - - bool isReceiveTimeoutExpired() const { return is_receive_timeout_expired; } - private: - /// When epoll file descriptor is ready, check if it's an expired timeout + /// When epoll file descriptor is ready, check if it's an expired timeout. + /// Return false if receive timeout expired and socket is not ready, return true otherwise. bool checkReceiveTimeout() { bool is_socket_ready = false; - is_receive_timeout_expired = false; + bool is_pipe_ready = false; + bool is_receive_timeout_expired = false; epoll_event events[2]; events[0].data.fd = events[1].data.fd = -1; @@ -68,10 +109,19 @@ private: { if (events[i].data.fd == connection->getSocket()->impl()->sockfd()) is_socket_ready = true; + if (events[i].data.fd == pipe_fd[0]) + is_pipe_ready = true; if (events[i].data.fd == receive_timeout.getDescriptor()) is_receive_timeout_expired = true; } + if (is_pipe_ready) + { + LOG_DEBUG(&Poco::Logger::get("PacketReceiver"), "pipe"); + drainPipe(); + return true; + } + if (is_receive_timeout_expired && !is_socket_ready) { receive_timeout.reset(); @@ -81,6 +131,23 @@ private: return true; } + void drainPipe() + { + uint64_t buf; + while (true) + { + ssize_t res = read(pipe_fd[0], &buf, sizeof(buf)); + if (res < 0) + { + if (errno == EAGAIN) + break; + + if (errno != EINTR) + throwFromErrno("Cannot drain pipe_fd", ErrorCodes::CANNOT_READ_FROM_SOCKET); + } + } + } + struct Routine { PacketReceiver & receiver; @@ -131,14 +198,28 @@ private: }; Connection * connection; - TimerDescriptor receive_timeout; - Epoll epoll; + Packet packet; + Fiber fiber; FiberStack fiber_stack; - Packet packet; - bool is_read_in_process = false; - bool is_receive_timeout_expired = false; + + /// We use timer descriptor for checking socket receive timeout. + TimerDescriptor receive_timeout; + + /// In read callback we add socket file descriptor and timer descriptor with receive timeout + /// in epoll, so we can return epoll file descriptor outside for polling. + Epoll epoll; + + /// Pipe is used when there is pending data in buffer + /// after receiving packet socket won't be ready in epoll in this case), + /// so we add pipe_fd in epoll and write something in it to tell + /// outside that we are ready to receive new packet. + int pipe_fd[2]; + + /// If and exception occurred in fiber resume, we save it and rethrow. std::exception_ptr exception; + + bool is_read_in_process = false; }; } diff --git a/src/Common/Epoll.cpp b/src/Common/Epoll.cpp index 89c6c8abc49..a17cce8545c 100644 --- a/src/Common/Epoll.cpp +++ b/src/Common/Epoll.cpp @@ -57,22 +57,19 @@ void Epoll::remove(int fd) throwFromErrno("Cannot remove descriptor from epoll", DB::ErrorCodes::EPOLL_ERROR); } -size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback) const +size_t Epoll::getManyReady(int max_events, epoll_event * events_out, bool blocking) const { if (events_count == 0) throw Exception("There is no events in epoll", ErrorCodes::LOGICAL_ERROR); int ready_size; - int timeout = blocking && !async_callback ? -1 : 0; + int timeout = blocking ? -1 : 0; do { ready_size = epoll_wait(epoll_fd, events_out, max_events, timeout); if (ready_size == -1 && errno != EINTR) throwFromErrno("Error in epoll_wait", DB::ErrorCodes::EPOLL_ERROR); - - if (ready_size == 0 && blocking && async_callback) - async_callback(epoll_fd, 0, "epoll"); } while (ready_size <= 0 && (ready_size != 0 || blocking)); diff --git a/src/Common/Epoll.h b/src/Common/Epoll.h index a7090bdb9b6..5d9aef9ef66 100644 --- a/src/Common/Epoll.h +++ b/src/Common/Epoll.h @@ -31,10 +31,8 @@ public: /// Get events from epoll. Events are written in events_out, this function returns an amount of ready events. /// If blocking is false and there are no ready events, - /// return empty vector, otherwise wait for ready events. If blocking is true, - /// async_callback is given and there is no ready events, async_callback is called - /// with epoll file descriptor. - size_t getManyReady(int max_events, epoll_event * events_out, bool blocking, AsyncCallback async_callback = {}) const; + /// return empty vector, otherwise wait for ready events. + size_t getManyReady(int max_events, epoll_event * events_out, bool blocking) const; int getFileDescriptor() const { return epoll_fd; } @@ -42,11 +40,14 @@ public: bool empty() const { return events_count == 0; } + const std::string & getDescription() const { return fd_description; } + ~Epoll(); private: int epoll_fd; std::atomic events_count; + const std::string fd_description = "epoll"; }; } diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 1f9c732e644..62020d32d8f 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -4,6 +4,7 @@ #include #include #include +#include namespace ProfileEvents @@ -34,11 +35,16 @@ bool ReadBufferFromPocoSocket::nextImpl() /// Add more details to exceptions. try { + if (!async_callback) + LOG_DEBUG(&Poco::Logger::get("ReadBufferFromPocoSocket"), "Don't have async callback"); + bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), internal_buffer.size(), flags); /// If async_callback is specified, and read is blocking, run async_callback and try again later. /// It is expected that file descriptor may be polled externally. /// Note that receive timeout is not checked here. External code should check it while polling. + LOG_DEBUG(&Poco::Logger::get("ReadBufferFromPocoSocket"), "Don't have async callback"); + while (bytes_read < 0 && async_callback && errno == EAGAIN) { async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), socket_description); diff --git a/src/Interpreters/tests/CMakeLists.txt b/src/Interpreters/tests/CMakeLists.txt index 1bc9d7fbacb..88633192945 100644 --- a/src/Interpreters/tests/CMakeLists.txt +++ b/src/Interpreters/tests/CMakeLists.txt @@ -40,6 +40,9 @@ add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor. target_link_libraries (in_join_subqueries_preprocessor PRIVATE clickhouse_aggregate_functions dbms clickhouse_parsers) add_check(in_join_subqueries_preprocessor) +add_executable (context context.cpp) +target_link_libraries (context PRIVATE dbms) + if (OS_LINUX) add_executable (internal_iotop internal_iotop.cpp) target_link_libraries (internal_iotop PRIVATE dbms) diff --git a/tests/integration/test_distributed_respect_user_timeouts/test.py b/tests/integration/test_distributed_respect_user_timeouts/test.py index c19323b2049..662bf7fa6de 100644 --- a/tests/integration/test_distributed_respect_user_timeouts/test.py +++ b/tests/integration/test_distributed_respect_user_timeouts/test.py @@ -78,6 +78,7 @@ def _check_exception(exception, expected_tries=3): expected_lines = ( 'Code: 209, ' + EXCEPTION_NETWORK + EXCEPTION_TIMEOUT, 'Code: 209, ' + EXCEPTION_NETWORK + EXCEPTION_CONNECT, + EXCEPTION_TIMEOUT, ) assert any(line.startswith(expected) for expected in expected_lines), \ From 7c76b4a98a416fa4173e4f51576cbfad708acf15 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:12:26 +0300 Subject: [PATCH 298/510] Remove debug logging --- src/IO/ReadBufferFromPocoSocket.cpp | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 4e3e81deb9f..abeec09201f 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -35,16 +35,11 @@ bool ReadBufferFromPocoSocket::nextImpl() /// Add more details to exceptions. try { - if (!async_callback) - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromPocoSocket"), "Don't have async callback"); - bytes_read = socket.impl()->receiveBytes(internal_buffer.begin(), internal_buffer.size(), flags); /// If async_callback is specified, and read is blocking, run async_callback and try again later. /// It is expected that file descriptor may be polled externally. /// Note that receive timeout is not checked here. External code should check it while polling. - LOG_DEBUG(&Poco::Logger::get("ReadBufferFromPocoSocket"), "Don't have async callback"); - while (bytes_read < 0 && async_callback && errno == EAGAIN) { async_callback(socket.impl()->sockfd(), socket.getReceiveTimeout(), socket_description); @@ -86,7 +81,7 @@ ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, { } -bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) const +bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) { return available() || socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); } From abb5993bd522693263d65580394be21b36ffa104 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:13:33 +0300 Subject: [PATCH 299/510] return const --- src/IO/ReadBufferFromPocoSocket.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index abeec09201f..3a9486678c3 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -81,7 +81,7 @@ ReadBufferFromPocoSocket::ReadBufferFromPocoSocket(Poco::Net::Socket & socket_, { } -bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) +bool ReadBufferFromPocoSocket::poll(size_t timeout_microseconds) const { return available() || socket.poll(timeout_microseconds, Poco::Net::Socket::SELECT_READ | Poco::Net::Socket::SELECT_ERROR); } From 7376e34788e95f3ccb5f71315ee8502cb042b2e4 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:15:17 +0300 Subject: [PATCH 300/510] remove test from cmake --- src/Interpreters/tests/CMakeLists.txt | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/tests/CMakeLists.txt b/src/Interpreters/tests/CMakeLists.txt index 88633192945..1bc9d7fbacb 100644 --- a/src/Interpreters/tests/CMakeLists.txt +++ b/src/Interpreters/tests/CMakeLists.txt @@ -40,9 +40,6 @@ add_executable (in_join_subqueries_preprocessor in_join_subqueries_preprocessor. target_link_libraries (in_join_subqueries_preprocessor PRIVATE clickhouse_aggregate_functions dbms clickhouse_parsers) add_check(in_join_subqueries_preprocessor) -add_executable (context context.cpp) -target_link_libraries (context PRIVATE dbms) - if (OS_LINUX) add_executable (internal_iotop internal_iotop.cpp) target_link_libraries (internal_iotop PRIVATE dbms) From 10f54f1714d38faeae6b29b1fe45d4533f97a6c1 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:18:50 +0300 Subject: [PATCH 301/510] remove include --- src/IO/ReadBufferFromPocoSocket.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/IO/ReadBufferFromPocoSocket.cpp b/src/IO/ReadBufferFromPocoSocket.cpp index 3a9486678c3..37896a387bb 100644 --- a/src/IO/ReadBufferFromPocoSocket.cpp +++ b/src/IO/ReadBufferFromPocoSocket.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace ProfileEvents From 93162716f459e0861645c31c6160d3b8525b3584 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:50:47 +0300 Subject: [PATCH 302/510] remove logging --- src/Client/HedgedConnectionsFactory.cpp | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 681b1e3cd3d..bed7c75527f 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -102,7 +102,6 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnection(Connection *& connection_out) { - LOG_DEBUG(log, "startNewConnection"); ++requested_connections_count; State state = startNewConnectionImpl(connection_out); /// If we cannot start new connection but there are connections in epoll, return NOT_READY. @@ -114,8 +113,6 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnection(Con HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnections(bool blocking, Connection *& connection_out) { - LOG_DEBUG(log, "waitForReadyConnections"); - State state = processEpollEvents(blocking, connection_out); if (state != State::CANNOT_CHOOSE) return state; @@ -165,8 +162,6 @@ int HedgedConnectionsFactory::getNextIndex() HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnectionImpl(Connection *& connection_out) { - LOG_DEBUG(log, "startNewConnectionImpl"); - int index; State state; do @@ -184,13 +179,9 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnectionImpl HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(bool blocking, Connection *& connection_out) { - LOG_DEBUG(log, "processEpollEvents"); - int event_fd; while (!epoll.empty()) { - LOG_DEBUG(log, "loop"); - event_fd = getReadyFileDescriptor(blocking); if (event_fd == -1) @@ -211,10 +202,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processEpollEvents(boo return state; } else if (timeout_fd_to_replica_index.contains(event_fd)) - { - LOG_DEBUG(log, "change_replica_timeout"); replicas[timeout_fd_to_replica_index[event_fd]].change_replica_timeout.reset(); - } else throw Exception("Unknown event from epoll", ErrorCodes::LOGICAL_ERROR); @@ -238,8 +226,6 @@ int HedgedConnectionsFactory::getReadyFileDescriptor(bool blocking) HedgedConnectionsFactory::State HedgedConnectionsFactory::resumeConnectionEstablisher(int index, Connection *& connection_out) { - LOG_DEBUG(log, "resumeConnectionEstablisher"); - auto res = replicas[index].connection_establisher.resume(); if (std::holds_alternative(res)) @@ -254,8 +240,6 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::resumeConnectionEstabl HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnection(int index, TryResult result, Connection *& connection_out) { - LOG_DEBUG(log, "processFinishedConnection"); - const std::string & fail_message = replicas[index].connection_establisher.getFailMessage(); if (!fail_message.empty()) fail_messages += fail_message + "\n"; @@ -301,8 +285,6 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnect void HedgedConnectionsFactory::stopChoosingReplicas() { - LOG_DEBUG(log, "stopChoosingReplicas"); - for (auto & [fd, index] : fd_to_replica_index) { --replicas_in_process_count; @@ -322,8 +304,6 @@ void HedgedConnectionsFactory::stopChoosingReplicas() void HedgedConnectionsFactory::addNewReplicaToEpoll(int index, int fd) { - LOG_DEBUG(log, "addNewReplicaToEpoll"); - ++replicas_in_process_count; epoll.add(fd); fd_to_replica_index[fd] = index; @@ -336,8 +316,6 @@ void HedgedConnectionsFactory::addNewReplicaToEpoll(int index, int fd) void HedgedConnectionsFactory::removeReplicaFromEpoll(int index, int fd) { - LOG_DEBUG(log, "removeReplicaFromEpoll"); - --replicas_in_process_count; epoll.remove(fd); fd_to_replica_index.erase(fd); @@ -357,8 +335,6 @@ int HedgedConnectionsFactory::numberOfProcessingReplicas() const HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(Connection *& connection_out) { - LOG_DEBUG(log, "setBestUsableReplica"); - std::vector indexes; for (size_t i = 0; i != replicas.size(); ++i) { From db591309a3b3e32f84c9812c932d188b62301a47 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 17:51:36 +0300 Subject: [PATCH 303/510] fix style --- src/Client/HedgedConnectionsFactory.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index bed7c75527f..22d52ba157f 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -8,7 +8,6 @@ namespace DB { namespace ErrorCodes { - extern const int LOGICAL_ERROR; extern const int ALL_CONNECTION_TRIES_FAILED; extern const int ALL_REPLICAS_ARE_STALE; } From e08be0a9a142395d2c9725c367f0779ff030f8e1 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 20:38:54 +0300 Subject: [PATCH 304/510] Commit contrib change --- contrib/boost | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/boost b/contrib/boost index 48f40ebb539..ee24fa55bc4 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 48f40ebb539220d328958f8823b094c0b07a4e79 +Subproject commit ee24fa55bc46e4d2ce7d0d052cc5a0d9b1be8c36 From f8a581dabaa672df71a498f7f2e68fa02eb49ff2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Feb 2021 20:42:19 +0300 Subject: [PATCH 305/510] Remove possibly redundant allocations --- src/Coordination/Changelog.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 6ffd6a6f04b..45d80eb2757 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -447,7 +447,7 @@ LogEntryPtr Changelog::getLastEntry() const if (entry == logs.end()) return fake_entry; - return makeClone(entry->second); + return entry->second; } LogEntriesPtr Changelog::getLogEntriesBetween(size_t start, size_t end) @@ -472,7 +472,7 @@ LogEntryPtr Changelog::entryAt(size_t index) return nullptr; src = entry->second; - return makeClone(src); + return src; } nuraft::ptr Changelog::serializeEntriesToBuffer(size_t index, int32_t count) From 4320782032386627c2a27b7df55a060480d1cf5c Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sun, 21 Feb 2021 20:43:36 +0300 Subject: [PATCH 306/510] Style --- src/Client/ConnectionEstablisher.cpp | 1 - src/Client/HedgedConnectionsFactory.cpp | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index fb8e977b569..d3674bd5e61 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -16,7 +16,6 @@ namespace ErrorCodes extern const int ATTEMPT_TO_READ_AFTER_EOF; extern const int NETWORK_ERROR; extern const int SOCKET_TIMEOUT; - extern const int LOGICAL_ERROR; } ConnectionEstablisher::ConnectionEstablisher( diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 22d52ba157f..78031e86c0b 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -10,6 +10,7 @@ namespace ErrorCodes { extern const int ALL_CONNECTION_TRIES_FAILED; extern const int ALL_REPLICAS_ARE_STALE; + extern const int LOGICAL_ERROR; } HedgedConnectionsFactory::HedgedConnectionsFactory( From 1aa0c0bff09bf5a7eecb701acad0554f25d405bc Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 21 Feb 2021 23:36:05 +0300 Subject: [PATCH 307/510] More stable test --- src/Coordination/tests/gtest_for_build.cpp | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 350d60f64c2..163393da573 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -206,11 +206,14 @@ TEST(CoordinationTest, TestSummingRaft3) nuraft::srv_config first_config(1, 0, "localhost:44444", "", false, 0); auto ret1 = s2.raft_instance->add_srv(first_config); - if (!ret1->get_accepted()) + while (!ret1->get_accepted()) { + std::cout << "failed to add server: " << ret1->get_result_str() << std::endl; - EXPECT_TRUE(false); + + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + ret1 = s2.raft_instance->add_srv(first_config); } while (s1.raft_instance->get_leader() != 2) @@ -225,7 +228,9 @@ TEST(CoordinationTest, TestSummingRaft3) { std::cout << "failed to add server: " << ret3->get_result_str() << std::endl; - EXPECT_TRUE(false); + + std::this_thread::sleep_for(std::chrono::milliseconds(100)); + ret3 = s2.raft_instance->add_srv(third_config); } while (s3.raft_instance->get_leader() != 2) From 6d8eb44c0126ec33d7b74c31f8e790a21b597026 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 22 Feb 2021 00:59:07 +0300 Subject: [PATCH 308/510] Fix PSV and remove debug logging --- src/Client/ConnectionEstablisher.cpp | 3 ++- src/Client/PacketReceiver.h | 1 - 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index d3674bd5e61..c031188ef3e 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -23,7 +23,8 @@ ConnectionEstablisher::ConnectionEstablisher( const ConnectionTimeouts * timeouts_, const Settings * settings_, Poco::Logger * log_, - const QualifiedTableName * table_to_check_) : pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_) + const QualifiedTableName * table_to_check_) + : pool(pool_), timeouts(timeouts_), settings(settings_), log(log_), table_to_check(table_to_check_), is_finished(false) { } diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index 665488ca30f..e82591f9444 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -117,7 +117,6 @@ private: if (is_pipe_ready) { - LOG_DEBUG(&Poco::Logger::get("PacketReceiver"), "pipe"); drainPipe(); return true; } From 74a07e406b7199dc5aa7804f5e5c63f6477118de Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Feb 2021 10:33:27 +0300 Subject: [PATCH 309/510] Even more stable --- .../test.py | 40 +++++++++++++++---- 1 file changed, 32 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py index 49d86ab9fe8..0f17fff8f76 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py @@ -14,6 +14,18 @@ node3 = cluster.add_instance('node3', main_configs=['configs/enable_test_keeper3 from kazoo.client import KazooClient, KazooState +""" +In this test, we blockade RAFT leader and check that the whole system is +able to recover. It's not a good test because we use ClickHouse's replicated +tables to check connectivity, but they may require special operations (or a long +wait) after session expiration. We don't use kazoo, because this client pretends +to be very smart: SUSPEND sessions, try to recover them, and so on. The test +will be even less predictable than with ClickHouse tables. + +TODO find (or write) not so smart python client. +TODO remove this when jepsen tests will be written. +""" + @pytest.fixture(scope="module") def started_cluster(): try: @@ -66,10 +78,16 @@ def get_fake_zk(nodename, timeout=30.0): # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader(started_cluster): - wait_nodes() - for i, node in enumerate([node1, node2, node3]): - node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") - node.query("CREATE TABLE ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1)) + for i in range(100): + wait_nodes() + try: + for i, node in enumerate([node1, node2, node3]): + node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") + node.query("CREATE TABLE IF NOT EXISTS ordinary.t1 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t1', '{}') ORDER BY tuple()".format(i + 1)) + break + except Exception as ex: + print("Got exception from node", smaller_exception(ex)) + time.sleep(0.1) node2.query("INSERT INTO ordinary.t1 SELECT number FROM numbers(10)") @@ -209,10 +227,16 @@ def restart_replica_for_sure(node, table_name, zk_replica_path): # in extremely rare case it can take more than 5 minutes in debug build with sanitizer @pytest.mark.timeout(600) def test_blocade_leader_twice(started_cluster): - wait_nodes() - for i, node in enumerate([node1, node2, node3]): - node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") - node.query("CREATE TABLE ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1)) + for i in range(100): + wait_nodes() + try: + for i, node in enumerate([node1, node2, node3]): + node.query("CREATE DATABASE IF NOT EXISTS ordinary ENGINE=Ordinary") + node.query("CREATE TABLE IF NOT EXISTS ordinary.t2 (value UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/t2', '{}') ORDER BY tuple()".format(i + 1)) + break + except Exception as ex: + print("Got exception from node", smaller_exception(ex)) + time.sleep(0.1) node2.query("INSERT INTO ordinary.t2 SELECT number FROM numbers(10)") From 1dbab602354201c5844db8d339c863679c21d75e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 22 Feb 2021 10:52:19 +0300 Subject: [PATCH 310/510] Fix --- src/Client/ConnectionEstablisher.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index c031188ef3e..17b8832aab4 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -103,6 +103,8 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: } } +#if defined(OS_LINUX) + ConnectionEstablisherAsync::ConnectionEstablisherAsync( IConnectionPool * pool_, const ConnectionTimeouts * timeouts_, @@ -234,4 +236,6 @@ void ConnectionEstablisherAsync::destroyFiber() fiber_created = false; } +#endif + } From 6433ab3da69e9f3ca88f50298387006db2f2155a Mon Sep 17 00:00:00 2001 From: jennyma Date: Mon, 22 Feb 2021 18:34:01 +0800 Subject: [PATCH 311/510] fix default_replica_path and default_replica_name values are useless on Replicated(*)MergeTree engine when the engine needs specify other parameters --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 10ebfa5ce1d..ba70b7aa65c 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -427,7 +427,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (is_extended_storage_def && arg_cnt == 0) + else if (is_extended_storage_def && (arg_cnt == 0 || !engine_args[arg_num]->as())) { /// Try use default values if arguments are not specified. /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. From f8501a2595d3ac00f474b50be10fbc59cbc010c0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Feb 2021 17:11:06 +0300 Subject: [PATCH 312/510] Add a test for #9781 --- tests/queries/0_stateless/01735_to_datetime64.reference | 1 + tests/queries/0_stateless/01735_to_datetime64.sql | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/queries/0_stateless/01735_to_datetime64.reference create mode 100644 tests/queries/0_stateless/01735_to_datetime64.sql diff --git a/tests/queries/0_stateless/01735_to_datetime64.reference b/tests/queries/0_stateless/01735_to_datetime64.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01735_to_datetime64.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01735_to_datetime64.sql b/tests/queries/0_stateless/01735_to_datetime64.sql new file mode 100644 index 00000000000..fe4eb521148 --- /dev/null +++ b/tests/queries/0_stateless/01735_to_datetime64.sql @@ -0,0 +1 @@ +SELECT toDate(toDateTime64(today(), 0, 'UTC')) = toDate(toDateTime(today(), 'UTC')); From 871f15343350fd1ba4e065c6f47672b3944b3319 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 22 Feb 2021 20:44:24 +0300 Subject: [PATCH 313/510] Fix tests --- .../01691_DateTime64_clamp.reference | 26 +++++++++---------- .../0_stateless/01691_DateTime64_clamp.sql | 18 ++++++------- 2 files changed, 21 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/01691_DateTime64_clamp.reference b/tests/queries/0_stateless/01691_DateTime64_clamp.reference index 849f8139640..3adc9a17e5c 100644 --- a/tests/queries/0_stateless/01691_DateTime64_clamp.reference +++ b/tests/queries/0_stateless/01691_DateTime64_clamp.reference @@ -1,19 +1,17 @@ -- { echo } -SELECT toDateTime(-2, 2); +SELECT toTimeZone(toDateTime(-2, 2), 'Europe/Moscow'); 1970-01-01 03:00:00.00 -SELECT toDateTime64(-2, 2); +SELECT toDateTime64(-2, 2, 'Europe/Moscow'); 1970-01-01 03:00:00.00 -SELECT CAST(-1 AS DateTime64); -1970-01-01 03:00:00.000 -SELECT CAST('2020-01-01 00:00:00.3' AS DateTime64); -2020-01-01 00:00:00.300 -SELECT toDateTime64(bitShiftLeft(toUInt64(1),33), 2); -2106-02-07 09:28:15.00 -SELECT toDateTime(-2., 2); +SELECT CAST(-1 AS DateTime64(0, 'Europe/Moscow')); +1970-01-01 03:00:00 +SELECT CAST('2020-01-01 00:00:00.3' AS DateTime64(0, 'Europe/Moscow')); +2020-01-01 00:00:00 +SELECT toDateTime64(bitShiftLeft(toUInt64(1), 33), 2, 'Europe/Moscow') FORMAT Null; +SELECT toTimeZone(toDateTime(-2., 2), 'Europe/Moscow'); 1970-01-01 03:00:00.00 -SELECT toDateTime64(-2., 2); +SELECT toDateTime64(-2., 2, 'Europe/Moscow'); 1970-01-01 03:00:00.00 -SELECT toDateTime64(toFloat32(bitShiftLeft(toUInt64(1),33)), 2); -2106-02-07 09:28:16.00 -SELECT toDateTime64(toFloat64(bitShiftLeft(toUInt64(1),33)), 2); -2106-02-07 09:28:15.00 +SELECT toDateTime64(toFloat32(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow'); +2106-02-07 09:00:00.00 +SELECT toDateTime64(toFloat64(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow') FORMAT Null; diff --git a/tests/queries/0_stateless/01691_DateTime64_clamp.sql b/tests/queries/0_stateless/01691_DateTime64_clamp.sql index f02d45a2cff..92d5a33328f 100644 --- a/tests/queries/0_stateless/01691_DateTime64_clamp.sql +++ b/tests/queries/0_stateless/01691_DateTime64_clamp.sql @@ -1,10 +1,10 @@ -- { echo } -SELECT toDateTime(-2, 2); -SELECT toDateTime64(-2, 2); -SELECT CAST(-1 AS DateTime64); -SELECT CAST('2020-01-01 00:00:00.3' AS DateTime64); -SELECT toDateTime64(bitShiftLeft(toUInt64(1),33), 2); -SELECT toDateTime(-2., 2); -SELECT toDateTime64(-2., 2); -SELECT toDateTime64(toFloat32(bitShiftLeft(toUInt64(1),33)), 2); -SELECT toDateTime64(toFloat64(bitShiftLeft(toUInt64(1),33)), 2); +SELECT toTimeZone(toDateTime(-2, 2), 'Europe/Moscow'); +SELECT toDateTime64(-2, 2, 'Europe/Moscow'); +SELECT CAST(-1 AS DateTime64(0, 'Europe/Moscow')); +SELECT CAST('2020-01-01 00:00:00.3' AS DateTime64(0, 'Europe/Moscow')); +SELECT toDateTime64(bitShiftLeft(toUInt64(1), 33), 2, 'Europe/Moscow') FORMAT Null; +SELECT toTimeZone(toDateTime(-2., 2), 'Europe/Moscow'); +SELECT toDateTime64(-2., 2, 'Europe/Moscow'); +SELECT toDateTime64(toFloat32(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow'); +SELECT toDateTime64(toFloat64(bitShiftLeft(toUInt64(1),33)), 2, 'Europe/Moscow') FORMAT Null; From edee592cb0859cd96569d54fc18ed0adb6a09fb7 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Mon, 22 Feb 2021 20:56:49 +0300 Subject: [PATCH 314/510] Fix --- src/Client/Connection.cpp | 1 + src/Client/ConnectionEstablisher.cpp | 1 + 2 files changed, 2 insertions(+) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 80d44a336a5..6f8fc3d12f2 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -110,6 +110,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts) in = std::make_shared(*socket); in->setAsyncCallback(std::move(async_callback)); + async_callback = {}; out = std::make_shared(*socket); diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 17b8832aab4..b3bb3c3a408 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -36,6 +36,7 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: { result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); + async_callback = {}; UInt64 server_revision = 0; if (table_to_check) From 99875c23bcd7446c09c2c5e7d4e2746153e5cd91 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 22 Feb 2021 21:34:23 +0300 Subject: [PATCH 315/510] Documentation low cardinality fix --- docs/ru/sql-reference/data-types/lowcardinality.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/lowcardinality.md b/docs/ru/sql-reference/data-types/lowcardinality.md index 3b5b337d731..d94cedd29ce 100644 --- a/docs/ru/sql-reference/data-types/lowcardinality.md +++ b/docs/ru/sql-reference/data-types/lowcardinality.md @@ -23,7 +23,7 @@ LowCardinality(data_type) Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных. -При работе со строками используйте `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. +При работе со строками, использование `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. ## Пример From 89c311981d00debc4b3a2194c7116c55d6b0c0fa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Feb 2021 22:27:31 +0300 Subject: [PATCH 316/510] Fix buffer size for trace collection [1]: https://clickhouse-test-reports.s3.yandex.net/0/a860e3e7582f489a640c42db3bd4bd320f97a5bc/stress_test_(debug).html#fail1 --- src/Common/TraceCollector.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index cbac9cd1a19..583dd5f6d34 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -60,10 +60,14 @@ void TraceCollector::collect(TraceType trace_type, const StackTrace & stack_trac 8 * sizeof(char) + // maximum VarUInt length for string size QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length sizeof(UInt8) + // number of stack frames - sizeof(StackTrace::Frames) + // collected stack trace, maximum capacity + sizeof(StackTrace::FramePointers) + // collected stack trace, maximum capacity sizeof(TraceType) + // trace type sizeof(UInt64) + // thread_id sizeof(Int64); // size + /// Write should be atomic to avoid overlaps + /// (since recursive collect() is possible) + static_assert(buf_size < PIPE_BUF, "Only write of PIPE_BUF to pipe is atomic"); + char buffer[buf_size]; WriteBufferFromFileDescriptorDiscardOnFailure out(pipe.fds_rw[1], buf_size, buffer); From 8520fc3e4c08fd97bd6e62474ebb8d92aa26983a Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 15 Feb 2021 21:53:24 +0300 Subject: [PATCH 317/510] Replace null fields in tuple during parsing with default values --- .../Impl/ConstantExpressionTemplate.cpp | 9 +++++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 34 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 1685688f02d..b2f4accf6a3 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -36,6 +36,7 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int SYNTAX_ERROR; + extern const int TYPE_MISMATCH; } @@ -209,6 +210,14 @@ private: if (map.size() % 2) return false; } + else if (literal->value.getType() == Field::Types::Tuple) + { + const Tuple & tuple = literal->value.get(); + + for (const auto & value : tuple) + if (value.isNull()) + return true; + } String column_name = "_dummy_" + std::to_string(replaced_literals.size()); replaced_literals.emplace_back(literal, column_name, force_nullable); diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 34a4a98f16b..247a8d082b5 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -12,6 +12,7 @@ #include #include #include +#include namespace DB @@ -181,6 +182,35 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) } } +namespace +{ + void tryToReplaceNullFieldsInTupleWithDefaultValues(Field & value, const IDataType & type) + { + const DataTypeTuple * type_tuple = typeid_cast(&type); + + if (!type_tuple || value.getType() != Field::Types::Tuple) + return; + + Tuple & tuple_value = value.get(); + + size_t src_tuple_size = tuple_value.size(); + size_t dst_tuple_size = type_tuple->getElements().size(); + + if (src_tuple_size != dst_tuple_size) + throw Exception("Bad size of tuple. Expected size: " + std::to_string(src_tuple_size) + ", actual size: " + std::to_string(dst_tuple_size), ErrorCodes::TYPE_MISMATCH); + + for (size_t i = 0; i < src_tuple_size; ++i) + { + const auto & element_type = *(type_tuple->getElements()[i]); + + if (tuple_value[i].isNull() && !element_type.isNullable()) + tuple_value[i] = element_type.getDefault(); + + tryToReplaceNullFieldsInTupleWithDefaultValues(tuple_value[i], element_type); + } + } +} + bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx) { const Block & header = getPort().getHeader(); @@ -298,6 +328,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx buf.position() = const_cast(token_iterator->begin); std::pair value_raw = evaluateConstantExpression(ast, *context); + + if (format_settings.null_as_default) + tryToReplaceNullFieldsInTupleWithDefaultValues(value_raw.first, type); + Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); /// Check that we are indeed allowed to insert a NULL. From 3f6050cee9da774864cec3711ade6acf3f82e625 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 15 Feb 2021 21:57:19 +0300 Subject: [PATCH 318/510] Fixed error code include --- src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index b2f4accf6a3..4aed9979ab2 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -36,7 +36,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int SYNTAX_ERROR; - extern const int TYPE_MISMATCH; } From d6b53d2b0297d1f039db08d9e31289989d4125e9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 15 Feb 2021 22:21:37 +0300 Subject: [PATCH 319/510] Added test --- .../01715_tuple_insert_null_as_default.reference | 2 ++ .../01715_tuple_insert_null_as_default.sql | 12 ++++++++++++ 2 files changed, 14 insertions(+) create mode 100644 tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference create mode 100644 tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference new file mode 100644 index 00000000000..982dfb6ab3c --- /dev/null +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference @@ -0,0 +1,2 @@ +(0,1) +(0,1) diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql new file mode 100644 index 00000000000..82b218a3d7f --- /dev/null +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS test_tuple; +CREATE TABLE test_tuple (value Tuple(UInt8, UInt8)) ENGINE=TinyLog; + +SET input_format_null_as_default = 1; +INSERT INTO test_tuple VALUES ((NULL, 1)); +SELECT * FROM test_tuple; + +SET input_format_null_as_default = 0; +INSERT INTO test_tuple VALUES ((NULL, 2)); -- { clientError 53 } +SELECT * FROM test_tuple; + +DROP TABLE test_tuple; From 992747e5ece4db051fceb4a4e489c892a91856cd Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 17 Feb 2021 13:29:06 +0300 Subject: [PATCH 320/510] Updated for Array --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 64 +++++++++++++------ ...715_tuple_insert_null_as_default.reference | 7 ++ .../01715_tuple_insert_null_as_default.sql | 32 ++++++++++ 3 files changed, 85 insertions(+), 18 deletions(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 247a8d082b5..384e46b3b80 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -13,6 +13,7 @@ #include #include #include +#include namespace DB @@ -26,6 +27,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int CANNOT_READ_ALL_DATA; + extern const int UNSUPPORTED_METHOD; } @@ -184,29 +186,52 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) namespace { - void tryToReplaceNullFieldsInTupleWithDefaultValues(Field & value, const IDataType & type) + void tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(Field & value, const IDataType & type, size_t stack_depth) { + if (stack_depth > 1000) + throw Exception("Stack overflow for replacing null fields in Tuple or Array", ErrorCodes::UNSUPPORTED_METHOD); + const DataTypeTuple * type_tuple = typeid_cast(&type); + const DataTypeArray * type_array = typeid_cast(&type); - if (!type_tuple || value.getType() != Field::Types::Tuple) - return; - - Tuple & tuple_value = value.get(); - - size_t src_tuple_size = tuple_value.size(); - size_t dst_tuple_size = type_tuple->getElements().size(); - - if (src_tuple_size != dst_tuple_size) - throw Exception("Bad size of tuple. Expected size: " + std::to_string(src_tuple_size) + ", actual size: " + std::to_string(dst_tuple_size), ErrorCodes::TYPE_MISMATCH); - - for (size_t i = 0; i < src_tuple_size; ++i) + if (type_tuple && value.getType() == Field::Types::Tuple) { - const auto & element_type = *(type_tuple->getElements()[i]); + Tuple & tuple_value = value.get(); - if (tuple_value[i].isNull() && !element_type.isNullable()) - tuple_value[i] = element_type.getDefault(); + size_t src_tuple_size = tuple_value.size(); + size_t dst_tuple_size = type_tuple->getElements().size(); - tryToReplaceNullFieldsInTupleWithDefaultValues(tuple_value[i], element_type); + if (src_tuple_size != dst_tuple_size) + throw Exception(fmt::format("Bad size of tuple. Expected size: {}, actual size: {}.", + std::to_string(src_tuple_size), std::to_string(dst_tuple_size)), ErrorCodes::TYPE_MISMATCH); + + for (size_t i = 0; i < src_tuple_size; ++i) + { + const auto & element_type = *(type_tuple->getElements()[i]); + + if (tuple_value[i].isNull() && !element_type.isNullable()) + tuple_value[i] = element_type.getDefault(); + + tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(tuple_value[i], element_type, stack_depth + 1); + } + } + else if (type_array && value.getType() == Field::Types::Array) + { + const auto & element_type = *(type_array->getNestedType()); + + if (element_type.isNullable()) + return; + + Array & array_value = value.get(); + size_t array_value_size = array_value.size(); + + for (size_t i = 0; i < array_value_size; ++i) + { + if (array_value[i].isNull()) + array_value[i] = element_type.getDefault(); + + tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(array_value[i], element_type, stack_depth + 1); + } } } } @@ -330,7 +355,10 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx std::pair value_raw = evaluateConstantExpression(ast, *context); if (format_settings.null_as_default) - tryToReplaceNullFieldsInTupleWithDefaultValues(value_raw.first, type); + { + size_t initial_stack_depth = 0; + tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(value_raw.first, type, initial_stack_depth); + } Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference index 982dfb6ab3c..d49d3392cb9 100644 --- a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference @@ -1,2 +1,9 @@ +Tuple (0,1) (0,1) +Tuple nested in Array +[(0,1)] +[(0,1)] +Tuple nested in Array nested in Tuple +(0,[(0,1)]) +(0,[(0,1)]) diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql index 82b218a3d7f..44a6ff667ff 100644 --- a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql @@ -1,3 +1,5 @@ +SELECT 'Tuple'; + DROP TABLE IF EXISTS test_tuple; CREATE TABLE test_tuple (value Tuple(UInt8, UInt8)) ENGINE=TinyLog; @@ -10,3 +12,33 @@ INSERT INTO test_tuple VALUES ((NULL, 2)); -- { clientError 53 } SELECT * FROM test_tuple; DROP TABLE test_tuple; + +SELECT 'Tuple nested in Array'; + +DROP TABLE IF EXISTS test_tuple_nested_in_array; +CREATE TABLE test_tuple_nested_in_array (value Array(Tuple(UInt8, UInt8))) ENGINE=TinyLog; + +SET input_format_null_as_default = 1; +INSERT INTO test_tuple_nested_in_array VALUES ([(NULL, 1)]); +SELECT * FROM test_tuple_nested_in_array; + +SET input_format_null_as_default = 0; +INSERT INTO test_tuple_nested_in_array VALUES ([(NULL, 1)]); -- { clientError 53 } +SELECT * FROM test_tuple_nested_in_array; + +DROP TABLE test_tuple_nested_in_array; + +SELECT 'Tuple nested in Array nested in Tuple'; + +DROP TABLE IF EXISTS test_tuple_nested_in_array_nested_in_tuple; +CREATE TABLE test_tuple_nested_in_array_nested_in_tuple (value Tuple(UInt8, Array(Tuple(UInt8, UInt8)))) ENGINE=TinyLog; + +SET input_format_null_as_default = 1; +INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 1)]) ); +SELECT * FROM test_tuple_nested_in_array_nested_in_tuple; + +SET input_format_null_as_default = 0; +INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 1)]) ); -- { clientError 53 } +SELECT * FROM test_tuple_nested_in_array_nested_in_tuple; + +DROP TABLE test_tuple_nested_in_array_nested_in_tuple; From 5eec754fd616c03a5d49cdd305142a09e818d905 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 17 Feb 2021 13:33:41 +0300 Subject: [PATCH 321/510] Use standard checkStackSize --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 384e46b3b80..e1fa518e89b 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -8,8 +8,9 @@ #include #include #include -#include #include +#include +#include #include #include #include @@ -186,10 +187,9 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) namespace { - void tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(Field & value, const IDataType & type, size_t stack_depth) + void tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(Field & value, const IDataType & type) { - if (stack_depth > 1000) - throw Exception("Stack overflow for replacing null fields in Tuple or Array", ErrorCodes::UNSUPPORTED_METHOD); + checkStackSize(); const DataTypeTuple * type_tuple = typeid_cast(&type); const DataTypeArray * type_array = typeid_cast(&type); @@ -212,7 +212,7 @@ namespace if (tuple_value[i].isNull() && !element_type.isNullable()) tuple_value[i] = element_type.getDefault(); - tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(tuple_value[i], element_type, stack_depth + 1); + tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(tuple_value[i], element_type); } } else if (type_array && value.getType() == Field::Types::Array) @@ -230,7 +230,7 @@ namespace if (array_value[i].isNull()) array_value[i] = element_type.getDefault(); - tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(array_value[i], element_type, stack_depth + 1); + tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(array_value[i], element_type); } } } @@ -355,10 +355,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx std::pair value_raw = evaluateConstantExpression(ast, *context); if (format_settings.null_as_default) - { - size_t initial_stack_depth = 0; - tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(value_raw.first, type, initial_stack_depth); - } + tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(value_raw.first, type); Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); From 3b80395b1b06fbae4dfcc8356ca431e5cb82437c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 17 Feb 2021 19:13:38 +0300 Subject: [PATCH 322/510] Fixed style check --- src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index e1fa518e89b..f1200fb8672 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -28,7 +28,6 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int CANNOT_READ_ALL_DATA; - extern const int UNSUPPORTED_METHOD; } From f001a6755e73e86e9b8e18c1325ec0d02d4d3f84 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sun, 21 Feb 2021 14:57:03 +0300 Subject: [PATCH 323/510] Updated for map --- src/DataTypes/DataTypeTuple.cpp | 37 ++++++------ src/Interpreters/convertFieldToType.cpp | 38 ++++++++++++ .../Formats/Impl/ValuesBlockInputFormat.cpp | 59 +++++++++++++++---- .../Formats/Impl/ValuesBlockInputFormat.h | 2 +- ...715_tuple_insert_null_as_default.reference | 14 +++-- .../01715_tuple_insert_null_as_default.sql | 37 +++++++++++- 6 files changed, 150 insertions(+), 37 deletions(-) diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 2261e776ea2..5d2050c09e9 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -138,7 +138,7 @@ void DataTypeTuple::serializeBinary(const IColumn & column, size_t row_num, Writ idx_elem.second->serializeBinary(extractElementColumn(column, idx_elem.first), row_num, ostr); } - +/// Function must atomically insert values into tuple column template static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl) { @@ -151,7 +151,8 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl) // Check that all columns now have the same size. size_t new_size = column.size(); - for (auto i : ext::range(1, ext::size(elems))) + + for (auto i : ext::range(0, ext::size(elems))) { const auto & element_column = extractElementColumn(column, i); if (element_column.size() != new_size) @@ -168,6 +169,7 @@ static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl) for (const auto & i : ext::range(0, ext::size(elems))) { auto & element_column = extractElementColumn(column, i); + if (element_column.size() > old_size) element_column.popBack(1); } @@ -215,17 +217,18 @@ void DataTypeTuple::deserializeText(IColumn & column, ReadBuffer & istr, const F } elems[i]->deserializeAsTextQuoted(extractElementColumn(column, i), istr, settings); } - }); - // Special format for one element tuple (1,) - if (1 == elems.size()) - { + // Special format for one element tuple (1,) + if (1 == elems.size()) + { + skipWhitespaceIfAny(istr); + // Allow both (1) and (1,) + checkChar(',', istr); + } + skipWhitespaceIfAny(istr); - // Allow both (1) and (1,) - checkChar(',', istr); - } - skipWhitespaceIfAny(istr); - assertChar(')', istr); + assertChar(')', istr); + }); } void DataTypeTuple::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const @@ -290,10 +293,10 @@ void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con auto & element_column = extractElementColumn(column, element_pos); elems[element_pos]->deserializeAsTextJSON(element_column, istr, settings); } - }); - skipWhitespaceIfAny(istr); - assertChar('}', istr); + skipWhitespaceIfAny(istr); + assertChar('}', istr); + }); } else { @@ -312,10 +315,10 @@ void DataTypeTuple::deserializeTextJSON(IColumn & column, ReadBuffer & istr, con } elems[i]->deserializeAsTextJSON(extractElementColumn(column, i), istr, settings); } - }); - skipWhitespaceIfAny(istr); - assertChar(']', istr); + skipWhitespaceIfAny(istr); + assertChar(']', istr); + }); } } diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 73bf493fa65..1d93ef56dea 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -276,6 +276,44 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID return have_unconvertible_element ? Field(Null()) : Field(res); } } + else if (const DataTypeMap * type_map = typeid_cast(&type)) + { + if (src.getType() == Field::Types::Map) + { + const auto & key_type = *type_map->getKeyType(); + const auto & value_type = *type_map->getValueType(); + + const auto & map = src.get(); + size_t map_size = map.size(); + + Map res(map_size); + + bool have_unconvertible_element = false; + + for (size_t i = 0; i < map_size; ++i) + { + const auto & map_entry = map[i].get(); + + const auto & key = map_entry[0]; + const auto & value = map_entry[1]; + + Tuple updated_entry(2); + + updated_entry[0] = convertFieldToType(key, key_type); + + if (updated_entry[0].isNull() && !key_type.isNullable()) + have_unconvertible_element = true; + + updated_entry[1] = convertFieldToType(value, value_type); + if (updated_entry[1].isNull() && !value_type.isNullable()) + have_unconvertible_element = true; + + res[i] = updated_entry; + } + + return have_unconvertible_element ? Field(Null()) : Field(res); + } + } else if (const DataTypeAggregateFunction * agg_func_type = typeid_cast(&type)) { if (src.getType() != Field::Types::AggregateFunctionState) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index f1200fb8672..7d911beb741 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace DB @@ -186,19 +187,20 @@ bool ValuesBlockInputFormat::tryReadValue(IColumn & column, size_t column_idx) namespace { - void tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(Field & value, const IDataType & type) + void tryToReplaceNullFieldsInComplexTypesWithDefaultValues(Field & value, const IDataType & data_type) { checkStackSize(); - const DataTypeTuple * type_tuple = typeid_cast(&type); - const DataTypeArray * type_array = typeid_cast(&type); + WhichDataType type(data_type); - if (type_tuple && value.getType() == Field::Types::Tuple) + if (type.isTuple() && value.getType() == Field::Types::Tuple) { + const DataTypeTuple & type_tuple = static_cast(data_type); + Tuple & tuple_value = value.get(); size_t src_tuple_size = tuple_value.size(); - size_t dst_tuple_size = type_tuple->getElements().size(); + size_t dst_tuple_size = type_tuple.getElements().size(); if (src_tuple_size != dst_tuple_size) throw Exception(fmt::format("Bad size of tuple. Expected size: {}, actual size: {}.", @@ -206,17 +208,18 @@ namespace for (size_t i = 0; i < src_tuple_size; ++i) { - const auto & element_type = *(type_tuple->getElements()[i]); + const auto & element_type = *(type_tuple.getElements()[i]); if (tuple_value[i].isNull() && !element_type.isNullable()) tuple_value[i] = element_type.getDefault(); - tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(tuple_value[i], element_type); + tryToReplaceNullFieldsInComplexTypesWithDefaultValues(tuple_value[i], element_type); } } - else if (type_array && value.getType() == Field::Types::Array) + else if (type.isArray() && value.getType() == Field::Types::Array) { - const auto & element_type = *(type_array->getNestedType()); + const DataTypeArray & type_aray = static_cast(data_type); + const auto & element_type = *(type_aray.getNestedType()); if (element_type.isNullable()) return; @@ -229,7 +232,35 @@ namespace if (array_value[i].isNull()) array_value[i] = element_type.getDefault(); - tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(array_value[i], element_type); + tryToReplaceNullFieldsInComplexTypesWithDefaultValues(array_value[i], element_type); + } + } + else if (type.isMap() && value.getType() == Field::Types::Map) + { + const DataTypeMap & type_map = static_cast(data_type); + + const auto & key_type = *type_map.getKeyType(); + const auto & value_type = *type_map.getValueType(); + + auto & map = value.get(); + size_t map_size = map.size(); + + for (size_t i = 0; i < map_size; ++i) + { + auto & map_entry = map[i].get(); + + auto & entry_key = map_entry[0]; + auto & entry_value = map_entry[1]; + + if (entry_key.isNull() && !key_type.isNullable()) + entry_key = key_type.getDefault(); + + tryToReplaceNullFieldsInComplexTypesWithDefaultValues(entry_key, key_type); + + if (entry_value.isNull() && !value_type.isNullable()) + entry_value = value_type.getDefault(); + + tryToReplaceNullFieldsInComplexTypesWithDefaultValues(entry_value, value_type); } } } @@ -353,10 +384,12 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx std::pair value_raw = evaluateConstantExpression(ast, *context); - if (format_settings.null_as_default) - tryToReplaceNullFieldsInTupleOrArrayWithDefaultValues(value_raw.first, type); + Field & expression_value = value_raw.first; - Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); + if (format_settings.null_as_default) + tryToReplaceNullFieldsInComplexTypesWithDefaultValues(expression_value, type); + + Field value = convertFieldToType(expression_value, type, value_raw.second.get()); /// Check that we are indeed allowed to insert a NULL. if (value.isNull() && !type.isNullable()) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index 01deb2865bb..a541870e484 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -48,7 +48,7 @@ private: SingleExpressionEvaluation }; - typedef std::vector> ConstantExpressionTemplates; + using ConstantExpressionTemplates = std::vector>; Chunk generate() override; diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference index d49d3392cb9..20f0fadfdcf 100644 --- a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.reference @@ -2,8 +2,14 @@ Tuple (0,1) (0,1) Tuple nested in Array -[(0,1)] -[(0,1)] +[(0,2),(3,0),(0,4)] +[(0,2),(3,0),(0,4)] Tuple nested in Array nested in Tuple -(0,[(0,1)]) -(0,[(0,1)]) +(0,[(0,2),(3,0),(0,4)]) +(0,[(0,2),(3,0),(0,4)]) +Tuple nested in Map +{'test':(0,1)} +{'test':(0,1)} +Tuple nested in Map nested in Tuple +(0,{'test':(0,1)}) +(0,{'test':(0,1)}) diff --git a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql index 44a6ff667ff..d5fd9af22bd 100644 --- a/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql +++ b/tests/queries/0_stateless/01715_tuple_insert_null_as_default.sql @@ -19,7 +19,7 @@ DROP TABLE IF EXISTS test_tuple_nested_in_array; CREATE TABLE test_tuple_nested_in_array (value Array(Tuple(UInt8, UInt8))) ENGINE=TinyLog; SET input_format_null_as_default = 1; -INSERT INTO test_tuple_nested_in_array VALUES ([(NULL, 1)]); +INSERT INTO test_tuple_nested_in_array VALUES ([(NULL, 2), (3, NULL), (NULL, 4)]); SELECT * FROM test_tuple_nested_in_array; SET input_format_null_as_default = 0; @@ -34,7 +34,7 @@ DROP TABLE IF EXISTS test_tuple_nested_in_array_nested_in_tuple; CREATE TABLE test_tuple_nested_in_array_nested_in_tuple (value Tuple(UInt8, Array(Tuple(UInt8, UInt8)))) ENGINE=TinyLog; SET input_format_null_as_default = 1; -INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 1)]) ); +INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 2), (3, NULL), (NULL, 4)]) ); SELECT * FROM test_tuple_nested_in_array_nested_in_tuple; SET input_format_null_as_default = 0; @@ -42,3 +42,36 @@ INSERT INTO test_tuple_nested_in_array_nested_in_tuple VALUES ( (NULL, [(NULL, 1 SELECT * FROM test_tuple_nested_in_array_nested_in_tuple; DROP TABLE test_tuple_nested_in_array_nested_in_tuple; + +SELECT 'Tuple nested in Map'; + +SET allow_experimental_map_type = 1; + +DROP TABLE IF EXISTS test_tuple_nested_in_map; +CREATE TABLE test_tuple_nested_in_map (value Map(String, Tuple(UInt8, UInt8))) ENGINE=TinyLog; + +SET input_format_null_as_default = 1; +INSERT INTO test_tuple_nested_in_map VALUES (map('test', (NULL, 1))); + +SELECT * FROM test_tuple_nested_in_map; + +SET input_format_null_as_default = 0; +INSERT INTO test_tuple_nested_in_map VALUES (map('test', (NULL, 1))); -- { clientError 53 } +SELECT * FROM test_tuple_nested_in_map; + +DROP TABLE test_tuple_nested_in_map; + +SELECT 'Tuple nested in Map nested in Tuple'; + +DROP TABLE IF EXISTS test_tuple_nested_in_map_nested_in_tuple; +CREATE TABLE test_tuple_nested_in_map_nested_in_tuple (value Tuple(UInt8, Map(String, Tuple(UInt8, UInt8)))) ENGINE=TinyLog; + +SET input_format_null_as_default = 1; +INSERT INTO test_tuple_nested_in_map_nested_in_tuple VALUES ( (NULL, map('test', (NULL, 1))) ); +SELECT * FROM test_tuple_nested_in_map_nested_in_tuple; + +SET input_format_null_as_default = 0; +INSERT INTO test_tuple_nested_in_map_nested_in_tuple VALUES ( (NULL, map('test', (NULL, 1))) ); -- { clientError 53 } +SELECT * FROM test_tuple_nested_in_map_nested_in_tuple; + +DROP TABLE test_tuple_nested_in_map_nested_in_tuple; From f274343cf0e949e43f4c124e1ac9e47a2cc8b1d4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 22 Feb 2021 23:32:06 +0300 Subject: [PATCH 324/510] Fixed Map type getExtremes --- src/Columns/ColumnMap.cpp | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index cc2640a9cf6..e0e79d071b4 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -229,7 +229,21 @@ void ColumnMap::protect() void ColumnMap::getExtremes(Field & min, Field & max) const { - nested->getExtremes(min, max); + Field nested_min; + Field nested_max; + + nested->getExtremes(nested_min, nested_max); + + /// Convert result Array fields to Map fields because client expect min and max field to have type Map + + Array nested_min_value = nested_min.get(); + Array nested_max_value = nested_max.get(); + + Map map_min_value(nested_min_value.begin(), nested_min_value.end()); + Map map_max_value(nested_max_value.begin(), nested_max_value.end()); + + min = std::move(map_min_value); + max = std::move(map_max_value); } void ColumnMap::forEachSubcolumn(ColumnCallback callback) From d7c3dfe5fabb768574656603b19c119464139638 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Feb 2021 23:51:58 +0300 Subject: [PATCH 325/510] Don't allow to start with broken changelogs --- src/Coordination/Changelog.cpp | 11 ++++++++++ src/Coordination/tests/gtest_for_build.cpp | 25 ++++++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/src/Coordination/Changelog.cpp b/src/Coordination/Changelog.cpp index 45d80eb2757..63102ceedaa 100644 --- a/src/Coordination/Changelog.cpp +++ b/src/Coordination/Changelog.cpp @@ -264,12 +264,20 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_index) size_t incomplete_log_index = 0; ChangelogReadResult result{}; + bool started = false; for (const auto & [changelog_start_index, changelog_description] : existing_changelogs) { entries_in_last = changelog_description.to_log_index - changelog_description.from_log_index + 1; if (changelog_description.to_log_index >= from_log_index) { + if (!started) + { + if (changelog_description.from_log_index > start_index) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Cannot read changelog from index {}, smallest available index {}", start_index, changelog_description.from_log_index); + started = true; + } + ChangelogReader reader(changelog_description.path); result = reader.readChangelog(logs, from_log_index, index_to_start_pos, log); total_read += result.entries_read; @@ -283,6 +291,9 @@ void Changelog::readChangelogAndInitWriter(size_t from_log_index) } } + if (!started && start_index != 1) + throw Exception(ErrorCodes::CORRUPTED_DATA, "Required to read data from {}, but we don't have any active changelogs", from_log_index); + if (incomplete_log_index != 0) { /// All subsequent logs shouldn't exist. But they may exist if we crashed after writeAt started. Remove them. diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index 163393da573..dcd4e718f3d 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -917,6 +917,31 @@ TEST(CoordinationTest, ChangelogTestReadAfterBrokenTruncate2) EXPECT_EQ(changelog_reader2.last_entry()->get_term(), 7777); } +TEST(CoordinationTest, ChangelogTestLostFiles) +{ + ChangelogDirTest test("./logs"); + + DB::NuKeeperLogStore changelog("./logs", 20, true); + changelog.init(1); + + for (size_t i = 0; i < 35; ++i) + { + auto entry = getLogEntry(std::to_string(i) + "_hello_world", (i + 44) * 10); + changelog.append(entry); + } + + EXPECT_TRUE(fs::exists("./logs/changelog_1_20.bin")); + EXPECT_TRUE(fs::exists("./logs/changelog_21_40.bin")); + + fs::remove("./logs/changelog_1_20.bin"); + + DB::NuKeeperLogStore changelog_reader("./logs", 20, true); + EXPECT_THROW(changelog_reader.init(5), DB::Exception); + + fs::remove("./logs/changelog_21_40.bin"); + EXPECT_THROW(changelog_reader.init(3), DB::Exception); +} + int main(int argc, char ** argv) { Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); From 4661073750775d1b43129284c1300969aa73f72e Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 23 Feb 2021 00:09:20 +0300 Subject: [PATCH 326/510] Update src/Storages/MergeTree/registerStorageMergeTree.cpp --- src/Storages/MergeTree/registerStorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index ba70b7aa65c..d48f85f3e4d 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -427,7 +427,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) "No replica name in config" + getMergeTreeVerboseHelp(is_extended_storage_def), ErrorCodes::NO_REPLICA_NAME_GIVEN); ++arg_num; } - else if (is_extended_storage_def && (arg_cnt == 0 || !engine_args[arg_num]->as())) + else if (is_extended_storage_def && (arg_cnt == 0 || !engine_args[arg_num]->as() || (arg_cnt == 1 && merging_params.mode == MergeTreeData::MergingParams::Graphite))) { /// Try use default values if arguments are not specified. /// Note: {uuid} macro works for ON CLUSTER queries when database engine is Atomic. From 2aee89724306f187ebc3333a6602743cab4f44bf Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 22 Feb 2021 18:21:30 -0400 Subject: [PATCH 327/510] Update lowcardinality.md --- docs/ru/sql-reference/data-types/lowcardinality.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/data-types/lowcardinality.md b/docs/ru/sql-reference/data-types/lowcardinality.md index d94cedd29ce..52713e2d747 100644 --- a/docs/ru/sql-reference/data-types/lowcardinality.md +++ b/docs/ru/sql-reference/data-types/lowcardinality.md @@ -23,7 +23,7 @@ LowCardinality(data_type) Эффективность использования типа данных `LowCarditality` зависит от разнообразия данных. Если словарь содержит менее 10 000 различных значений, ClickHouse в основном показывает более высокую эффективность чтения и хранения данных. Если же словарь содержит более 100 000 различных значений, ClickHouse может работать хуже, чем при использовании обычных типов данных. -При работе со строками, использование `LowCardinality` вместо [Enum](enum.md). `LowCardinality` обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. +При работе со строками, использование `LowCardinality` вместо [Enum](enum.md) обеспечивает большую гибкость в использовании и часто показывает такую же или более высокую эффективность. ## Пример From f2b8f81e40eeadf3b85a1dae873828f4da40c8e7 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Mon, 22 Feb 2021 21:05:25 -0400 Subject: [PATCH 328/510] test for window functions --- .../01592_window_functions.reference | 81 +++++++++++++ .../0_stateless/01592_window_functions.sql | 107 ++++++++++++++++++ .../01592_window_functions1.reference | 4 + .../0_stateless/01592_window_functions1.sql | 35 ++++++ 4 files changed, 227 insertions(+) create mode 100644 tests/queries/0_stateless/01592_window_functions.reference create mode 100644 tests/queries/0_stateless/01592_window_functions.sql create mode 100644 tests/queries/0_stateless/01592_window_functions1.reference create mode 100644 tests/queries/0_stateless/01592_window_functions1.sql diff --git a/tests/queries/0_stateless/01592_window_functions.reference b/tests/queries/0_stateless/01592_window_functions.reference new file mode 100644 index 00000000000..8ca5086821b --- /dev/null +++ b/tests/queries/0_stateless/01592_window_functions.reference @@ -0,0 +1,81 @@ +---- Q1 ---- +Dell Vostro 800.00 Laptop 850 +HP Elite 1200.00 Laptop 850 +Lenovo Thinkpad 700.00 Laptop 850 +Sony VAIO 700.00 Laptop 850 +HTC One 400.00 Smartphone 500 +Microsoft Lumia 200.00 Smartphone 500 +Nexus 500.00 Smartphone 500 +iPhone 900.00 Smartphone 500 +Kindle Fire 150.00 Tablet 350 +Samsung Galaxy Tab 200.00 Tablet 350 +iPad 700.00 Tablet 350 +---- Q2 ---- +Lenovo Thinkpad Laptop 700.00 1 +Sony VAIO Laptop 700.00 1 +Dell Vostro Laptop 800.00 3 +HP Elite Laptop 1200.00 4 +Microsoft Lumia Smartphone 200.00 1 +HTC One Smartphone 400.00 2 +Nexus Smartphone 500.00 3 +iPhone Smartphone 900.00 4 +Kindle Fire Tablet 150.00 1 +Samsung Galaxy Tab Tablet 200.00 2 +iPad Tablet 700.00 3 +---- Q3 ---- +HP Elite Laptop 1200.00 1 +Dell Vostro Laptop 800.00 2 +Lenovo Thinkpad Laptop 700.00 3 +Sony VAIO Laptop 700.00 4 +iPhone Smartphone 900.00 1 +Nexus Smartphone 500.00 2 +HTC One Smartphone 400.00 3 +Microsoft Lumia Smartphone 200.00 4 +iPad Tablet 700.00 1 +Samsung Galaxy Tab Tablet 200.00 2 +Kindle Fire Tablet 150.00 3 +---- Q4 ---- +Lenovo Thinkpad Laptop 700.00 700.00 1 +Sony VAIO Laptop 700.00 700.00 1 +Dell Vostro Laptop 800.00 700.00 2 +HP Elite Laptop 1200.00 700.00 3 +Microsoft Lumia Smartphone 200.00 200.00 1 +HTC One Smartphone 400.00 200.00 2 +Nexus Smartphone 500.00 200.00 3 +iPhone Smartphone 900.00 200.00 4 +---- Q5 ---- +Sony VAIO Laptop 700.00 700.00 +Lenovo Thinkpad Laptop 700.00 700.00 +HP Elite Laptop 1200.00 700.00 +Dell Vostro Laptop 800.00 700.00 +iPhone Smartphone 900.00 900.00 +Nexus Smartphone 500.00 900.00 +Microsoft Lumia Smartphone 200.00 900.00 +HTC One Smartphone 400.00 900.00 +iPad Tablet 700.00 700.00 +Samsung Galaxy Tab Tablet 200.00 700.00 +Kindle Fire Tablet 150.00 700.00 +---- Q6 ---- +Dell Vostro Laptop 800.00 1200.00 +HP Elite Laptop 1200.00 1200.00 +Lenovo Thinkpad Laptop 700.00 1200.00 +Sony VAIO Laptop 700.00 1200.00 +HTC One Smartphone 400.00 900.00 +Microsoft Lumia Smartphone 200.00 900.00 +Nexus Smartphone 500.00 900.00 +iPhone Smartphone 900.00 900.00 +Kindle Fire Tablet 150.00 700.00 +Samsung Galaxy Tab Tablet 200.00 700.00 +iPad Tablet 700.00 700.00 +---- Q7 ---- +Dell Vostro 800.00 Laptop 733 850 +HP Elite 1200.00 Laptop 850 850 +Lenovo Thinkpad 700.00 Laptop 700 850 +Sony VAIO 700.00 Laptop 700 850 +HTC One 400.00 Smartphone 300 500 +Microsoft Lumia 200.00 Smartphone 200 500 +Nexus 500.00 Smartphone 367 500 +iPhone 900.00 Smartphone 500 500 +Kindle Fire 150.00 Tablet 150 350 +Samsung Galaxy Tab 200.00 Tablet 175 350 +iPad 700.00 Tablet 350 350 diff --git a/tests/queries/0_stateless/01592_window_functions.sql b/tests/queries/0_stateless/01592_window_functions.sql new file mode 100644 index 00000000000..8d5033fc821 --- /dev/null +++ b/tests/queries/0_stateless/01592_window_functions.sql @@ -0,0 +1,107 @@ +set allow_experimental_window_functions = 1; + +drop table if exists product_groups; +drop table if exists products; + +CREATE TABLE product_groups ( + group_id Int64, + group_name String +) Engine = Memory; + + +CREATE TABLE products ( + product_id Int64, + product_name String, + price DECIMAL(11, 2), + group_id Int64 +) Engine = Memory; + +INSERT INTO product_groups VALUES (1, 'Smartphone'),(2, 'Laptop'),(3, 'Tablet'); + +INSERT INTO products (product_id,product_name, group_id,price) VALUES (1, 'Microsoft Lumia', 1, 200), (2, 'HTC One', 1, 400), (3, 'Nexus', 1, 500), (4, 'iPhone', 1, 900),(5, 'HP Elite', 2, 1200),(6, 'Lenovo Thinkpad', 2, 700),(7, 'Sony VAIO', 2, 700),(8, 'Dell Vostro', 2, 800),(9, 'iPad', 3, 700),(10, 'Kindle Fire', 3, 150),(11, 'Samsung Galaxy Tab', 3, 200); + +select '---- Q1 ----'; + +SELECT + product_name, + price, + group_name, + AVG(price) OVER (PARTITION BY group_name) +FROM products INNER JOIN product_groups USING (group_id) +order by group_name, product_name, price; + +select '---- Q2 ----'; + +SELECT + product_name, + group_name, + price, + rank() OVER (PARTITION BY group_name ORDER BY price) rank +FROM products INNER JOIN product_groups USING (group_id) +order by group_name, rank, price; + +select '---- Q3 ----'; +SELECT + product_name, + group_name, + price, + row_number() OVER (PARTITION BY group_name ORDER BY price desc) rn +FROM products INNER JOIN product_groups USING (group_id) +ORDER BY group_name, rn; + +select '---- Q4 ----'; +SELECT * +FROM +( + SELECT + product_name, + group_name, + price, + min(price) OVER (PARTITION BY group_name) AS min_price, + dense_rank() OVER (PARTITION BY group_name ORDER BY price ASC) AS r + FROM products + INNER JOIN product_groups USING (group_id) +) AS t +WHERE min_price > 160 +ORDER BY + group_name ASC, + r ASC, + product_name ASC; + +select '---- Q5 ----'; +SELECT + product_name, + group_name, + price, + FIRST_VALUE (price) OVER (PARTITION BY group_name ORDER BY product_name desc) AS price_per_group_per_alphab +FROM products INNER JOIN product_groups USING (group_id) +order by group_name, product_name desc; + +select '---- Q6 ----'; +SELECT + product_name, + group_name, + price, + LAST_VALUE (price) OVER (PARTITION BY group_name ORDER BY + price RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING + ) AS highest_price_per_group +FROM + products +INNER JOIN product_groups USING (group_id) +order by group_name, product_name; + +select '---- Q7 ----'; +select product_name, price, group_name, round(avg0), round(avg1) +from ( +SELECT + product_name, + price, + group_name, + avg(price) OVER (PARTITION BY group_name ORDER BY price) avg0, + avg(price) OVER (PARTITION BY group_name ORDER BY + price RANGE BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) avg1 +FROM products INNER JOIN product_groups USING (group_id)) t +order by group_name, product_name, price; + +drop table product_groups; +drop table products; diff --git a/tests/queries/0_stateless/01592_window_functions1.reference b/tests/queries/0_stateless/01592_window_functions1.reference new file mode 100644 index 00000000000..5160cca9c3e --- /dev/null +++ b/tests/queries/0_stateless/01592_window_functions1.reference @@ -0,0 +1,4 @@ +---- arrays ---- +6360452672161319041 +---- window f ---- +6360452672161319041 diff --git a/tests/queries/0_stateless/01592_window_functions1.sql b/tests/queries/0_stateless/01592_window_functions1.sql new file mode 100644 index 00000000000..c7751ab9f33 --- /dev/null +++ b/tests/queries/0_stateless/01592_window_functions1.sql @@ -0,0 +1,35 @@ +drop table if exists stack; + +set allow_experimental_window_functions = 1; + +create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64) +Engine = MergeTree +partition by toYYYYMM(dt) +order by (brand_id, toStartOfHour(dt)); + +insert into stack +select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100, + toDateTime('2020-02-01 00:00:00')+number/10, intDiv(number,100)+1 +from numbers(10000000); + +select '---- arrays ----'; + +select cityHash64( toString( groupArray (tuple(*) ) )) from ( + select brand_id, rack_id, arrayJoin(arraySlice(arraySort(groupArray(quantity)),1,2)) quantity + from stack + group by brand_id, rack_id + order by brand_id, rack_id, quantity +) t; + + +select '---- window f ----'; + +select cityHash64( toString( groupArray (tuple(*) ) )) from ( + select brand_id, rack_id, quantity from + ( select brand_id, rack_id, quantity, row_number() over (partition by brand_id, rack_id order by quantity) rn + from stack ) as t0 + where rn <= 2 + order by brand_id, rack_id, quantity +) t; + +drop table if exists stack; From 2ebae14f12af45a89ace4dc6ace681669935698f Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Mon, 22 Feb 2021 23:40:38 -0400 Subject: [PATCH 329/510] mark test as long --- ...unctions1.reference => 01592_long_window_functions1.reference} | 0 ...592_window_functions1.sql => 01592_long_window_functions1.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01592_window_functions1.reference => 01592_long_window_functions1.reference} (100%) rename tests/queries/0_stateless/{01592_window_functions1.sql => 01592_long_window_functions1.sql} (100%) diff --git a/tests/queries/0_stateless/01592_window_functions1.reference b/tests/queries/0_stateless/01592_long_window_functions1.reference similarity index 100% rename from tests/queries/0_stateless/01592_window_functions1.reference rename to tests/queries/0_stateless/01592_long_window_functions1.reference diff --git a/tests/queries/0_stateless/01592_window_functions1.sql b/tests/queries/0_stateless/01592_long_window_functions1.sql similarity index 100% rename from tests/queries/0_stateless/01592_window_functions1.sql rename to tests/queries/0_stateless/01592_long_window_functions1.sql From 6b39e61d09e8ed13b0e858d851428db9a426c71b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 23 Feb 2021 07:45:53 +0300 Subject: [PATCH 330/510] Decrease QUERY_ID_MAX_LEN for freebsd/osx (PIPE_BUF=512) --- src/Common/TraceCollector.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/TraceCollector.cpp b/src/Common/TraceCollector.cpp index 583dd5f6d34..ab1845ebbd2 100644 --- a/src/Common/TraceCollector.cpp +++ b/src/Common/TraceCollector.cpp @@ -22,7 +22,9 @@ namespace { /// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id. /// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler. - constexpr size_t QUERY_ID_MAX_LEN = 1024; + /// + /// And it cannot be large, since otherwise it will not fit into PIPE_BUF. + constexpr size_t QUERY_ID_MAX_LEN = sizeof("00000000-0000-0000-0000-000000000000") - 1; // 36 } LazyPipeFDs pipe; From 2ced21c837006053542b9f04c338faebf504f413 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 23 Feb 2021 07:52:01 +0300 Subject: [PATCH 331/510] Decrease number of stored stack frame pointers to fit into 512 PIPE_BUF --- src/Common/StackTrace.h | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Common/StackTrace.h b/src/Common/StackTrace.h index 58660f9e4da..35df77b6b8a 100644 --- a/src/Common/StackTrace.h +++ b/src/Common/StackTrace.h @@ -37,8 +37,12 @@ public: static constexpr size_t capacity = #ifndef NDEBUG - /* The stacks are normally larger in debug version due to less inlining. */ - 64 + /* The stacks are normally larger in debug version due to less inlining. + * + * NOTE: it cannot be larger then 56 right now, since otherwise it will + * not fit into minimal PIPE_BUF (512) in TraceCollector. + */ + 56 #else 32 #endif From ddc9fb151ee45f506a1bb97e4b284338a2ed19cb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 23 Feb 2021 11:37:27 +0300 Subject: [PATCH 332/510] Fix tests --- .../test.py | 29 ++++++++++--------- .../test_testkeeper_multinode_simple/test.py | 7 +++-- 2 files changed, 19 insertions(+), 17 deletions(-) diff --git a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py index 0f17fff8f76..47064413b45 100644 --- a/tests/integration/test_testkeeper_multinode_blocade_leader/test.py +++ b/tests/integration/test_testkeeper_multinode_blocade_leader/test.py @@ -6,6 +6,7 @@ import os import time from multiprocessing.dummy import Pool from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) @@ -94,9 +95,9 @@ def test_blocade_leader(started_cluster): node1.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) node3.query("SYSTEM SYNC REPLICA ordinary.t1", timeout=10) - assert node1.query("SELECT COUNT() FROM ordinary.t1") == "10\n" - assert node2.query("SELECT COUNT() FROM ordinary.t1") == "10\n" - assert node3.query("SELECT COUNT() FROM ordinary.t1") == "10\n" + assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t1", "10") + assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t1", "10") + assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t1", "10") with PartitionManager() as pm: pm.partition_instances(node2, node1) @@ -187,9 +188,9 @@ def test_blocade_leader(started_cluster): for num, node in enumerate([node1, node2, node3]): dump_zk(node, '/clickhouse/t1', '/clickhouse/t1/replicas/{}'.format(num + 1)) - assert node1.query("SELECT COUNT() FROM ordinary.t1") == "310\n" - assert node2.query("SELECT COUNT() FROM ordinary.t1") == "310\n" - assert node3.query("SELECT COUNT() FROM ordinary.t1") == "310\n" + assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t1", "310") + assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t1", "310") + assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t1", "310") def dump_zk(node, zk_path, replica_path): @@ -243,9 +244,9 @@ def test_blocade_leader_twice(started_cluster): node1.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) node3.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - assert node1.query("SELECT COUNT() FROM ordinary.t2") == "10\n" - assert node2.query("SELECT COUNT() FROM ordinary.t2") == "10\n" - assert node3.query("SELECT COUNT() FROM ordinary.t2") == "10\n" + assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t2", "10") + assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "10") + assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "10") with PartitionManager() as pm: pm.partition_instances(node2, node1) @@ -288,8 +289,8 @@ def test_blocade_leader_twice(started_cluster): node2.query("SYSTEM SYNC REPLICA ordinary.t2", timeout=10) - assert node2.query("SELECT COUNT() FROM ordinary.t2") == "210\n" - assert node3.query("SELECT COUNT() FROM ordinary.t2") == "210\n" + assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "210") + assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "210") # Total network partition pm.partition_instances(node3, node2) @@ -363,10 +364,10 @@ def test_blocade_leader_twice(started_cluster): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) assert False, "Cannot reconnect in i {} retries".format(i) - assert node1.query("SELECT COUNT() FROM ordinary.t2") == "510\n" + assert_eq_with_retry(node1, "SELECT COUNT() FROM ordinary.t2", "510") if node2.query("SELECT COUNT() FROM ordinary.t2") != "510\n": for num, node in enumerate([node1, node2, node3]): dump_zk(node, '/clickhouse/t2', '/clickhouse/t2/replicas/{}'.format(num + 1)) - assert node2.query("SELECT COUNT() FROM ordinary.t2") == "510\n" - assert node3.query("SELECT COUNT() FROM ordinary.t2") == "510\n" + assert_eq_with_retry(node2, "SELECT COUNT() FROM ordinary.t2", "510") + assert_eq_with_retry(node3, "SELECT COUNT() FROM ordinary.t2", "510") diff --git a/tests/integration/test_testkeeper_multinode_simple/test.py b/tests/integration/test_testkeeper_multinode_simple/test.py index a7ece4bbd56..985915e10a1 100644 --- a/tests/integration/test_testkeeper_multinode_simple/test.py +++ b/tests/integration/test_testkeeper_multinode_simple/test.py @@ -6,6 +6,7 @@ import os import time from multiprocessing.dummy import Pool from helpers.network import PartitionManager +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=['configs/enable_test_keeper1.xml', 'configs/log_conf.xml', 'configs/use_test_keeper.xml'], stay_alive=True) @@ -234,6 +235,6 @@ def test_simple_replicated_table(started_cluster): node1.query("SYSTEM SYNC REPLICA t", timeout=10) node3.query("SYSTEM SYNC REPLICA t", timeout=10) - assert node1.query("SELECT COUNT() FROM t") == "10\n" - assert node2.query("SELECT COUNT() FROM t") == "10\n" - assert node3.query("SELECT COUNT() FROM t") == "10\n" + assert_eq_with_retry(node1, "SELECT COUNT() FROM t", "10") + assert_eq_with_retry(node2, "SELECT COUNT() FROM t", "10") + assert_eq_with_retry(node3, "SELECT COUNT() FROM t", "10") From 6c5781cf34a7a045427b0d1abe68be3254fcb6c6 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 23 Feb 2021 11:48:51 +0300 Subject: [PATCH 333/510] Update fetchPostgreSQLTableStructure.cpp --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 29ed85c003a..740f69667ef 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -60,8 +60,8 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl if (type.ends_with(")")) { res = DataTypeFactory::instance().get(type); - uint32_t precision = getDecimalPrecision(*res); - uint32_t scale = getDecimalScale(*res); + precision = getDecimalPrecision(*res); + scale = getDecimalScale(*res); if (precision <= DecimalUtils::max_precision) res = std::make_shared>(precision, scale); From bf88d693d5b81998b0584731f9fb4a5a1d2d2f08 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 23 Feb 2021 11:59:38 +0300 Subject: [PATCH 334/510] Update fetchPostgreSQLTableStructure.cpp --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index 740f69667ef..e41d3bbe7e3 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -77,7 +77,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl else { precision = DecimalUtils::max_precision(); - scale = precision >> 1; + scale = precision / 2; res = std::make_shared>(precision, scale); } } From 609c35a49c832f51a456c7e17a1356735dd0a48c Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Feb 2021 12:36:26 +0300 Subject: [PATCH 335/510] Added special case in documentation example --- .../aggregate-functions/reference/argmax.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/argmax.md b/docs/en/sql-reference/aggregate-functions/reference/argmax.md index 1af188ad026..72aa607a751 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/argmax.md +++ b/docs/en/sql-reference/aggregate-functions/reference/argmax.md @@ -17,12 +17,12 @@ argMax(arg, val) or ``` sql -argMax(tuple(arg1, arg2), val) +argMax(tuple(arg, val)) ``` **Arguments** -- `arg{i}` — Argument. +- `arg` — Argument. - `val` — Value. **Returned value** @@ -33,7 +33,7 @@ Type: matches `arg` type. For tuple in the input: -- Tuple `(arg1, arg2)`, where `arg1` and `arg2` are the corresponding values. +- Tuple `(arg, val)`, where `val` is the maximum value and `arg` is a corresponding value. Type: [Tuple](../../../sql-reference/data-types/tuple.md). @@ -52,15 +52,15 @@ Input table: Query: ``` sql -SELECT argMax(user, salary), argMax(tuple(user, salary), salary) FROM salary; +SELECT argMax(user, salary), argMax(tuple(user, salary), salary), argMax(tuple(user, salary)) FROM salary; ``` Result: ``` text -┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┐ -│ director │ ('director',5000) │ -└──────────────────────┴─────────────────────────────┘ +┌─argMax(user, salary)─┬─argMax(tuple(user, salary), salary)─┬─argMax(tuple(user, salary))─┐ +│ director │ ('director',5000) │ ('director',5000) │ +└──────────────────────┴─────────────────────────────────────┴─────────────────────────────┘ ``` [Original article](https://clickhouse.tech/docs/en/sql-reference/aggregate-functions/reference/argmax/) From fb932ce290cf19034527204b4b6f17b78824b6a1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Tue, 23 Feb 2021 12:54:38 +0300 Subject: [PATCH 336/510] Updated tests for map type --- .../0_stateless/01550_create_map_type.reference | 2 ++ .../0_stateless/01550_create_map_type.sql | 16 +++++++++++++++- 2 files changed, 17 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01550_create_map_type.reference b/tests/queries/0_stateless/01550_create_map_type.reference index 877199e92d5..d4794ac1459 100644 --- a/tests/queries/0_stateless/01550_create_map_type.reference +++ b/tests/queries/0_stateless/01550_create_map_type.reference @@ -6,6 +6,8 @@ lisi female zhangsan gender +2020-01-01 {1:0,2:1} +2020-01-01 {1:0,2:-1} 1116 1117 1118 diff --git a/tests/queries/0_stateless/01550_create_map_type.sql b/tests/queries/0_stateless/01550_create_map_type.sql index 695a40c42b4..26bbf3c7dde 100644 --- a/tests/queries/0_stateless/01550_create_map_type.sql +++ b/tests/queries/0_stateless/01550_create_map_type.sql @@ -22,7 +22,21 @@ select a[b] from table_map; select b from table_map where a = map('name','lisi', 'gender', 'female'); drop table if exists table_map; --- Int type +-- Big Integer type + +create table table_map (d DATE, m Map(Int8, UInt256)) ENGINE = MergeTree() order by d; +insert into table_map values ('2020-01-01', map(1, 0, 2, 1)); +select * from table_map; +drop table table_map; + +-- Integer type + +create table table_map (d DATE, m Map(Int8, Int8)) ENGINE = MergeTree() order by d; +insert into table_map values ('2020-01-01', map(1, 0, 2, -1)); +select * from table_map; +drop table table_map; + +-- Unsigned Int type drop table if exists table_map; create table table_map(a Map(UInt8, UInt64), b UInt8) Engine = MergeTree() order by b; insert into table_map select map(number, number+5), number from numbers(1111,4); From e123c6ef9dde3c009535f97fff21e36032c3e0c3 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Tue, 23 Feb 2021 18:42:33 +0800 Subject: [PATCH 337/510] Bug fix: Tuple cast to Map --- src/Functions/FunctionsConversion.h | 4 ++-- .../0_stateless/01744_tuple_cast_to_map_bugfix.reference | 3 +++ tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.sql | 3 +++ 3 files changed, 8 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.reference create mode 100644 tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 2e2a4ce9cfa..4889132eeb2 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -2586,7 +2586,7 @@ private: WrapperType createTupleToMapWrapper(const DataTypes & from_kv_types, const DataTypes & to_kv_types) const { return [element_wrappers = getElementWrappers(from_kv_types, to_kv_types), from_kv_types, to_kv_types] - (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t input_rows_count) -> ColumnPtr + (ColumnsWithTypeAndName & arguments, const DataTypePtr &, const ColumnNullable * nullable_source, size_t /*input_rows_count*/) -> ColumnPtr { const auto * col = arguments.front().column.get(); const auto & column_tuple = assert_cast(*col); @@ -2597,7 +2597,7 @@ private: { const auto & column_array = assert_cast(column_tuple.getColumn(i)); ColumnsWithTypeAndName element = {{column_array.getDataPtr(), from_kv_types[i], ""}}; - converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, input_rows_count); + converted_columns[i] = element_wrappers[i](element, to_kv_types[i], nullable_source, (element[0].column)->size()); offsets[i] = column_array.getOffsetsPtr(); } diff --git a/tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.reference b/tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.reference new file mode 100644 index 00000000000..c5143f7c4e9 --- /dev/null +++ b/tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.reference @@ -0,0 +1,3 @@ +{1:'Ready',2:'Steady',3:'Go'} +{1:'Ready',2:'Steady',3:'Go'} +{1:'Ready',2:'Steady',3:'Go'} diff --git a/tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.sql b/tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.sql new file mode 100644 index 00000000000..c5c50c5b039 --- /dev/null +++ b/tests/queries/0_stateless/01744_tuple_cast_to_map_bugfix.sql @@ -0,0 +1,3 @@ +SELECT CAST((['1', '2', '3'], ['Ready', 'Steady', 'Go']), 'Map(UInt8, String)') AS map; +SELECT CAST((['1', '2', '3'], ['Ready', 'Steady', 'Go']), 'Map(UInt8, String)') AS map; +SELECT CAST((['1', '2', '3'], ['Ready', 'Steady', 'Go']), 'Map(UInt8, String)') AS map; From 65549657119bb375135dfa2f92c0059da60d74f8 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 23 Feb 2021 10:46:41 +0000 Subject: [PATCH 338/510] Fix build --- src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp index e41d3bbe7e3..b6128f1bb18 100644 --- a/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp +++ b/src/Databases/PostgreSQL/fetchPostgreSQLTableStructure.cpp @@ -76,7 +76,7 @@ static DataTypePtr convertPostgreSQLDataType(std::string & type, bool is_nullabl } else { - precision = DecimalUtils::max_precision(); + precision = DecimalUtils::max_precision; scale = precision / 2; res = std::make_shared>(precision, scale); } From 07e4cbcc06a2738cbad0be5fa46a1e15589352fa Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 23 Feb 2021 15:08:29 +0300 Subject: [PATCH 339/510] Update docs/ru/sql-reference/functions/tuple-map-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 9516deb243a..e33fc83e67e 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -276,7 +276,7 @@ mapKeys(map) **Возвращаемое значение** -- Массив со всеми значениями `map`. +- Массив со всеми значениями контейнера `map`. Тип: [Array](../../sql-reference/data-types/array.md). From c223a87658088754de9791e8344b644e1b749752 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 23 Feb 2021 15:08:34 +0300 Subject: [PATCH 340/510] Update docs/ru/sql-reference/functions/tuple-map-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index e33fc83e67e..52a6a972f75 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -178,7 +178,7 @@ select mapPopulateSeries([1,2,4], [11,22,44], 5) as res, toTypeName(res) as type ## mapContains {#mapcontains} -Определяет, включает ли в себя `map` параметр `key`. +Определяет, содержит ли контейнер `map` ключ `key`. **Синтаксис** From 9f169aa13e415c8a040f8e0e282ec0b972543669 Mon Sep 17 00:00:00 2001 From: gyuton <40863448+gyuton@users.noreply.github.com> Date: Tue, 23 Feb 2021 15:08:39 +0300 Subject: [PATCH 341/510] Update docs/ru/sql-reference/functions/tuple-map-functions.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/tuple-map-functions.md b/docs/ru/sql-reference/functions/tuple-map-functions.md index 52a6a972f75..696fdb9e5ae 100644 --- a/docs/ru/sql-reference/functions/tuple-map-functions.md +++ b/docs/ru/sql-reference/functions/tuple-map-functions.md @@ -235,7 +235,7 @@ mapKeys(map) **Возвращаемое значение** -- Массив со всеми ключами контейнера `map`. +- Массив со всеми ключами контейнера `map`. Тип: [Array](../../sql-reference/data-types/array.md). From 9e052ae17850b89077f48109ae00e52d85de8cb0 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 23 Feb 2021 15:23:26 +0300 Subject: [PATCH 342/510] Examples moved to section end --- docs/en/interfaces/http.md | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index d82d8baeb75..18533cfc6c2 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -155,8 +155,7 @@ You can use compression to reduce network traffic when transmitting a large amou You can use the internal ClickHouse compression format when transmitting data. The compressed data has a non-standard format, and you need `clickhouse-compressor` program to work with it. It is installed with the `clickhouse-client` package. To increase the efficiency of data insertion, you can disable server-side checksum verification by using the [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress) setting. -If you specify `compress=1` in the URL, the server will compress the data it sends to you. -If you specify `decompress=1` in the URL, the server will decompress the data which you pass in the `POST` method. +If you specify `compress=1` in the URL, the server will compress the data it sends to you. If you specify `decompress=1` in the URL, the server will decompress the data which you pass in the `POST` method. You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). ClickHouse supports the following [compression methods](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens): @@ -165,14 +164,22 @@ You can also choose to use [HTTP compression](https://en.wikipedia.org/wiki/HTTP - `deflate` - `xz` -To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. Example: +To send a compressed `POST` request, append the request header `Content-Encoding: compression_method`. +In order for ClickHouse to compress the response, enable compression with [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting and append `Accept-Encoding: compression_method` header to the request. You can configure the data compression level in the [http_zlib_compression_level](../operations/settings/settings.md#settings-http_zlib_compression_level) setting for all compression methods. + +!!! note "Note" + Some HTTP clients might decompress data from the server by default (with `gzip` and `deflate`) and you might get decompressed data even if you use the compression settings correctly. + +**Examples** + ``` bash +# Sending compressed data to the server $ echo "SELECT 1" | gzip -c | \ curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' ``` -In order for ClickHouse to compress the response, enable compression with [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) setting and append `Accept-Encoding: compression_method` header to the request. You can configure the data compression level in the [http_zlib_compression_level](../operations/settings/settings.md#settings-http_zlib_compression_level) setting for all compression methods. ``` bash +# Receiving compressed data from the server $ curl -vsS "http://localhost:8123/?enable_http_compression=1" \ -H 'Accept-Encoding: gzip' --output result.gz -d 'SELECT number FROM system.numbers LIMIT 3' $ zcat result.gz @@ -181,10 +188,6 @@ $ zcat result.gz 2 ``` -!!! note "Note" - Some HTTP clients might decompress data from the server by default (with `gzip` and `deflate`) and you might get decompressed data even if you use the compression settings correctly. - - ## Default Database {#default-database} You can use the ‘database’ URL parameter or the ‘X-ClickHouse-Database’ header to specify the default database. From 0b82889740df72a8f2ddbbc5ca0e84eda94a2379 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 23 Feb 2021 15:48:22 +0300 Subject: [PATCH 343/510] Translated to ru --- docs/ru/interfaces/http.md | 44 +++++++++++++++++++++++++++----------- 1 file changed, 32 insertions(+), 12 deletions(-) diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 096fb6c05bc..20d23200621 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -149,28 +149,48 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- Для запросов, которые не возвращают таблицу с данными, в случае успеха, выдаётся пустое тело ответа. -Вы можете использовать внутренний формат сжатия Clickhouse при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу `clickhouse-compressor` (устанавливается вместе с пакетом `clickhouse-client`). Для повышения эффективности вставки данных можно отключить проверку контрольной суммы на стороне сервера с помощью настройки[http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress). -Если вы указали `compress = 1` в URL, то сервер сжимает данные, которые он отправляет. -Если вы указали `decompress = 1` в URL, сервер распаковывает те данные, которые вы передаёте методом `POST`. +## Сжатие {#compression} -Также, можно использовать [HTTP compression](https://en.wikipedia.org/wiki/HTTP_compression). Для отправки сжатого запроса `POST`, добавьте заголовок `Content-Encoding: compression_method`. Чтобы ClickHouse сжимал ответ, добавьте заголовок `Accept-Encoding: compression_method`. ClickHouse поддерживает следующие [методы сжатия](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens): `gzip`, `br`, and `deflate`. Чтобы включить HTTP compression, используйте настройку ClickHouse [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression). Уровень сжатия данных для всех методов сжатия можно настроить с помощью настройки [http_zlib_compression_level](#settings-http_zlib_compression_level). +Сжатие можно использовать для уменьшения трафика по сети при передаче большого количества данных, а также для создания сразу сжатых дампов. -Это может быть использовано для уменьшения трафика по сети при передаче большого количества данных, а также для создания сразу сжатых дампов. +Вы можете использовать внутренний формат сжатия Clickhouse при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу `clickhouse-compressor`. Она устанавливается вместе с пакетом `clickhouse-client`. Для повышения эффективности вставки данных можно отключить проверку контрольной суммы на стороне сервера с помощью настройки [http_native_compression_disable_checksumming_on_decompress](../operations/settings/settings.md#settings-http_native_compression_disable_checksumming_on_decompress). -Примеры отправки данных со сжатием: +Если вы указали `compress=1` в URL, то сервер сжимает данные, которые он отправляет. Если вы указали `decompress=1` в URL, сервер распаковывает те данные, которые вы передаёте методом `POST`. -``` bash -$ #Отправка данных на сервер: -$ curl -vsS "http://localhost:8123/?enable_http_compression=1" -d 'SELECT number FROM system.numbers LIMIT 10' -H 'Accept-Encoding: gzip' +Также можно использовать [сжатие HTTP](https://en.wikipedia.org/wiki/HTTP_compression). ClickHouse поддерживает следующие [методы сжатия](https://en.wikipedia.org/wiki/HTTP_compression#Content-Encoding_tokens): -$ #Отправка данных клиенту: -$ echo "SELECT 1" | gzip -c | curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' -``` +- `gzip` +- `br` +- `deflate` +- `xz` + +Для отправки сжатого запроса `POST`, добавьте заголовок `Content-Encoding: compression_method`. +Чтобы ClickHouse сжимал ответ, разрешите сжатие настройкой [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) и добавьте заголовок `Accept-Encoding: compression_method`. Уровень сжатия данных для всех методов сжатия можно задать с помощью настройки [http_zlib_compression_level](#settings-http_zlib_compression_level). !!! note "Примечание" Некоторые HTTP-клиенты могут по умолчанию распаковывать данные (`gzip` и `deflate`) с сервера в фоновом режиме и вы можете получить распакованные данные, даже если правильно используете настройки сжатия. +**Примеры** + +``` bash +# Отправка сжатых данных на сервер +$ echo "SELECT 1" | gzip -c | \ + curl -sS --data-binary @- -H 'Content-Encoding: gzip' 'http://localhost:8123/' +``` + +``` bash +# Получение сжатых данных с сервера +$ curl -vsS "http://localhost:8123/?enable_http_compression=1" \ + -H 'Accept-Encoding: gzip' --output result.gz -d 'SELECT number FROM system.numbers LIMIT 3' +$ zcat result.gz +0 +1 +2 +``` + +## База данных по умолчанию {#default-database} + Вы можете использовать параметр URL `database` или заголовок `X-ClickHouse-Database`, чтобы указать БД по умолчанию. ``` bash From bb2ee9609f0a950d5351947f5c01a6de813b880d Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 23 Feb 2021 16:43:40 +0300 Subject: [PATCH 344/510] Compression mentioned in File engine --- docs/en/engines/table-engines/special/file.md | 3 ++- docs/ru/engines/table-engines/special/file.md | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/special/file.md b/docs/en/engines/table-engines/special/file.md index cda16c2a226..2acec40ef02 100644 --- a/docs/en/engines/table-engines/special/file.md +++ b/docs/en/engines/table-engines/special/file.md @@ -66,7 +66,8 @@ SELECT * FROM file_engine_table ## Usage in ClickHouse-local {#usage-in-clickhouse-local} -In [clickhouse-local](../../../operations/utilities/clickhouse-local.md) File engine accepts file path in addition to `Format`. Default input/output streams can be specified using numeric or human-readable names like `0` or `stdin`, `1` or `stdout`. +In [clickhouse-local](../../../operations/utilities/clickhouse-local.md) File engine accepts file path in addition to `Format`. Default input/output streams can be specified using numeric or human-readable names like `0` or `stdin`, `1` or `stdout`. It is possible to read and write compressed files based on an additional engine parameter or file extension (`gz`, `br` or `xz`). + **Example:** ``` bash diff --git a/docs/ru/engines/table-engines/special/file.md b/docs/ru/engines/table-engines/special/file.md index 6a55ef31732..9be09fd33e6 100644 --- a/docs/ru/engines/table-engines/special/file.md +++ b/docs/ru/engines/table-engines/special/file.md @@ -63,7 +63,7 @@ SELECT * FROM file_engine_table ## Использование движка в Clickhouse-local {#ispolzovanie-dvizhka-v-clickhouse-local} -В [clickhouse-local](../../../engines/table-engines/special/file.md) движок в качестве параметра принимает не только формат, но и путь к файлу. В том числе можно указать стандартные потоки ввода/вывода цифровым или буквенным обозначением `0` или `stdin`, `1` или `stdout`. +В [clickhouse-local](../../../engines/table-engines/special/file.md) движок в качестве параметра принимает не только формат, но и путь к файлу. В том числе можно указать стандартные потоки ввода/вывода цифровым или буквенным обозначением `0` или `stdin`, `1` или `stdout`. Можно записывать и читать сжатые файлы. Для этого нужно задать дополнительный параметр движка или расширение файла (`gz`, `br` или `xz`). **Пример:** From 7ee449282355d2ae444ea4d5aec413a343f71580 Mon Sep 17 00:00:00 2001 From: lehasm Date: Tue, 23 Feb 2021 17:24:15 +0300 Subject: [PATCH 345/510] Fixed link --- docs/ru/interfaces/http.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index 20d23200621..d6f930f3f63 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -166,7 +166,7 @@ $ echo 'DROP TABLE t' | curl 'http://localhost:8123/' --data-binary @- - `xz` Для отправки сжатого запроса `POST`, добавьте заголовок `Content-Encoding: compression_method`. -Чтобы ClickHouse сжимал ответ, разрешите сжатие настройкой [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) и добавьте заголовок `Accept-Encoding: compression_method`. Уровень сжатия данных для всех методов сжатия можно задать с помощью настройки [http_zlib_compression_level](#settings-http_zlib_compression_level). +Чтобы ClickHouse сжимал ответ, разрешите сжатие настройкой [enable_http_compression](../operations/settings/settings.md#settings-enable_http_compression) и добавьте заголовок `Accept-Encoding: compression_method`. Уровень сжатия данных для всех методов сжатия можно задать с помощью настройки [http_zlib_compression_level](../operations/settings/settings.md#settings-http_zlib_compression_level). !!! note "Примечание" Некоторые HTTP-клиенты могут по умолчанию распаковывать данные (`gzip` и `deflate`) с сервера в фоновом режиме и вы можете получить распакованные данные, даже если правильно используете настройки сжатия. From acfbae7107897d3557247317328ceb11cf6b2aed Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 23 Feb 2021 17:00:22 +0800 Subject: [PATCH 346/510] Fix replace,All functions so that they don't generate garbage to stderr --- src/Functions/ReplaceRegexpImpl.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/Functions/ReplaceRegexpImpl.h b/src/Functions/ReplaceRegexpImpl.h index d9531cb343d..3e80dd5b337 100644 --- a/src/Functions/ReplaceRegexpImpl.h +++ b/src/Functions/ReplaceRegexpImpl.h @@ -164,7 +164,10 @@ struct ReplaceRegexpImpl size_t size = offsets.size(); res_offsets.resize(size); - re2_st::RE2 searcher(needle); + typename re2_st::RE2::Options regexp_options; + /// Never write error messages to stderr. It's ignorant to do it from library code. + regexp_options.set_log_errors(false); + re2_st::RE2 searcher(needle, regexp_options); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, static_cast(max_captures)); Instructions instructions = createInstructions(replacement, num_captures); @@ -193,7 +196,10 @@ struct ReplaceRegexpImpl res_data.reserve(data.size()); res_offsets.resize(size); - re2_st::RE2 searcher(needle); + typename re2_st::RE2::Options regexp_options; + /// Never write error messages to stderr. It's ignorant to do it from library code. + regexp_options.set_log_errors(false); + re2_st::RE2 searcher(needle, regexp_options); int num_captures = std::min(searcher.NumberOfCapturingGroups() + 1, static_cast(max_captures)); Instructions instructions = createInstructions(replacement, num_captures); From e4fdbbfab6f9b8f6bc4f224f4e162657264894d8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Feb 2021 19:27:44 +0300 Subject: [PATCH 347/510] Add test race_condition_storage.sh --- .../01731_race_condition_storage.reference | 0 .../01731_race_condition_storage.sh | 50 +++++++++++++++++++ 2 files changed, 50 insertions(+) create mode 100644 tests/queries/0_stateless/01731_race_condition_storage.reference create mode 100755 tests/queries/0_stateless/01731_race_condition_storage.sh diff --git a/tests/queries/0_stateless/01731_race_condition_storage.reference b/tests/queries/0_stateless/01731_race_condition_storage.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01731_race_condition_storage.sh b/tests/queries/0_stateless/01731_race_condition_storage.sh new file mode 100755 index 00000000000..0ce6f2b0c7a --- /dev/null +++ b/tests/queries/0_stateless/01731_race_condition_storage.sh @@ -0,0 +1,50 @@ +#!/usr/bin/env bash + +unset CLICKHOUSE_LOG_COMMENT + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -o errexit +set -o pipefail + +echo " + DROP TABLE IF EXISTS storage_join_race; + CREATE TABLE storage_join_race (x UInt64, y UInt64) Engine = Join(ALL, FULL, x); +" | $CLICKHOUSE_CLIENT -n + +function read_thread_big() +{ + while true; do + echo " + SELECT * FROM ( SELECT number AS x FROM numbers(100000) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done +} + +function read_thread_small() +{ + while true; do + echo " + SELECT * FROM ( SELECT number AS x FROM numbers(10) ) AS t1 ALL FULL JOIN storage_join_race USING (x) FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done +} + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f read_thread_big; +export -f read_thread_small; + +TIMEOUT=20 + +timeout $TIMEOUT bash -c read_thread_big 2> /dev/null & +timeout $TIMEOUT bash -c read_thread_small 2> /dev/null & + +echo " + INSERT INTO storage_join_race SELECT number AS x, number AS y FROM numbers (10000000); +" | $CLICKHOUSE_CLIENT -n + +wait + +$CLICKHOUSE_CLIENT -q "DROP TABLE storage_join_race" From 8ab578065a5ac9864851aa077204ac1b3d81997d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 22 Feb 2021 23:52:16 +0300 Subject: [PATCH 348/510] Fix __has_feature macros under gcc __has_feature supported only by clang. But libcxx/libcxxabi overrides it to 0: $ fgrep -r 'define __has_feature' contrib/libcxx* contrib/libcxx/include/__config:#define __has_feature(__x) 0 contrib/libcxxabi/src/demangle/DemangleConfig.h:#define __has_feature(x) 0 Thus the checks for __has_feature will be wrong, undefine it again to avoid such issues. This will also fix building with sanitizers under gcc (before this patch BOOST_USE_UCONTEXT wasn't set for sanitizers). --- base/common/defines.h | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/base/common/defines.h b/base/common/defines.h index 845a53179ef..54567dabc90 100644 --- a/base/common/defines.h +++ b/base/common/defines.h @@ -1,5 +1,13 @@ #pragma once +/// __has_feature supported only by clang. +/// +/// But libcxx/libcxxabi overrides it to 0, thus the checks for __has_feature will be wrong, +/// undefine it again to avoid such issues. +#if defined(__has_feature) && !defined(__clang__) +# undef __has_feature +#endif + #if defined(_MSC_VER) # if !defined(likely) # define likely(x) (x) From 810023a65cdb578f94b2529f3cb1059f13182eda Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 23 Feb 2021 07:34:58 +0300 Subject: [PATCH 349/510] Do not undefine __has_feature (may break c++ headers) If the order of and c++ header will be wrong the compilation will be broken. v2: rename __ch_has_feature to ch_has_feature to fix -Wreserved-id-macro v3: do not fallback to 0 --- base/common/defines.h | 27 +++++++++++++++++---------- base/common/phdr_cache.cpp | 10 +++++----- src/Common/MemorySanitizer.h | 19 +++++++++++-------- src/Common/SymbolIndex.cpp | 10 +++++----- 4 files changed, 38 insertions(+), 28 deletions(-) diff --git a/base/common/defines.h b/base/common/defines.h index 54567dabc90..367bdd64234 100644 --- a/base/common/defines.h +++ b/base/common/defines.h @@ -2,10 +2,17 @@ /// __has_feature supported only by clang. /// -/// But libcxx/libcxxabi overrides it to 0, thus the checks for __has_feature will be wrong, -/// undefine it again to avoid such issues. -#if defined(__has_feature) && !defined(__clang__) -# undef __has_feature +/// But libcxx/libcxxabi overrides it to 0, +/// thus the checks for __has_feature will be wrong. +/// +/// NOTE: +/// - __has_feature cannot be simply undefined, +/// since this will be broken if some C++ header will be included after +/// including +/// - it should not have fallback to 0, +/// since this may create false-positive detection (common problem) +#if defined(__clang__) && defined(__has_feature) +# define ch_has_feature __has_feature #endif #if defined(_MSC_VER) @@ -40,8 +47,8 @@ /// Check for presence of address sanitizer #if !defined(ADDRESS_SANITIZER) -# if defined(__has_feature) -# if __has_feature(address_sanitizer) +# if defined(ch_has_feature) +# if ch_has_feature(address_sanitizer) # define ADDRESS_SANITIZER 1 # endif # elif defined(__SANITIZE_ADDRESS__) @@ -50,8 +57,8 @@ #endif #if !defined(THREAD_SANITIZER) -# if defined(__has_feature) -# if __has_feature(thread_sanitizer) +# if defined(ch_has_feature) +# if ch_has_feature(thread_sanitizer) # define THREAD_SANITIZER 1 # endif # elif defined(__SANITIZE_THREAD__) @@ -60,8 +67,8 @@ #endif #if !defined(MEMORY_SANITIZER) -# if defined(__has_feature) -# if __has_feature(memory_sanitizer) +# if defined(ch_has_feature) +# if ch_has_feature(memory_sanitizer) # define MEMORY_SANITIZER 1 # endif # elif defined(__MEMORY_SANITIZER__) diff --git a/base/common/phdr_cache.cpp b/base/common/phdr_cache.cpp index 4f6a066adab..49d566dac19 100644 --- a/base/common/phdr_cache.cpp +++ b/base/common/phdr_cache.cpp @@ -15,11 +15,11 @@ #endif #define __msan_unpoison(X, Y) // NOLINT -#if defined(__has_feature) -# if __has_feature(memory_sanitizer) -# undef __msan_unpoison -# include -# endif +#if defined(ch_has_feature) +# if ch_has_feature(memory_sanitizer) +# undef __msan_unpoison +# include +# endif #endif #include diff --git a/src/Common/MemorySanitizer.h b/src/Common/MemorySanitizer.h index 54a92ea3a19..9e34e454090 100644 --- a/src/Common/MemorySanitizer.h +++ b/src/Common/MemorySanitizer.h @@ -1,5 +1,7 @@ #pragma once +#include + #ifdef __clang__ #pragma clang diagnostic push #pragma clang diagnostic ignored "-Wreserved-id-macro" @@ -9,14 +11,15 @@ #define __msan_test_shadow(X, Y) (false) #define __msan_print_shadow(X, Y) #define __msan_unpoison_string(X) -#if defined(__has_feature) -# if __has_feature(memory_sanitizer) -# undef __msan_unpoison -# undef __msan_test_shadow -# undef __msan_print_shadow -# undef __msan_unpoison_string -# include -# endif + +#if defined(ch_has_feature) +# if ch_has_feature(memory_sanitizer) +# undef __msan_unpoison +# undef __msan_test_shadow +# undef __msan_print_shadow +# undef __msan_unpoison_string +# include +# endif #endif #ifdef __clang__ diff --git a/src/Common/SymbolIndex.cpp b/src/Common/SymbolIndex.cpp index bd6c5d9eda0..a23184c9c0a 100644 --- a/src/Common/SymbolIndex.cpp +++ b/src/Common/SymbolIndex.cpp @@ -60,11 +60,11 @@ Otherwise you will get only exported symbols from program headers. #endif #define __msan_unpoison_string(X) // NOLINT -#if defined(__has_feature) -# if __has_feature(memory_sanitizer) -# undef __msan_unpoison_string -# include -# endif +#if defined(ch_has_feature) +# if ch_has_feature(memory_sanitizer) +# undef __msan_unpoison_string +# include +# endif #endif From 2d31019addca8981fbab107d289f84b28a1da570 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 23 Feb 2021 16:54:50 +0000 Subject: [PATCH 350/510] Update .reference file --- .../0_stateless/01656_test_query_log_factories_info.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference index c80d31a343a..af7feae5a38 100644 --- a/tests/queries/0_stateless/01656_test_query_log_factories_info.reference +++ b/tests/queries/0_stateless/01656_test_query_log_factories_info.reference @@ -11,7 +11,7 @@ arraySort(used_table_functions) ['numbers'] arraySort(used_functions) -['CAST', 'addDays','array','arrayFlatten','crc32','modulo','plus','pow','round','substring','tanh','toDate','toDayOfYear','toTypeName','toWeek'] +['CAST','CRC32','addDays','array','arrayFlatten','modulo','plus','pow','round','substring','tanh','toDate','toDayOfYear','toTypeName','toWeek'] arraySort(used_data_type_families) ['Array','Int32','Nullable','String'] From b0065334e246433f09ba2cf755197dace17b0dcf Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Feb 2021 00:27:10 +0300 Subject: [PATCH 351/510] Summing raft test more stable --- src/Coordination/tests/gtest_for_build.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp index dcd4e718f3d..c064fcdbef4 100644 --- a/src/Coordination/tests/gtest_for_build.cpp +++ b/src/Coordination/tests/gtest_for_build.cpp @@ -247,8 +247,11 @@ TEST(CoordinationTest, TestSummingRaft3) std::cerr << "Starting to add entries\n"; auto entry = getBuffer(1); auto ret = s2.raft_instance->append_entries({entry}); - EXPECT_TRUE(ret->get_accepted()) << "failed to replicate: entry 1" << ret->get_result_code(); - EXPECT_EQ(ret->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 1" << ret->get_result_code(); + while (!ret->get_accepted() || ret->get_result_code() != nuraft::cmd_result_code::OK) + { + std::cerr << ret->get_accepted() << "failed to replicate: entry 1" << ret->get_result_code() << std::endl; + ret = s2.raft_instance->append_entries({entry}); + } while (s1.state_machine->getValue() != 1) { @@ -283,8 +286,11 @@ TEST(CoordinationTest, TestSummingRaft3) auto leader_entry = getBuffer(77); auto ret_leader = s2.raft_instance->append_entries({leader_entry}); - EXPECT_TRUE(ret_leader->get_accepted()) << "failed to replicate: entry 78" << ret_leader->get_result_code(); - EXPECT_EQ(ret_leader->get_result_code(), nuraft::cmd_result_code::OK) << "failed to replicate: entry 78" << ret_leader->get_result_code(); + while (!ret_leader->get_accepted() || ret_leader->get_result_code() != nuraft::cmd_result_code::OK) + { + std::cerr << "failed to replicate: entry 78" << ret_leader->get_result_code() << std::endl; + ret_leader = s2.raft_instance->append_entries({leader_entry}); + } while (s1.state_machine->getValue() != 78) { From 613854df0d167ba0e0c52ece857d4eeb16caa966 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 23 Feb 2021 17:43:52 -0400 Subject: [PATCH 352/510] Update settings.md fix hyperlink --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 6440f09bb40..ae43ec77056 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1104,7 +1104,7 @@ The maximum number of replicas for each shard when executing a query. In limited - the sampling key is an expression that is expensive to calculate - the cluster's latency distribution has a long tail, so that querying more servers increases the query's overall latency -In addition, this setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain conditions. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md/#max_parallel_replica-subqueries) for more details. +In addition, this setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain conditions. See [Distributed Subqueries and max_parallel_replicas](../../../sql-reference/operators/in.md#max_parallel_replica-subqueries) for more details. ## compile {#compile} From c25ea35099793d001255f4cbcc87cf288eb7c796 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Tue, 23 Feb 2021 18:22:47 -0400 Subject: [PATCH 353/510] Update settings.md --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index ae43ec77056..d94b69508f3 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1104,7 +1104,7 @@ The maximum number of replicas for each shard when executing a query. In limited - the sampling key is an expression that is expensive to calculate - the cluster's latency distribution has a long tail, so that querying more servers increases the query's overall latency -In addition, this setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain conditions. See [Distributed Subqueries and max_parallel_replicas](../../../sql-reference/operators/in.md#max_parallel_replica-subqueries) for more details. +In addition, this setting will produce incorrect results when joins or subqueries are involved, and all tables don't meet certain conditions. See [Distributed Subqueries and max_parallel_replicas](../../sql-reference/operators/in.md#max_parallel_replica-subqueries) for more details. ## compile {#compile} From 5628fd6da55367717fa4792bdb00f5b6ff541650 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 24 Feb 2021 02:22:07 +0300 Subject: [PATCH 354/510] Small update --- src/Client/Connection.cpp | 1 - src/Client/Connection.h | 4 +--- src/Client/ConnectionEstablisher.cpp | 1 - 3 files changed, 1 insertion(+), 5 deletions(-) diff --git a/src/Client/Connection.cpp b/src/Client/Connection.cpp index 6f8fc3d12f2..80d44a336a5 100644 --- a/src/Client/Connection.cpp +++ b/src/Client/Connection.cpp @@ -110,7 +110,6 @@ void Connection::connect(const ConnectionTimeouts & timeouts) in = std::make_shared(*socket); in->setAsyncCallback(std::move(async_callback)); - async_callback = {}; out = std::make_shared(*socket); diff --git a/src/Client/Connection.h b/src/Client/Connection.h index 95fbb902c2c..bff1e391129 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -27,7 +27,6 @@ #include #include - namespace DB { @@ -199,10 +198,9 @@ public: /// Each time read from socket blocks and async_callback is set, it will be called. You can poll socket inside it. void setAsyncCallback(AsyncCallback async_callback_) { + async_callback = std::move(async_callback_); if (in) in->setAsyncCallback(std::move(async_callback_)); - else - async_callback = std::move(async_callback_); } private: diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index b3bb3c3a408..17b8832aab4 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -36,7 +36,6 @@ void ConnectionEstablisher::run(ConnectionEstablisher::TryResult & result, std:: { result.entry = pool->get(*timeouts, settings, /* force_connected = */ false); AsyncCallbackSetter async_setter(&*result.entry, std::move(async_callback)); - async_callback = {}; UInt64 server_revision = 0; if (table_to_check) From 0f7f2a4962d1b78d15b773ad3afbdd217993d49f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 Feb 2021 11:35:45 +0800 Subject: [PATCH 355/510] Support LC type for joinGet --- src/Interpreters/HashJoin.cpp | 4 ++-- .../0_stateless/01735_join_get_low_card_fix.reference | 1 + .../queries/0_stateless/01735_join_get_low_card_fix.sql | 9 +++++++++ 3 files changed, 12 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/01735_join_get_low_card_fix.reference create mode 100644 tests/queries/0_stateless/01735_join_get_low_card_fix.sql diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 5c50b53e2ca..74ad2877d9e 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1232,8 +1232,8 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, { const auto & left_type_origin = data_types[i]; const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); - auto left_type = removeNullable(left_type_origin); - auto right_type = removeNullable(right_type_origin); + auto left_type = removeLowCardinality(removeNullable(left_type_origin)); + auto right_type = removeLowCardinality(removeNullable(right_type_origin)); if (!left_type->equals(*right_type)) throw Exception( "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " diff --git a/tests/queries/0_stateless/01735_join_get_low_card_fix.reference b/tests/queries/0_stateless/01735_join_get_low_card_fix.reference new file mode 100644 index 00000000000..0b20aead00e --- /dev/null +++ b/tests/queries/0_stateless/01735_join_get_low_card_fix.reference @@ -0,0 +1 @@ +yyy diff --git a/tests/queries/0_stateless/01735_join_get_low_card_fix.sql b/tests/queries/0_stateless/01735_join_get_low_card_fix.sql new file mode 100644 index 00000000000..bdc979bc11e --- /dev/null +++ b/tests/queries/0_stateless/01735_join_get_low_card_fix.sql @@ -0,0 +1,9 @@ +drop table if exists join_tbl; + +create table join_tbl (`id` String, `name` String) engine Join(any, left, id); + +insert into join_tbl values ('xxx', 'yyy'); + +select joinGet('join_tbl', 'name', toLowCardinality('xxx')); + +drop table if exists join_tbl; From abf26d4935a831ba663554377f3ac5967ec68795 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 Feb 2021 14:45:14 +0800 Subject: [PATCH 356/510] disable null_as_default when type is Nullable --- .../Formats/Impl/ValuesBlockInputFormat.cpp | 12 +++++++++--- .../0_stateless/01736_null_as_default.reference | 2 ++ tests/queries/0_stateless/01736_null_as_default.sql | 5 +++++ 3 files changed, 16 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/01736_null_as_default.reference create mode 100644 tests/queries/0_stateless/01736_null_as_default.sql diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 34a4a98f16b..7860498e4ff 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -255,9 +255,15 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx bool found_in_cache = false; const auto & result_type = header.getByPosition(column_idx).type; const char * delimiter = (column_idx + 1 == num_columns) ? ")" : ","; - auto structure = templates_cache.getFromCacheOrConstruct(result_type, format_settings.null_as_default, - TokenIterator(tokens), token_iterator, - ast, *context, &found_in_cache, delimiter); + auto structure = templates_cache.getFromCacheOrConstruct( + result_type, + !result_type->isNullable() && format_settings.null_as_default, + TokenIterator(tokens), + token_iterator, + ast, + *context, + &found_in_cache, + delimiter); templates[column_idx].emplace(structure); if (found_in_cache) ++attempts_to_deduce_template_cached[column_idx]; diff --git a/tests/queries/0_stateless/01736_null_as_default.reference b/tests/queries/0_stateless/01736_null_as_default.reference new file mode 100644 index 00000000000..baf83eb21d7 --- /dev/null +++ b/tests/queries/0_stateless/01736_null_as_default.reference @@ -0,0 +1,2 @@ +A +\N diff --git a/tests/queries/0_stateless/01736_null_as_default.sql b/tests/queries/0_stateless/01736_null_as_default.sql new file mode 100644 index 00000000000..f9a4bc69acf --- /dev/null +++ b/tests/queries/0_stateless/01736_null_as_default.sql @@ -0,0 +1,5 @@ +drop table if exists test_num; +create table test_enum (c Nullable(Enum16('A' = 1, 'B' = 2))) engine Log; +insert into test_enum values (1), (NULL); +select * from test_enum; +drop table if exists test_num; From c006b41afcb79e978b1977ac284836e7fef51ab1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 09:48:12 +0300 Subject: [PATCH 357/510] Lock before reuseJoinedData --- src/Storages/StorageJoin.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index f130316566f..db80f0adb17 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -96,8 +96,8 @@ HashJoinPtr StorageJoin::getJoin(std::shared_ptr analyzed_join) const analyzed_join->setRightKeys(key_names); HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns()); - join_clone->reuseJoinedData(*join); join_clone->setLock(rwlock); + join_clone->reuseJoinedData(*join); return join_clone; } From 2cb273d4368545395a9d40ae783f61da9c80ec19 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 24 Feb 2021 16:44:51 +0800 Subject: [PATCH 358/510] Fix --- src/Interpreters/HashJoin.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 74ad2877d9e..142ab07927f 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1232,8 +1232,8 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, { const auto & left_type_origin = data_types[i]; const auto & [c2, right_type_origin, right_name] = right_table_keys.safeGetByPosition(i); - auto left_type = removeLowCardinality(removeNullable(left_type_origin)); - auto right_type = removeLowCardinality(removeNullable(right_type_origin)); + auto left_type = removeNullable(recursiveRemoveLowCardinality(left_type_origin)); + auto right_type = removeNullable(recursiveRemoveLowCardinality(right_type_origin)); if (!left_type->equals(*right_type)) throw Exception( "Type mismatch in joinGet key " + toString(i) + ": found type " + left_type->getName() + ", while the needed type is " From deb566a40b9cafc7195d57acab8c59da16ab7014 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Wed, 24 Feb 2021 11:48:04 +0300 Subject: [PATCH 359/510] Resorted from std::random_device to std::rand() --- base/mysqlxx/PoolWithFailover.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 9132773f727..4b77b41d251 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -10,8 +11,6 @@ static bool startsWith(const std::string & s, const char * prefix) return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix)); } -/// This reads from "/dev/urandom" and thus is thread-safe -std::random_device rd; using namespace mysqlxx; @@ -42,9 +41,9 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & con /// PoolWithFailover objects are stored in a cache inside PoolFactory. /// This cache is reset by ExternalDictionariesLoader after every SYSTEM RELOAD DICTIONAR{Y|IES} /// which triggers massive re-constructing of connection pools. - /// The state of PRNDGs like std::mt19937 is considered to be quite heavy + /// The state of PRNGs like std::mt19937 is considered to be quite heavy /// thus here we attempt to optimize its construction. - static thread_local std::mt19937 rnd_generator(rd()); + static thread_local std::mt19937 rnd_generator(std::rand()); for (auto & [_, replicas] : replicas_by_priority) { if (replicas.size() > 1) From 96f22b4290257f9d9913783167e476349cb7bb7f Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 24 Feb 2021 10:41:08 +0000 Subject: [PATCH 360/510] Fix rabbit bug --- .../WriteBufferToRabbitMQProducer.cpp | 23 ++++++++++++------- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 1 + 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index ac1b253b4bb..a9b29236bb4 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -189,11 +189,12 @@ void WriteBufferToRabbitMQProducer::setupChannel() /// Delivery tags are scoped per channel. delivery_record.clear(); delivery_tag = 0; + producer_ready = false; }); producer_channel->onReady([&]() { - channel_id = channel_id_base + std::to_string(channel_id_counter++); + channel_id = channel_id_base + "_" + std::to_string(channel_id_counter++); LOG_DEBUG(log, "Producer's channel {} is ready", channel_id); /* if persistent == true, onAck is received when message is persisted to disk or when it is consumed on every queue. If fails, @@ -211,6 +212,7 @@ void WriteBufferToRabbitMQProducer::setupChannel() { removeRecord(nacked_delivery_tag, multiple, true); }); + producer_ready = true; }); } @@ -308,13 +310,18 @@ void WriteBufferToRabbitMQProducer::writingFunc() { while ((!payloads.empty() || wait_all) && wait_confirm.load()) { - /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished - * as fast as possible and no new publishes are made before returned messages are handled - */ - if (!returned.empty() && producer_channel->usable()) - publish(returned, true); - else if (!payloads.empty() && producer_channel->usable()) - publish(payloads, false); + /// If onReady callback is not received, producer->usable() will anyway return true, + /// but must publish only after onReady callback. + if (producer_ready) + { + /* Publish main paylods only when there are no returned messages. This way it is ensured that returned messages are republished + * as fast as possible and no new publishes are made before returned messages are handled + */ + if (!returned.empty() && producer_channel->usable()) + publish(returned, true); + else if (!payloads.empty() && producer_channel->usable()) + publish(payloads, false); + } iterateEventLoop(); diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index e88f92239ca..c2ca98de610 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -73,6 +73,7 @@ private: std::unique_ptr event_handler; std::unique_ptr connection; std::unique_ptr producer_channel; + bool producer_ready = false; /// Channel errors lead to channel closure, need to count number of recreated channels to update channel id UInt64 channel_id_counter = 0; From a1901383ae03ade91e79ed15699ecf6b88d38cb1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Feb 2021 14:54:14 +0300 Subject: [PATCH 361/510] Improve performance and fix condition variable wait --- contrib/NuRaft | 2 +- src/Coordination/NuKeeperStateMachine.cpp | 2 +- src/Server/NuKeeperTCPHandler.cpp | 166 ++++++++++------------ 3 files changed, 81 insertions(+), 89 deletions(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index c250d5ad58c..3af9f53d9e2 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit c250d5ad58c82e751264df40a94da682a2fc3519 +Subproject commit 3af9f53d9e2e36f0c975a4cd665b446b875c4dde diff --git a/src/Coordination/NuKeeperStateMachine.cpp b/src/Coordination/NuKeeperStateMachine.cpp index 0061645c75c..33f15fca09c 100644 --- a/src/Coordination/NuKeeperStateMachine.cpp +++ b/src/Coordination/NuKeeperStateMachine.cpp @@ -46,7 +46,7 @@ NuKeeperStateMachine::NuKeeperStateMachine(ResponsesQueue & responses_queue_, co , storage(coordination_settings->dead_session_check_period_ms.totalMilliseconds()) , responses_queue(responses_queue_) , last_committed_idx(0) - , log(&Poco::Logger::get("NuRaftStateMachine")) + , log(&Poco::Logger::get("NuKeeperStateMachine")) { LOG_DEBUG(log, "Created nukeeper state machine"); } diff --git a/src/Server/NuKeeperTCPHandler.cpp b/src/Server/NuKeeperTCPHandler.cpp index 92c7f4b968f..cc8f8f2ff2d 100644 --- a/src/Server/NuKeeperTCPHandler.cpp +++ b/src/Server/NuKeeperTCPHandler.cpp @@ -40,7 +40,7 @@ namespace ErrorCodes struct PollResult { - bool has_response{false}; + size_t responses_count{0}; bool has_requests{false}; bool error{false}; }; @@ -94,101 +94,90 @@ struct SocketInterruptablePollWrapper PollResult poll(Poco::Timespan remaining_time, const std::shared_ptr & in) { - PollResult result{}; - if (response_in.available() != 0) - { - UInt8 dummy; - readIntBinary(dummy, response_in); - result.has_response = true; - } - - if (in->available() != 0) - result.has_requests = true; - - if (result.has_response) - return result; bool socket_ready = false; bool fd_ready = false; -#if defined(POCO_HAVE_FD_EPOLL) - int rc; - epoll_event evout[2]; - evout[0].data.fd = evout[1].data.fd = -1; - do - { - Poco::Timestamp start; - rc = epoll_wait(epollfd, evout, 2, remaining_time.totalMilliseconds()); - if (rc < 0 && errno == EINTR) - { - Poco::Timestamp end; - Poco::Timespan waited = end - start; - if (waited < remaining_time) - remaining_time -= waited; - else - remaining_time = 0; - } - } - while (rc < 0 && errno == EINTR); - for (int i = 0; i < rc; ++i) - { - if (evout[i].data.fd == sockfd) - socket_ready = true; - if (evout[i].data.fd == pipe.fds_rw[0]) - fd_ready = true; - } -#else - pollfd poll_buf[2]; - poll_buf[0].fd = sockfd; - poll_buf[0].events = POLLIN; - poll_buf[1].fd = pipe.fds_rw[0]; - poll_buf[1].events = POLLIN; - - int rc; - do - { - Poco::Timestamp start; - rc = ::poll(poll_buf, 2, remaining_time.totalMilliseconds()); - if (rc < 0 && errno == POCO_EINTR) - { - Poco::Timestamp end; - Poco::Timespan waited = end - start; - if (waited < remaining_time) - remaining_time -= waited; - else - remaining_time = 0; - } - } - while (rc < 0 && errno == POCO_EINTR); - - if (rc >= 1 && poll_buf[0].revents & POLLIN) + if (in->available() != 0) socket_ready = true; - if (rc == 2 && poll_buf[1].revents & POLLIN) + + if (response_in.available() != 0) fd_ready = true; + + int rc = 0; + if (!fd_ready) + { +#if defined(POCO_HAVE_FD_EPOLL) + epoll_event evout[2]; + evout[0].data.fd = evout[1].data.fd = -1; + do + { + Poco::Timestamp start; + rc = epoll_wait(epollfd, evout, 2, remaining_time.totalMilliseconds()); + if (rc < 0 && errno == EINTR) + { + Poco::Timestamp end; + Poco::Timespan waited = end - start; + if (waited < remaining_time) + remaining_time -= waited; + else + remaining_time = 0; + } + } + while (rc < 0 && errno == EINTR); + + for (int i = 0; i < rc; ++i) + { + if (evout[i].data.fd == sockfd) + socket_ready = true; + if (evout[i].data.fd == pipe.fds_rw[0]) + fd_ready = true; + } +#else + pollfd poll_buf[2]; + poll_buf[0].fd = sockfd; + poll_buf[0].events = POLLIN; + poll_buf[1].fd = pipe.fds_rw[0]; + poll_buf[1].events = POLLIN; + + do + { + Poco::Timestamp start; + rc = ::poll(poll_buf, 2, remaining_time.totalMilliseconds()); + if (rc < 0 && errno == POCO_EINTR) + { + Poco::Timestamp end; + Poco::Timespan waited = end - start; + if (waited < remaining_time) + remaining_time -= waited; + else + remaining_time = 0; + } + } + while (rc < 0 && errno == POCO_EINTR); + + if (rc >= 1 && poll_buf[0].revents & POLLIN) + socket_ready = true; + if (rc == 2 && poll_buf[1].revents & POLLIN) + fd_ready = true; #endif + } + + PollResult result{}; + result.has_requests = socket_ready; + if (fd_ready) + { + UInt8 dummy; + readIntBinary(dummy, response_in); + result.responses_count = 1; + auto available = response_in.available(); + response_in.ignore(available); + result.responses_count += available; + } if (rc < 0) - { result.error = true; - return result; - } - else if (rc == 0) - { - return result; - } - else - { - if (socket_ready) - { - result.has_requests = true; - } - if (fd_ready) - { - UInt8 dummy; - readIntBinary(dummy, response_in); - result.has_response = true; - } - } + return result; } @@ -366,7 +355,7 @@ void NuKeeperTCPHandler::runImpl() /// Process exact amount of responses from pipe /// otherwise state of responses queue and signaling pipe /// became inconsistent and race condition is possible. - if (result.has_response) + while (result.responses_count != 0) { Coordination::ZooKeeperResponsePtr response; @@ -378,6 +367,7 @@ void NuKeeperTCPHandler::runImpl() LOG_DEBUG(log, "Session #{} successfully closed", session_id); return; } + response->write(*out); if (response->error == Coordination::Error::ZSESSIONEXPIRED) { @@ -385,6 +375,8 @@ void NuKeeperTCPHandler::runImpl() nu_keeper_storage_dispatcher->finishSession(session_id); return; } + + result.responses_count--; } if (result.error) From 196d45411463050b9ff7d352689cec92f6e5eae4 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Wed, 24 Feb 2021 14:55:16 +0300 Subject: [PATCH 362/510] Initialize random engine with clock and thread_id Thread-safe and fairly descent approach in our case --- base/mysqlxx/PoolWithFailover.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/base/mysqlxx/PoolWithFailover.cpp b/base/mysqlxx/PoolWithFailover.cpp index 4b77b41d251..5e9f70f4ac1 100644 --- a/base/mysqlxx/PoolWithFailover.cpp +++ b/base/mysqlxx/PoolWithFailover.cpp @@ -1,6 +1,7 @@ #include -#include +#include #include +#include #include @@ -43,7 +44,8 @@ PoolWithFailover::PoolWithFailover(const Poco::Util::AbstractConfiguration & con /// which triggers massive re-constructing of connection pools. /// The state of PRNGs like std::mt19937 is considered to be quite heavy /// thus here we attempt to optimize its construction. - static thread_local std::mt19937 rnd_generator(std::rand()); + static thread_local std::mt19937 rnd_generator( + std::hash{}(std::this_thread::get_id()) + std::clock()); for (auto & [_, replicas] : replicas_by_priority) { if (replicas.size() > 1) From 24e9abddb86d9e4f9455ca602a7535e51d5e6432 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 24 Feb 2021 15:11:07 +0300 Subject: [PATCH 363/510] Fix --- src/Client/Connection.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/Connection.h b/src/Client/Connection.h index bff1e391129..8e653dc4153 100644 --- a/src/Client/Connection.h +++ b/src/Client/Connection.h @@ -200,7 +200,7 @@ public: { async_callback = std::move(async_callback_); if (in) - in->setAsyncCallback(std::move(async_callback_)); + in->setAsyncCallback(std::move(async_callback)); } private: From cdaad3ad949dfa006bc6815dd5e80fc1bc4349d8 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 24 Feb 2021 16:51:56 +0300 Subject: [PATCH 364/510] Fix server stop --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 3af9f53d9e2..9a0d78de4b9 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 3af9f53d9e2e36f0c975a4cd665b446b875c4dde +Subproject commit 9a0d78de4b90546368d954b6434f0e9a823e8d80 From d966725f3320ce48b76d43e0092aaeb5161b2d23 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 Feb 2021 17:10:35 +0300 Subject: [PATCH 365/510] Try fix hung in void PullingAsyncPipelineExecutor::cancel() --- .../PullingAsyncPipelineExecutor.cpp | 21 +++++++++---------- src/Processors/Formats/LazyOutputFormat.h | 2 ++ 2 files changed, 12 insertions(+), 11 deletions(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 21741d30dfa..3270d1186f6 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -14,6 +14,7 @@ struct PullingAsyncPipelineExecutor::Data { PipelineExecutorPtr executor; std::exception_ptr exception; + LazyOutputFormat * lazy_format = nullptr; std::atomic_bool is_finished = false; std::atomic_bool has_exception = false; ThreadFromGlobalPool thread; @@ -82,6 +83,10 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou { data.exception = std::current_exception(); data.has_exception = true; + + /// Finish lazy format in case of exception. Otherwise thread.join() may hung. + if (data.lazy_format) + data.lazy_format->cancel(); } data.is_finished = true; @@ -95,6 +100,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) { data = std::make_unique(); data->executor = pipeline.execute(); + data->lazy_format = lazy_format.get(); auto func = [&, thread_group = CurrentThread::getGroup()]() { @@ -105,14 +111,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) } if (data->has_exception) - { - /// Finish lazy format in case of exception. Otherwise thread.join() may hung. - if (lazy_format) - lazy_format->finish(); - - data->has_exception = false; std::rethrow_exception(std::move(data->exception)); - } bool is_execution_finished = lazy_format ? lazy_format->isFinished() : data->is_finished.load(); @@ -172,14 +171,14 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) void PullingAsyncPipelineExecutor::cancel() { - /// Cancel execution if it wasn't finished. - if (data && !data->is_finished && data->executor) - data->executor->cancel(); - /// Finish lazy format. Otherwise thread.join() may hung. if (lazy_format && !lazy_format->isFinished()) lazy_format->finish(); + /// Cancel execution if it wasn't finished. + if (data && !data->is_finished && data->executor) + data->executor->cancel(); + /// Join thread here to wait for possible exception. if (data && data->thread.joinable()) data->thread.join(); diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 06ec116f3dd..647daa33889 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -36,6 +36,8 @@ public: queue.clear(); } + void onCancel() override { finalize(); } + protected: void consume(Chunk chunk) override { From 9ded4dd3966c9359bdddad75b638c0431f836057 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 24 Feb 2021 17:13:29 +0300 Subject: [PATCH 366/510] Update test. --- tests/queries/0_stateless/00205_scalar_subqueries.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/00205_scalar_subqueries.sql b/tests/queries/0_stateless/00205_scalar_subqueries.sql index 14244377e5f..03bcd0a3ebc 100644 --- a/tests/queries/0_stateless/00205_scalar_subqueries.sql +++ b/tests/queries/0_stateless/00205_scalar_subqueries.sql @@ -7,3 +7,4 @@ SELECT (SELECT toDate('2015-01-02'), 'Hello'); SELECT (SELECT toDate('2015-01-02'), 'Hello') AS x, x, identity((SELECT 1)), identity((SELECT 1) AS y); -- SELECT (SELECT uniqState('')); + SELECT ( SELECT throwIf(1 + dummy) ); -- { serverError 395 } From 6aa4791be290d24ee6e5ffddf9b9221bc87d4b58 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 19:19:04 +0300 Subject: [PATCH 367/510] Remove unused *Locked methods from HashJoin, add comments --- src/Interpreters/HashJoin.cpp | 24 +++++++----------------- src/Interpreters/HashJoin.h | 8 -------- src/Storages/StorageJoin.cpp | 20 +++++++++++++++++--- src/Storages/StorageJoin.h | 4 ++++ 4 files changed, 28 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index cd158241860..9c64b9522b9 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -421,22 +421,12 @@ bool HashJoin::empty() const return data->type == Type::EMPTY; } -size_t HashJoin::getTotalByteCount() const -{ - return getTotalByteCountLocked(); -} - -size_t HashJoin::getTotalRowCount() const -{ - return getTotalRowCountLocked(); -} - bool HashJoin::alwaysReturnsEmptySet() const { return isInnerOrRight(getKind()) && data->empty && !overDictionary(); } -size_t HashJoin::getTotalRowCountLocked() const +size_t HashJoin::getTotalRowCount() const { size_t res = 0; @@ -453,7 +443,7 @@ size_t HashJoin::getTotalRowCountLocked() const return res; } -size_t HashJoin::getTotalByteCountLocked() const +size_t HashJoin::getTotalByteCount() const { size_t res = 0; @@ -674,8 +664,8 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) return true; /// TODO: Do not calculate them every time - total_rows = getTotalRowCountLocked(); - total_bytes = getTotalByteCountLocked(); + total_rows = getTotalRowCount(); + total_bytes = getTotalByteCount(); } return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); @@ -1249,7 +1239,7 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, template ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const { - // Assemble the key block with correct names. + /// Assemble the key block with correct names. Block keys; for (size_t i = 0; i < block.columns(); ++i) { @@ -1264,8 +1254,8 @@ ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & b } -// TODO: return multiple columns as named tuple -// TODO: return array of values when strictness == ASTTableJoin::Strictness::All +/// TODO: return multiple columns as named tuple +/// TODO: return array of values when strictness == ASTTableJoin::Strictness::All ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 06e07dc10dd..d212e16b175 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -306,10 +306,6 @@ public: struct RightTableData { - /// Protect state for concurrent use in insertFromBlock and joinBlock. - /// @note that these methods could be called simultaneously only while use of StorageJoin. -// mutable std::shared_mutex rwlock; - Type type = Type::EMPTY; bool empty = true; @@ -400,10 +396,6 @@ private: static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); - /// Call with already locked rwlock. - size_t getTotalRowCountLocked() const; - size_t getTotalByteCountLocked() const; - bool empty() const; bool overDictionary() const; }; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index db80f0adb17..bfe866ed223 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -109,9 +109,23 @@ void StorageJoin::insertBlock(const Block & block) join->addJoinedBlock(block, true); } -size_t StorageJoin::getSize() const { return join->getTotalRowCount(); } -std::optional StorageJoin::totalRows(const Settings &) const { return join->getTotalRowCount(); } -std::optional StorageJoin::totalBytes(const Settings &) const { return join->getTotalByteCount(); } +size_t StorageJoin::getSize() const +{ + std::shared_lock lock(rwlock); + return join->getTotalRowCount(); +} + +std::optional StorageJoin::totalRows(const Settings &) const +{ + std::shared_lock lock(rwlock); + return join->getTotalRowCount(); +} + +std::optional StorageJoin::totalBytes(const Settings &) const +{ + std::shared_lock lock(rwlock); + return join->getTotalByteCount(); +} void registerStorageJoin(StorageFactory & factory) diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 6d3ec2710c9..7e4dea5d223 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -69,6 +69,10 @@ private: std::shared_ptr table_join; HashJoinPtr join; + + /// Protect state for concurrent use in insertFromBlock and joinBlock. + /// Lock hold via HashJoin instance (or HashJoinHolder for joinGet) + /// during all query and block insertions. mutable std::shared_mutex rwlock; void insertBlock(const Block & block) override; From f93e5b891e66662fe8c3531e3cdb42240cee0208 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 19:22:40 +0300 Subject: [PATCH 368/510] Rename 01731_race_condition_storage.sh -> 01732_race_condition_storage_long --- ...rage.reference => 01732_race_condition_storage_long.reference} | 0 ..._condition_storage.sh => 01732_race_condition_storage_long.sh} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{01731_race_condition_storage.reference => 01732_race_condition_storage_long.reference} (100%) rename tests/queries/0_stateless/{01731_race_condition_storage.sh => 01732_race_condition_storage_long.sh} (100%) diff --git a/tests/queries/0_stateless/01731_race_condition_storage.reference b/tests/queries/0_stateless/01732_race_condition_storage_long.reference similarity index 100% rename from tests/queries/0_stateless/01731_race_condition_storage.reference rename to tests/queries/0_stateless/01732_race_condition_storage_long.reference diff --git a/tests/queries/0_stateless/01731_race_condition_storage.sh b/tests/queries/0_stateless/01732_race_condition_storage_long.sh similarity index 100% rename from tests/queries/0_stateless/01731_race_condition_storage.sh rename to tests/queries/0_stateless/01732_race_condition_storage_long.sh From 6eaf9281b5525e0773bf7c08c4b9a2aedb8e050d Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 24 Feb 2021 14:25:28 +0000 Subject: [PATCH 369/510] Review fix --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 2 +- .../WriteBufferToRabbitMQProducer.cpp | 37 +++++++++---------- 2 files changed, 18 insertions(+), 21 deletions(-) diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 48305ab1b61..a0b014604fb 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -482,7 +482,7 @@ bool StorageRabbitMQ::restoreConnection(bool reconnecting) /* Connection is not closed immediately (firstly, all pending operations are completed, and then * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed */ - while (!connection->closed() && ++cnt_retries != RETRIES_MAX) + while (!connection->closed() && cnt_retries++ != RETRIES_MAX) event_handler->iterateLoop(); /// This will force immediate closure if not yet closed diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index a9b29236bb4..ea6195770d4 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -97,7 +97,7 @@ WriteBufferToRabbitMQProducer::~WriteBufferToRabbitMQProducer() connection->close(); size_t cnt_retries = 0; - while (!connection->closed() && ++cnt_retries != RETRIES_MAX) + while (!connection->closed() && cnt_retries++ != RETRIES_MAX) { event_handler->iterateLoop(); std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); @@ -220,30 +220,27 @@ void WriteBufferToRabbitMQProducer::setupChannel() void WriteBufferToRabbitMQProducer::removeRecord(UInt64 received_delivery_tag, bool multiple, bool republish) { auto record_iter = delivery_record.find(received_delivery_tag); + assert(record_iter != delivery_record.end()); - if (record_iter != delivery_record.end()) + if (multiple) { - if (multiple) - { - /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). - ++record_iter; + /// If multiple is true, then all delivery tags up to and including current are confirmed (with ack or nack). + ++record_iter; - if (republish) - for (auto record = delivery_record.begin(); record != record_iter; ++record) - returned.tryPush(record->second); + if (republish) + for (auto record = delivery_record.begin(); record != record_iter; ++record) + returned.tryPush(record->second); - /// Delete the records even in case when republished because new delivery tags will be assigned by the server. - delivery_record.erase(delivery_record.begin(), record_iter); - } - else - { - if (republish) - returned.tryPush(record_iter->second); - - delivery_record.erase(record_iter); - } + /// Delete the records even in case when republished because new delivery tags will be assigned by the server. + delivery_record.erase(delivery_record.begin(), record_iter); + } + else + { + if (republish) + returned.tryPush(record_iter->second); + + delivery_record.erase(record_iter); } - /// else is theoretically not possible } From 2480e4ee3d3f0ffef3abd3f66be8f3373e092697 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Wed, 24 Feb 2021 18:56:32 +0300 Subject: [PATCH 370/510] Better tests for protobuf format #2. --- src/Formats/ProtobufSerializer.cpp | 29 +- .../0_stateless/00825_protobuf_format_map.sh | 2 - ...protobuf_format_nested_in_nested.reference | 52 ++ .../00825_protobuf_format_nested_in_nested.sh | 36 + ...825_protobuf_format_nested_in_nested.proto | 11 + tests/queries/skip_list.json | 11 +- utils/CMakeLists.txt | 1 - utils/test-data-generator/CMakeLists.txt | 18 - .../ProtobufDelimitedMessagesSerializer.cpp | 728 ------------------ 9 files changed, 123 insertions(+), 765 deletions(-) create mode 100644 tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.reference create mode 100755 tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh create mode 100644 tests/queries/0_stateless/format_schemas/00825_protobuf_format_nested_in_nested.proto delete mode 100644 utils/test-data-generator/CMakeLists.txt delete mode 100644 utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp diff --git a/src/Formats/ProtobufSerializer.cpp b/src/Formats/ProtobufSerializer.cpp index 66efa76fe97..2ab8e11a854 100644 --- a/src/Formats/ProtobufSerializer.cpp +++ b/src/Formats/ProtobufSerializer.cpp @@ -2811,12 +2811,7 @@ namespace const auto & array_data_type = assert_cast(*data_type); if (!allow_repeat) - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) - + " must be repeated in the protobuf schema to match the column " + backQuote(StringRef{column_name}), - ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); - } + throwFieldNotRepeated(field_descriptor, column_name); auto nested_serializer = buildFieldSerializer(column_name, array_data_type.getNestedType(), field_descriptor, /* allow_repeat = */ false); // We do our repeating now, so for nested type we forget about the repeating. @@ -2861,12 +2856,7 @@ namespace /// Serialize as a repeated field. if (!allow_repeat && (size_of_tuple > 1)) - { - throw Exception( - "The field " + quoteString(field_descriptor.full_name()) - + " must be repeated in the protobuf schema to match the column " + backQuote(StringRef{column_name}), - ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); - } + throwFieldNotRepeated(field_descriptor, column_name); std::vector> nested_serializers; for (const auto & nested_data_type : tuple_data_type.getElements()) @@ -2892,6 +2882,21 @@ namespace } } + [[noreturn]] static void throwFieldNotRepeated(const FieldDescriptor & field_descriptor, const std::string_view & column_name) + { + if (!field_descriptor.is_repeated()) + throw Exception( + "The field " + quoteString(field_descriptor.full_name()) + + " must be repeated in the protobuf schema to match the column " + backQuote(StringRef{column_name}), + ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); + + throw Exception( + "The field " + quoteString(field_descriptor.full_name()) + + " is repeated but the level of repeatedness is not enough to serialize a multidimensional array from the column " + + backQuote(StringRef{column_name}) + ". It's recommended to make the parent field repeated as well.", + ErrorCodes::PROTOBUF_FIELD_NOT_REPEATED); + } + const ProtobufReaderOrWriter reader_or_writer; }; } diff --git a/tests/queries/0_stateless/00825_protobuf_format_map.sh b/tests/queries/0_stateless/00825_protobuf_format_map.sh index da7cacf76b8..877fe73603e 100755 --- a/tests/queries/0_stateless/00825_protobuf_format_map.sh +++ b/tests/queries/0_stateless/00825_protobuf_format_map.sh @@ -1,7 +1,5 @@ #!/usr/bin/env bash -# https://github.com/ClickHouse/ClickHouse/issues/6497 - CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) SCHEMADIR=$CURDIR/format_schemas # shellcheck source=../shell_config.sh diff --git a/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.reference b/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.reference new file mode 100644 index 00000000000..08fb4696506 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.reference @@ -0,0 +1,52 @@ +[[(1),(2)],[(3),(4),(5)]] +[[(6)]] +[[]] +[] + +Binary representation: +00000000 18 0a 08 12 02 18 01 12 02 18 02 0a 0c 12 02 18 |................| +00000010 03 12 02 18 04 12 02 18 05 06 0a 04 12 02 18 06 |................| +00000020 02 0a 00 00 |....| +00000024 + +MESSAGE #1 AT 0x00000001 +x { + y { + z: 1 + } + y { + z: 2 + } +} +x { + y { + z: 3 + } + y { + z: 4 + } + y { + z: 5 + } +} +MESSAGE #2 AT 0x0000001A +x { + y { + z: 6 + } +} +MESSAGE #3 AT 0x00000021 +x { +} +MESSAGE #4 AT 0x00000024 + +Binary representation is as expected + +[[(1),(2)],[(3),(4),(5)]] +[[(6)]] +[[]] +[] +[[(1),(2)],[(3),(4),(5)]] +[[(6)]] +[[]] +[] diff --git a/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh b/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh new file mode 100755 index 00000000000..9d382f8c589 --- /dev/null +++ b/tests/queries/0_stateless/00825_protobuf_format_nested_in_nested.sh @@ -0,0 +1,36 @@ +#!/usr/bin/env bash + +# https://github.com/ClickHouse/ClickHouse/issues/11117 + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +SCHEMADIR=$CURDIR/format_schemas +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +set -eo pipefail + +# Run the client. +$CLICKHOUSE_CLIENT --multiquery < "$BINARY_FILE_PATH" + +# Check the output in the protobuf format +echo +$CURDIR/helpers/protobuf_length_delimited_encoder.py --decode_and_check --format_schema "$SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType" --input "$BINARY_FILE_PATH" + +# Check the input in the protobuf format (now the table contains the same data twice). +echo +$CLICKHOUSE_CLIENT --query "INSERT INTO nested_in_nested_protobuf_00825 FORMAT Protobuf SETTINGS format_schema='$SCHEMADIR/00825_protobuf_format_nested_in_nested:MessageType'" < "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "SELECT * FROM nested_in_nested_protobuf_00825" + +rm "$BINARY_FILE_PATH" +$CLICKHOUSE_CLIENT --query "DROP TABLE nested_in_nested_protobuf_00825" diff --git a/tests/queries/0_stateless/format_schemas/00825_protobuf_format_nested_in_nested.proto b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_nested_in_nested.proto new file mode 100644 index 00000000000..f9617572f94 --- /dev/null +++ b/tests/queries/0_stateless/format_schemas/00825_protobuf_format_nested_in_nested.proto @@ -0,0 +1,11 @@ +syntax = "proto3"; + +message MessageType { + message XType { + message YType { + int32 z=3; + }; + repeated YType y=2; + }; + repeated XType x=1; +}; \ No newline at end of file diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 7f537ea7f88..c8a7cd2cdad 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -276,13 +276,16 @@ "00746_sql_fuzzy", "00763_create_query_as_table_engine_bug", "00765_sql_compatibility_aliases", - "00825_protobuf_format_input", - "00825_protobuf_format_nested_optional", "00825_protobuf_format_array_3dim", - "00825_protobuf_format_map", "00825_protobuf_format_array_of_arrays", - "00825_protobuf_format_table_default", "00825_protobuf_format_enum_mapping", + "00825_protobuf_format_map", + "00825_protobuf_format_nested_in_nested", + "00825_protobuf_format_nested_optional", + "00825_protobuf_format_no_length_delimiter", + "00825_protobuf_format_persons", + "00825_protobuf_format_squares", + "00825_protobuf_format_table_default", "00826_cross_to_inner_join", "00834_not_between", "00909_kill_not_initialized_query", diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index a27a7e9dadc..8a39d591612 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -27,7 +27,6 @@ if (NOT DEFINED ENABLE_UTILS OR ENABLE_UTILS) add_subdirectory (zookeeper-adjust-block-numbers-to-parts) add_subdirectory (wikistat-loader) add_subdirectory (check-marks) - add_subdirectory (test-data-generator) add_subdirectory (convert-month-partitioned-parts) add_subdirectory (checksum-for-compressed-block) add_subdirectory (db-generator) diff --git a/utils/test-data-generator/CMakeLists.txt b/utils/test-data-generator/CMakeLists.txt deleted file mode 100644 index 80d7d4301e7..00000000000 --- a/utils/test-data-generator/CMakeLists.txt +++ /dev/null @@ -1,18 +0,0 @@ -# Disable clang-tidy for protobuf generated files -set (CMAKE_CXX_CLANG_TIDY "") - -if (USE_PROTOBUF) - protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs ProtobufDelimitedMessagesSerializer_Hdrs ${CMAKE_CURRENT_SOURCE_DIR}/../../tests/queries/0_stateless/00825_protobuf_format.proto) - protobuf_generate_cpp(ProtobufDelimitedMessagesSerializer_Srcs2 ProtobufDelimitedMessagesSerializer_Hdrs2 ${CMAKE_CURRENT_SOURCE_DIR}/../../tests/queries/0_stateless/00825_protobuf_format_syntax2.proto) - add_executable (ProtobufDelimitedMessagesSerializer ProtobufDelimitedMessagesSerializer.cpp ${ProtobufDelimitedMessagesSerializer_Srcs} ${ProtobufDelimitedMessagesSerializer_Hdrs} ${ProtobufDelimitedMessagesSerializer_Srcs2} ${ProtobufDelimitedMessagesSerializer_Hdrs2}) - target_include_directories (ProtobufDelimitedMessagesSerializer SYSTEM BEFORE PRIVATE ${Protobuf_INCLUDE_DIR} ${CMAKE_CURRENT_BINARY_DIR}) - target_link_libraries (ProtobufDelimitedMessagesSerializer PRIVATE ${Protobuf_LIBRARY} boost::program_options) - get_filename_component(ProtobufDelimitedMessagesSerializer_OutputDir "${CMAKE_CURRENT_LIST_DIR}/../../tests/queries/0_stateless" REALPATH) - target_compile_definitions(ProtobufDelimitedMessagesSerializer PRIVATE OUTPUT_DIR="${ProtobufDelimitedMessagesSerializer_OutputDir}") - - # Protoc generates substandard code. - check_cxx_compiler_flag("-Wsuggest-destructor-override" HAS_SUGGEST_DESTRUCTOR_OVERRIDE) - if (HAS_SUGGEST_OVERRIDE) - target_compile_options(ProtobufDelimitedMessagesSerializer PRIVATE -Wno-suggest-destructor-override) - endif() -endif () diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp deleted file mode 100644 index ad465913313..00000000000 --- a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp +++ /dev/null @@ -1,728 +0,0 @@ -// Generator of protobuf delimited messages used in the protobuf IO tests -// tests/queries/0_stateless/00825_protobuf_format* - -#include -#include -#include -#include -#include "00825_protobuf_format.pb.h" -#include "00825_protobuf_format_syntax2.pb.h" - - -void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, const std::string & table_name, const std::string & format_schema, std::ostream & out) // STYLE_CHECK_ALLOW_STD_STRING_STREAM -{ - out << "echo -ne '"; - std::string bytes = delimited_messages.str(); - delimited_messages.str(""); - for (const char c : bytes) - { - char buf[5]; - sprintf(buf, "\\x%02x", static_cast(c)); - out << buf; - } - out << "' | $CLICKHOUSE_CLIENT --query=\"INSERT INTO " << table_name << " FORMAT Protobuf" - " SETTINGS format_schema = '$CURDIR/" - << format_schema << "'\"" << std::endl; -} - -void writeInsertDataQueriesForInputTest(std::ostream & out) -{ - std::stringstream ss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - { - Person person; - person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); - person.set_name("Ivan"); - person.set_surname("Petrov"); - person.set_gender(Gender::male); - person.set_birthdate(4015); // 1980-12-29 - person.set_photo("png"); - person.set_phonenumber("+74951234567"); - person.set_isonline(true); - person.set_visittime(1546703100); // 2019-01-05 18:45:00 - person.set_age(38); - person.set_zodiacsign(ZodiacSign::capricorn); - person.add_songs("Yesterday"); - person.add_songs("Flowers"); - person.add_color(255); - person.add_color(0); - person.add_color(0); - person.set_hometown("Moscow"); - person.add_location(55.753215); - person.add_location(37.622504); - person.set_pi(3.14); - person.set_lotterywin(214.10); - person.set_someratio(0.1); - person.set_temperature(5.8); - person.set_randombignumber(17060000000); - auto* mu = person.add_measureunits(); - mu->set_unit("meter"); - mu->set_coef(1); - mu = person.add_measureunits(); - mu->set_unit("centimeter"); - mu->set_coef(0.01); - mu = person.add_measureunits(); - mu->set_unit("kilometer"); - mu->set_coef(1000); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(500); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(501); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(502); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - { - Person person; - person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); - person.set_name("Natalia"); - person.set_surname("Sokolova"); - person.set_gender(Gender::female); - person.set_birthdate(8102); // 1992-03-08 - person.set_photo("jpg"); - person.set_isonline(false); - person.set_age(26); - person.set_zodiacsign(ZodiacSign::pisces); - person.add_color(100); - person.add_color(200); - person.add_color(50); - person.set_hometown("Plymouth"); - person.add_location(50.403724); - person.add_location(-4.142123); - person.set_pi(3.14159); - person.set_someratio(0.007); - person.set_temperature(5.4); - person.set_randombignumber(-20000000000000); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - { - Person person; - person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); - person.set_name("Vasily"); - person.set_surname("Sidorov"); - person.set_gender(Gender::male); - person.set_birthdate(9339); // 1995-07-28 - person.set_photo("bmp"); - person.set_phonenumber("+442012345678"); - person.set_isonline(true); - person.set_visittime(1546117200); // 2018-12-30 00:00:00 - person.set_age(23); - person.set_zodiacsign(ZodiacSign::leo); - person.add_songs("Sunny"); - person.add_color(250); - person.add_color(244); - person.add_color(10); - person.set_hometown("Murmansk"); - person.add_location(68.970682); - person.add_location(33.074981); - person.set_pi(3.14159265358979); - person.set_lotterywin(100000000000); - person.set_someratio(800); - person.set_temperature(-3.2); - person.set_randombignumber(154400000); - auto* mu = person.add_measureunits(); - mu->set_unit("pound"); - mu->set_coef(16); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format:Person", out); - - { - AltPerson person; - person.add_location(42); - person.add_location(-88); - person.set_pi(3.141); - person.set_uuid("20fcd95a-332d-41db-a9ec-161f644d059c"); - person.set_name("Frida"); - person.set_gender(AltPerson::female); - person.set_zodiacsign(1122); // sagittarius - person.set_birthdate(3267); // 1978-12-12 - person.set_age("40"); - person.set_isonline(OnlineStatus::offline); - person.set_someratio(0.5); - person.set_visittime(1363005000); // 2013-03-11 16:30:00 - person.set_randombignumber(8010000009); - person.add_color(110); - person.add_color(210); - person.add_color(74); - person.set_lotterywin(311); - person.set_surname("Ermakova"); - person.set_phonenumber(3124555929); - person.set_temperature(10); - person.add_measureunits_unit("KB"); - person.add_measureunits_coef(1024); - person.add_measureunits_unit("MB"); - person.add_measureunits_coef(1048576); - person.set_nestiness_a_b_c_d(700); - person.add_nestiness_a_b_c_e(701); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - { - AltPerson person; - person.add_location(26); - person.add_location(-80); - person.set_pi(3.1416); - person.set_uuid("7cfa6856-a54a-4786-b8e5-745159d52278"); - person.set_name("Isolde"); - person.set_gender(AltPerson::female); - person.set_zodiacsign(120); // aquarius - person.set_birthdate(6248); // 1987-02-09 - person.set_age("32"); - person.set_isonline(OnlineStatus::online); - person.set_someratio(4.5); - person.set_randombignumber(-11111111111111); - person.add_color(255); - person.add_color(0); - person.add_color(255); - person.set_surname("Lavrova"); - person.set_temperature(25); - person.set_newfieldstr("abc"); - person.set_newfieldbool(true); - person.add_newfieldint(44); - person.add_measureunits_unit("Byte"); - person.add_measureunits_coef(8); - person.add_measureunits_unit("Bit"); - person.add_measureunits_coef(1); - person.mutable_newmessage()->set_z(91); - person.set_nestiness_a_b_c_d(702); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format:AltPerson", out); - - { - StrPerson person; - person.set_uuid("aa0e5a06-cab2-4034-a6a2-48e82b91664e"); - person.set_name("Leonid"); - person.set_surname("Kirillov"); - person.set_gender("male"); - person.set_birthdate("1983-06-24"); - person.set_phonenumber("+74950275864"); - person.set_isonline("1"); - person.set_visittime("2019-02-04 09:45:00"); - person.set_age("35"); - person.set_zodiacsign("cancer"); - person.add_songs("7 rings"); - person.add_songs("Eastside"); - person.add_songs("Last Hurrah"); - person.add_color("0"); - person.add_color("0"); - person.add_color("255"); - person.set_hometown("San Diego"); - person.add_location("32.823943"); - person.add_location("-117.081327"); - person.set_pi("3.1415927"); - person.set_lotterywin("15000000"); - person.set_someratio("186.75"); - person.set_temperature("-2.1"); - person.set_randombignumber("20659829331"); - person.mutable_measureunits()->add_unit("minute"); - person.mutable_measureunits()->add_coef("60"); - person.mutable_measureunits()->add_unit("hour"); - person.mutable_measureunits()->add_coef("3600"); - person.mutable_nestiness_a()->mutable_b_c()->add_e("1800"); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format:StrPerson", out); - - { - Syntax2Person person; - person.set_uuid("3faee064-c4f7-4d34-b6f3-8d81c2b6a15d"); - person.set_name("Nick"); - person.set_surname("Kolesnikov"); - person.set_gender(Syntax2Person::male); - person.set_birthdate(10586); // 1998-12-26 - person.set_photo("bmp"); - person.set_phonenumber("412-687-5007"); - person.set_isonline(true); - person.set_visittime(1542596399); // 2018-11-19 05:59:59 - person.set_age(20); - person.set_zodiacsign(Syntax2Person::capricorn); - person.add_songs("Havana"); - person.add_color(128); - person.add_color(0); - person.add_color(128); - person.set_hometown("Pittsburgh"); - person.add_location(40.517193); - person.add_location(-79.949452); - person.set_pi(3.1415926535898); - person.set_lotterywin(50000000000); - person.set_someratio(780); - person.set_temperature(18.3); - person.set_randombignumber(195500007); - person.mutable_measureunits()->add_unit("ounce"); - person.mutable_measureunits()->add_coef(28.35); - person.mutable_measureunits()->add_unit("carat"); - person.mutable_measureunits()->add_coef(0.2); - person.mutable_measureunits()->add_unit("gram"); - person.mutable_measureunits()->add_coef(1); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(9494); - google::protobuf::util::SerializeDelimitedToOstream(person, &ss); - } - - writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format_syntax2:Syntax2Person", out); - - { - NumberAndSquare ns; - ns.set_number(2); - ns.set_square(4); - google::protobuf::util::SerializeDelimitedToOstream(ns, &ss); - } - - { - NumberAndSquare ns; - ns.set_number(0); - ns.set_square(0); - google::protobuf::util::SerializeDelimitedToOstream(ns, &ss); - } - - { - NumberAndSquare ns; - ns.set_number(3); - ns.set_square(9); - google::protobuf::util::SerializeDelimitedToOstream(ns, &ss); - } - - writeInsertDataQueryForInputTest(ss, "in_squares_00825", "00825_protobuf_format:NumberAndSquare", out); -} - - -void writeReferenceForOutputTest(std::ostream & out) -{ - { - Person person; - person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); - person.set_name("Ivan"); - person.set_surname("Petrov"); - person.set_gender(Gender::male); - person.set_birthdate(4015); // 1980-12-29 - person.set_photo("png"); - person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) - person.set_isonline(true); - person.set_visittime(1546703100); // 2019-01-05 18:45:00 - person.set_age(38); - person.set_zodiacsign(ZodiacSign::capricorn); - person.add_songs("Yesterday"); - person.add_songs("Flowers"); - person.add_color(255); - person.add_color(0); - person.add_color(0); - person.set_hometown("Moscow"); - person.add_location(55.753215); - person.add_location(37.622504); - person.set_pi(3.14); - person.set_lotterywin(214.10); - person.set_someratio(0.1); - person.set_temperature(5.8); - person.set_randombignumber(17060000000); - auto* mu = person.add_measureunits(); - mu->set_unit("meter"); - mu->set_coef(1); - mu = person.add_measureunits(); - mu->set_unit("centimeter"); - mu->set_coef(0.01); - mu = person.add_measureunits(); - mu->set_unit("kilometer"); - mu->set_coef(1000); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(500); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(501); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(502); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - Person person; - person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); - person.set_name("Natalia"); - person.set_surname("Sokolova"); - person.set_gender(Gender::female); - person.set_birthdate(8102); // 1992-03-08 - person.set_photo("jpg"); - person.set_isonline(false); - person.set_age(26); - person.set_zodiacsign(ZodiacSign::pisces); - person.add_color(100); - person.add_color(200); - person.add_color(50); - person.set_hometown("Plymouth"); - person.add_location(50.403724); - person.add_location(-4.142123); - person.set_pi(3.14159); - person.set_someratio(0.007); - person.set_temperature(5.4); - person.set_randombignumber(-20000000000000); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - Person person; - person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); - person.set_name("Vasily"); - person.set_surname("Sidorov"); - person.set_gender(Gender::male); - person.set_birthdate(9339); // 1995-07-28 - person.set_photo("bmp"); - person.set_phonenumber("+442012345678"); - person.set_isonline(true); - person.set_visittime(1546117200); // 2018-12-30 00:00:00 - person.set_age(23); - person.set_zodiacsign(ZodiacSign::leo); - person.add_songs("Sunny"); - person.add_color(250); - person.add_color(244); - person.add_color(10); - person.set_hometown("Murmansk"); - person.add_location(68.970682); - person.add_location(33.074981); - person.set_pi(3.14159265358979); - person.set_lotterywin(100000000000); - person.set_someratio(800); - person.set_temperature(-3.2); - person.set_randombignumber(154400000); - auto* mu = person.add_measureunits(); - mu->set_unit("pound"); - mu->set_coef(16); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - out << "ALTERNATIVE->" << std::endl; - - { - AltPerson person; - person.add_location(55); - person.add_location(37); - person.set_pi(3.14); - person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); - person.set_name("Ivan"); - person.set_gender(AltPerson::male); - person.set_zodiacsign(1222); // capricorn - person.set_birthdate(4015); // 1980-12-29 - person.set_age("38"); - person.set_isonline(OnlineStatus::online); - person.set_someratio(0.100000001490116119384765625); // 0.1 converted from float to double - person.set_visittime(1546703100); // 2019-01-05 18:45:00 - person.set_randombignumber(17060000000); - person.add_color(255); - person.add_color(0); - person.add_color(0); - person.set_lotterywin(214); - person.set_surname("Petrov"); - person.set_phonenumber(+74951234567); - person.set_temperature(5); - person.add_measureunits_unit("meter"); - person.add_measureunits_coef(1); - person.add_measureunits_unit("centimeter"); - person.add_measureunits_coef(0.01); - person.add_measureunits_unit("kilometer"); - person.add_measureunits_coef(1000); - person.set_nestiness_a_b_c_d(500); - person.add_nestiness_a_b_c_e(501); - person.add_nestiness_a_b_c_e(502); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - AltPerson person; - person.add_location(50); - person.add_location(-4); - person.set_pi(3.14159); - person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); - person.set_name("Natalia"); - person.set_gender(AltPerson::female); - person.set_zodiacsign(219); // pisces - person.set_birthdate(8102); // 1992-03-08 - person.set_age("26"); - person.set_isonline(OnlineStatus::offline); - person.set_someratio(0.007000000216066837310791015625); // 0.007 converted from float to double - person.set_randombignumber(-20000000000000); - person.add_color(100); - person.add_color(200); - person.add_color(50); - person.set_surname("Sokolova"); - person.set_temperature(5); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - AltPerson person; - person.add_location(68); - person.add_location(33); - person.set_pi(3.1415926535897); - person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); - person.set_name("Vasily"); - person.set_gender(AltPerson::male); - person.set_zodiacsign(723); // leo - person.set_birthdate(9339); // 1995-07-28 - person.set_age("23"); - person.set_isonline(OnlineStatus::online); - person.set_someratio(800); - person.set_visittime(1546117200); // 2018-12-30 00:00:00 - person.set_randombignumber(154400000); - person.add_color(250); - person.add_color(244); - person.add_color(10); - person.set_lotterywin(100000000000); - person.set_surname("Sidorov"); - person.set_phonenumber(+442012345678); - person.set_temperature(-3); - person.add_measureunits_unit("pound"); - person.add_measureunits_coef(16); - person.set_nestiness_a_b_c_d(503); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - out << "STRINGS->" << std::endl; - - { - StrPerson person; - person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); - person.set_name("Ivan"); - person.set_surname("Petrov"); - person.set_gender("male"); - person.set_birthdate("1980-12-29"); - person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) - person.set_isonline("1"); - person.set_visittime("2019-01-05 18:45:00"); - person.set_age("38"); - person.set_zodiacsign("capricorn"); - person.add_songs("Yesterday"); - person.add_songs("Flowers"); - person.add_color("255"); - person.add_color("0"); - person.add_color("0"); - person.set_hometown("Moscow"); - person.add_location("55.753215"); - person.add_location("37.622504"); - person.set_pi("3.14"); - person.set_lotterywin("214.10"); - person.set_someratio("0.1"); - person.set_temperature("5.8"); - person.set_randombignumber("17060000000"); - person.mutable_measureunits()->add_unit("meter"); - person.mutable_measureunits()->add_coef("1"); - person.mutable_measureunits()->add_unit("centimeter"); - person.mutable_measureunits()->add_coef("0.01"); - person.mutable_measureunits()->add_unit("kilometer"); - person.mutable_measureunits()->add_coef("1000"); - person.mutable_nestiness_a()->mutable_b_c()->set_d("500"); - person.mutable_nestiness_a()->mutable_b_c()->add_e("501"); - person.mutable_nestiness_a()->mutable_b_c()->add_e("502"); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - StrPerson person; - person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); - person.set_name("Natalia"); - person.set_surname("Sokolova"); - person.set_gender("female"); - person.set_birthdate("1992-03-08"); - person.set_isonline("0"); - person.set_age("26"); - person.set_zodiacsign("pisces"); - person.add_color("100"); - person.add_color("200"); - person.add_color("50"); - person.set_hometown("Plymouth"); - person.add_location("50.403724"); - person.add_location("-4.142123"); - person.set_pi("3.14159"); - person.set_someratio("0.007"); - person.set_temperature("5.4"); - person.set_randombignumber("-20000000000000"); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - StrPerson person; - person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); - person.set_name("Vasily"); - person.set_surname("Sidorov"); - person.set_gender("male"); - person.set_birthdate("1995-07-28"); - person.set_phonenumber("+442012345678"); - person.set_isonline("1"); - person.set_visittime("2018-12-30 00:00:00"); - person.set_age("23"); - person.set_zodiacsign("leo"); - person.add_songs("Sunny"); - person.add_color("250"); - person.add_color("244"); - person.add_color("10"); - person.set_hometown("Murmansk"); - person.add_location("68.970682"); - person.add_location("33.074981"); - person.set_pi("3.14159265358979"); - person.set_lotterywin("100000000000.00"); - person.set_someratio("800"); - person.set_temperature("-3.2"); - person.set_randombignumber("154400000"); - person.mutable_measureunits()->add_unit("pound"); - person.mutable_measureunits()->add_coef("16"); - person.mutable_nestiness_a()->mutable_b_c()->set_d("503"); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - out << "SYNTAX2->" << std::endl; - - { - Syntax2Person person; - person.set_uuid("a7522158-3d41-4b77-ad69-6c598ee55c49"); - person.set_name("Ivan"); - person.set_surname("Petrov"); - person.set_gender(Syntax2Person::male); - person.set_birthdate(4015); // 1980-12-29 - person.set_photo("png"); - person.set_phonenumber(std::string("+74951234567\0", 13)); // Converted from FixedString(13) - person.set_isonline(true); - person.set_visittime(1546703100); // 2019-01-05 18:45:00 - person.set_age(38); - person.set_zodiacsign(Syntax2Person::capricorn); - person.add_songs("Yesterday"); - person.add_songs("Flowers"); - person.add_color(255); - person.add_color(0); - person.add_color(0); - person.set_hometown("Moscow"); - person.add_location(55.753215); - person.add_location(37.622504); - person.set_pi(3.14); - person.set_lotterywin(214.10); - person.set_someratio(0.1); - person.set_temperature(5.8); - person.set_randombignumber(17060000000); - person.mutable_measureunits()->add_unit("meter"); - person.mutable_measureunits()->add_coef(1); - person.mutable_measureunits()->add_unit("centimeter"); - person.mutable_measureunits()->add_coef(0.01); - person.mutable_measureunits()->add_unit("kilometer"); - person.mutable_measureunits()->add_coef(1000); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(500); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(501); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->add_e(502); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - Syntax2Person person; - person.set_uuid("c694ad8a-f714-4ea3-907d-fd54fb25d9b5"); - person.set_name("Natalia"); - person.set_surname("Sokolova"); - person.set_gender(Syntax2Person::female); - person.set_birthdate(8102); // 1992-03-08 - person.set_photo("jpg"); - person.set_isonline(false); - person.set_age(26); - person.set_zodiacsign(Syntax2Person::pisces); - person.add_color(100); - person.add_color(200); - person.add_color(50); - person.set_hometown("Plymouth"); - person.add_location(50.403724); - person.add_location(-4.142123); - person.set_pi(3.14159); - person.set_someratio(0.007); - person.set_temperature(5.4); - person.set_randombignumber(-20000000000000); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - { - Syntax2Person person; - person.set_uuid("a7da1aa6-f425-4789-8947-b034786ed374"); - person.set_name("Vasily"); - person.set_surname("Sidorov"); - person.set_gender(Syntax2Person::male); - person.set_birthdate(9339); // 1995-07-28 - person.set_photo("bmp"); - person.set_phonenumber("+442012345678"); - person.set_isonline(true); - person.set_visittime(1546117200); // 2018-12-30 00:00:00 - person.set_age(23); - person.set_zodiacsign(Syntax2Person::leo); - person.add_songs("Sunny"); - person.add_color(250); - person.add_color(244); - person.add_color(10); - person.set_hometown("Murmansk"); - person.add_location(68.970682); - person.add_location(33.074981); - person.set_pi(3.14159265358979); - person.set_lotterywin(100000000000); - person.set_someratio(800); - person.set_temperature(-3.2); - person.set_randombignumber(154400000); - person.mutable_measureunits()->add_unit("pound"); - person.mutable_measureunits()->add_coef(16); - person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503); - google::protobuf::util::SerializeDelimitedToOstream(person, &out); - } - - out << "SQUARES->" << std::endl; - - { - NumberAndSquare ns; - ns.set_number(0); - ns.set_square(0); - google::protobuf::util::SerializeDelimitedToOstream(ns, &out); - } - - { - NumberAndSquare ns; - ns.set_number(2); - ns.set_square(4); - google::protobuf::util::SerializeDelimitedToOstream(ns, &out); - } - - { - NumberAndSquare ns; - ns.set_number(3); - ns.set_square(9); - google::protobuf::util::SerializeDelimitedToOstream(ns, &out); - } -} - - -void parseCommandLine(int argc, char ** argv, std::string & output_dir) -{ - namespace po = boost::program_options; - po::options_description desc; - output_dir = OUTPUT_DIR; - desc.add_options() - ("help,h", "Show help") - ("directory,d", po::value(&output_dir), - "Set the output directory. By default it's " OUTPUT_DIR); - po::parsed_options parsed = po::command_line_parser(argc, argv).options(desc).run(); - po::variables_map vm; - po::store(parsed, vm); - po::notify(vm); - if (!output_dir.empty()) - return; - - // Show help. - std::cout << "This utility generates delimited messages for tests checking protobuf IO support." << std::endl; - std::cout << desc; - std::cout << "Example:" << std::endl; - std::cout << argv[0] << " -g OUTPUT_REFERENCE" << std::endl; - std::exit(0); -} - -void writeFile(const std::string & filepath, void (*fn)(std::ostream &)) -{ - std::cout << "Writing '" << filepath << "' ... "; - std::fstream out(filepath, std::fstream::out | std::fstream::trunc); - fn(out); - std::cout << "done." << std::endl; -} - -int main(int argc, char ** argv) -{ - std::string output_dir; - parseCommandLine(argc, argv, output_dir); - writeFile(output_dir + "/00825_protobuf_format_input.insh", writeInsertDataQueriesForInputTest); - writeFile(output_dir + "/00825_protobuf_format_output.reference", writeReferenceForOutputTest); - return 0; -} From 32f10f402ce8c911d4b89860809e6193bd1da6ce Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Feb 2021 21:48:28 +0300 Subject: [PATCH 371/510] Updated documentation --- docs/ru/sql-reference/functions/type-conversion-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 4de2b5c6e3e..7453b7cd819 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -675,7 +675,7 @@ parseDateTimeBestEffortUSOrNull(time_string[, time_zone]) **Поддерживаемые нестандартные форматы** -- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 символов. +- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 чисел. - Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss` и т.д. - Строка, содержащая дату без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` и т.д. - Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. @@ -761,7 +761,7 @@ parseDateTimeBestEffortUSOrZero(time_string[, time_zone]) **Поддерживаемые нестандартные форматы** -- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 символов. +- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 чисел. - Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss` и т.д. - Строка, содержащая дату без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` и т.д. - Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. From af13c36a4baff1c13efc5d0d8d4dd62a5aec3d54 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 24 Feb 2021 21:49:56 +0300 Subject: [PATCH 372/510] Updated documentation --- docs/ru/sql-reference/functions/type-conversion-functions.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 7453b7cd819..308655c9acb 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -675,7 +675,7 @@ parseDateTimeBestEffortUSOrNull(time_string[, time_zone]) **Поддерживаемые нестандартные форматы** -- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 чисел. +- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 цифр. - Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss` и т.д. - Строка, содержащая дату без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` и т.д. - Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. @@ -761,7 +761,7 @@ parseDateTimeBestEffortUSOrZero(time_string[, time_zone]) **Поддерживаемые нестандартные форматы** -- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 чисел. +- Строка в формате [unix timestamp](https://en.wikipedia.org/wiki/Unix_time), содержащая 9-10 цифр. - Строка, содержащая дату и время: `YYYYMMDDhhmmss`, `MM/DD/YYYY hh:mm:ss`, `MM-DD-YY hh:mm`, `YYYY-MM-DD hh:mm:ss` и т.д. - Строка, содержащая дату без времени: `YYYY`, `YYYYMM`, `YYYY*MM`, `MM/DD/YYYY`, `MM-DD-YY` и т.д. - Строка, содержащая день и время: `DD`, `DD hh`, `DD hh:mm`. В этом случае `YYYY-MM` заменяется на `2000-01`. From 58b0bbe6b1067b4eb3c2068b7bedb0c328b18372 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 24 Feb 2021 21:25:01 +0300 Subject: [PATCH 373/510] Fix ccache 4.2+ usage (RULE_LAUNCH_COMPILE/RULE_LAUNCH_LINK was not set) --- cmake/find/ccache.cmake | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cmake/find/ccache.cmake b/cmake/find/ccache.cmake index d9ccd1a9ac6..fea1f8b4c97 100644 --- a/cmake/find/ccache.cmake +++ b/cmake/find/ccache.cmake @@ -32,7 +32,10 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) if (CCACHE_VERSION VERSION_GREATER "3.2.0" OR NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") message(STATUS "Using ${CCACHE_FOUND} ${CCACHE_VERSION}") - # debian (debhlpers) set SOURCE_DATE_EPOCH environment variable, that is + set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) + set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) + + # debian (debhelpers) set SOURCE_DATE_EPOCH environment variable, that is # filled from the debian/changelog or current time. # # - 4.0+ ccache always includes this environment variable into the hash @@ -48,9 +51,6 @@ if (CCACHE_FOUND AND NOT COMPILER_MATCHES_CCACHE) message(STATUS "Ignore SOURCE_DATE_EPOCH for ccache") set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE "env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}") set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK "env -u SOURCE_DATE_EPOCH ${CCACHE_FOUND}") - else() - set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) - set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) endif() else () message(${RECONFIGURE_MESSAGE_LEVEL} "Not using ${CCACHE_FOUND} ${CCACHE_VERSION} bug: https://bugzilla.samba.org/show_bug.cgi?id=8118") From aa8632a1bc2eb6379e83655301d9ca00cc156b33 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Feb 2021 00:27:47 +0300 Subject: [PATCH 374/510] Fix tests. --- .../Executors/PullingAsyncPipelineExecutor.cpp | 4 ++-- src/Processors/Formats/LazyOutputFormat.h | 17 ++++++++--------- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index 3270d1186f6..d058ea9e6ac 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -86,7 +86,7 @@ static void threadFunction(PullingAsyncPipelineExecutor::Data & data, ThreadGrou /// Finish lazy format in case of exception. Otherwise thread.join() may hung. if (data.lazy_format) - data.lazy_format->cancel(); + data.lazy_format->finalize(); } data.is_finished = true; @@ -120,7 +120,7 @@ bool PullingAsyncPipelineExecutor::pull(Chunk & chunk, uint64_t milliseconds) { /// If lazy format is finished, we don't cancel pipeline but wait for main thread to be finished. data->is_finished = true; - /// Wait thread ant rethrow exception if any. + /// Wait thread and rethrow exception if any. cancel(); return false; } diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 647daa33889..7188458dd82 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -36,7 +36,14 @@ public: queue.clear(); } - void onCancel() override { finalize(); } + void finalize() override + { + std::cerr << StackTrace().toString() << std::endl; + finished_processing = true; + + /// In case we are waiting for result. + queue.emplace(Chunk()); + } protected: void consume(Chunk chunk) override @@ -48,14 +55,6 @@ protected: void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); } - void finalize() override - { - finished_processing = true; - - /// In case we are waiting for result. - queue.emplace(Chunk()); - } - private: ConcurrentBoundedQueue queue; From 6f5d4ba8cd9ffdec8743c9c7cdff996324b5481c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Feb 2021 00:31:15 +0300 Subject: [PATCH 375/510] Fix tests. --- src/Processors/Executors/PullingAsyncPipelineExecutor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp index d058ea9e6ac..c975153d317 100644 --- a/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp +++ b/src/Processors/Executors/PullingAsyncPipelineExecutor.cpp @@ -171,14 +171,14 @@ bool PullingAsyncPipelineExecutor::pull(Block & block, uint64_t milliseconds) void PullingAsyncPipelineExecutor::cancel() { - /// Finish lazy format. Otherwise thread.join() may hung. - if (lazy_format && !lazy_format->isFinished()) - lazy_format->finish(); - /// Cancel execution if it wasn't finished. if (data && !data->is_finished && data->executor) data->executor->cancel(); + /// Finish lazy format. Otherwise thread.join() may hung. + if (lazy_format && !lazy_format->isFinished()) + lazy_format->finish(); + /// Join thread here to wait for possible exception. if (data && data->thread.joinable()) data->thread.join(); From e4f17288e5ba2f0c3964feed626ae9cd98d92723 Mon Sep 17 00:00:00 2001 From: jennyma Date: Thu, 25 Feb 2021 10:24:29 +0800 Subject: [PATCH 376/510] restart the CI checks From 3ddb729e4a6c6c0c2a2ce4edacf072090309688a Mon Sep 17 00:00:00 2001 From: fastio Date: Wed, 24 Feb 2021 14:20:44 +0800 Subject: [PATCH 377/510] fix metadata leak when drop Replicated*MergeTree --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f2c88cdedd9..b0bf2d2ce73 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -750,7 +750,7 @@ void StorageReplicatedMergeTree::drop() { /// Table can be shut down, restarting thread is not active /// and calling StorageReplicatedMergeTree::getZooKeeper() won't suffice. - auto zookeeper = global_context.getZooKeeper(); + auto zookeeper = getZooKeeper(); /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. if (!zookeeper) From fea28366735790e4500335a6976941343a412ba7 Mon Sep 17 00:00:00 2001 From: fastio Date: Wed, 24 Feb 2021 16:33:39 +0800 Subject: [PATCH 378/510] get zookeeper from global context --- src/Storages/StorageReplicatedMergeTree.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index b0bf2d2ce73..577241e1e4a 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -749,8 +749,12 @@ void StorageReplicatedMergeTree::drop() if (has_metadata_in_zookeeper) { /// Table can be shut down, restarting thread is not active - /// and calling StorageReplicatedMergeTree::getZooKeeper() won't suffice. - auto zookeeper = getZooKeeper(); + /// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice. + zkutil::ZooKeeperPtr zookeeper; + if (zookeeper_name == default_zookeeper_name) + zookeeper = global_context.getZooKeeper(); + else + zookeeper = global_context.getAuxiliaryZooKeeper(zookeeper_name); /// If probably there is metadata in ZooKeeper, we don't allow to drop the table. if (!zookeeper) From 24321c28f649fdad91d8a800ee4244e103d07d15 Mon Sep 17 00:00:00 2001 From: fastio Date: Wed, 24 Feb 2021 17:23:52 +0800 Subject: [PATCH 379/510] add test case --- .../test.py | 25 ++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py b/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py index 91a25ec8d8a..a9dcce1b9d4 100644 --- a/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py +++ b/tests/integration/test_replicated_merge_tree_with_auxiliary_zookeepers/test.py @@ -6,7 +6,6 @@ from helpers.cluster import ClickHouseCluster from helpers.client import QueryRuntimeException from helpers.test_tools import TSV -cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance("node1", main_configs=["configs/zookeeper_config.xml", "configs/remote_servers.xml"], with_zookeeper=True) node2 = cluster.add_instance("node2", main_configs=["configs/zookeeper_config.xml", "configs/remote_servers.xml"], with_zookeeper=True) @@ -78,3 +77,27 @@ def test_create_replicated_merge_tree_with_not_exists_auxiliary_zookeeper(starte ENGINE = ReplicatedMergeTree('zookeeper_not_exits:/clickhouse/tables/test/test_auxiliary_zookeeper', '{replica}') ORDER BY a; '''.format(replica=node1.name)) + +# Drop table with auxiliary zookeeper. +def test_drop_replicated_merge_tree_with_auxiliary_zookeeper(started_cluster): + drop_table([node1, node2], "test_auxiliary_zookeeper") + for node in [node1, node2]: + node.query( + ''' + CREATE TABLE test_auxiliary_zookeeper(a Int32) + ENGINE = ReplicatedMergeTree('zookeeper2:/clickhouse/tables/test/test_auxiliary_zookeeper', '{replica}') + ORDER BY a; + '''.format(replica=node.name)) + + # Insert data into node1, and query it from node2. + node1.query("INSERT INTO test_auxiliary_zookeeper VALUES (1)") + time.sleep(5) + + expected = "1\n" + assert TSV(node1.query("SELECT a FROM test_auxiliary_zookeeper")) == TSV(expected) + assert TSV(node2.query("SELECT a FROM test_auxiliary_zookeeper")) == TSV(expected) + + zk = cluster.get_kazoo_client('zoo1') + assert zk.exists('/clickhouse/tables/test/test_auxiliary_zookeeper') + drop_table([node1, node2], "test_auxiliary_zookeeper") + assert zk.exists('/clickhouse/tables/test/test_auxiliary_zookeeper') is None From 731faeadbfa9effe9583cc4f244c3c748da7967e Mon Sep 17 00:00:00 2001 From: feng lv Date: Thu, 25 Feb 2021 07:47:08 +0000 Subject: [PATCH 380/510] union distinct improve --- .../InterpreterSelectWithUnionQuery.cpp | 117 ------------------ .../NormalizeSelectWithUnionQueryVisitor.cpp | 107 ++++++++++++++++ .../NormalizeSelectWithUnionQueryVisitor.h | 34 +++++ src/Interpreters/executeQuery.cpp | 28 +++-- 4 files changed, 162 insertions(+), 124 deletions(-) create mode 100644 src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp create mode 100644 src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 59fcff61936..62dc843f982 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -24,110 +24,8 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int UNION_ALL_RESULT_STRUCTURES_MISMATCH; - extern const int EXPECTED_ALL_OR_DISTINCT; } -struct CustomizeASTSelectWithUnionQueryNormalize -{ - using TypeToVisit = ASTSelectWithUnionQuery; - - const UnionMode & union_default_mode; - - static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) - { - if (auto * inner_union = ast_select->as()) - { - for (auto & child : inner_union->list_of_selects->children) - getSelectsFromUnionListNode(child, selects); - - return; - } - - selects.push_back(std::move(ast_select)); - } - - void visit(ASTSelectWithUnionQuery & ast, ASTPtr &) const - { - auto & union_modes = ast.list_of_modes; - ASTs selects; - auto & select_list = ast.list_of_selects->children; - - int i; - for (i = union_modes.size() - 1; i >= 0; --i) - { - /// Rewrite UNION Mode - if (union_modes[i] == ASTSelectWithUnionQuery::Mode::Unspecified) - { - if (union_default_mode == UnionMode::ALL) - union_modes[i] = ASTSelectWithUnionQuery::Mode::ALL; - else if (union_default_mode == UnionMode::DISTINCT) - union_modes[i] = ASTSelectWithUnionQuery::Mode::DISTINCT; - else - throw Exception( - "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", - DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); - } - - if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL) - { - if (auto * inner_union = select_list[i + 1]->as()) - { - /// Inner_union is an UNION ALL list, just lift up - for (auto child = inner_union->list_of_selects->children.rbegin(); - child != inner_union->list_of_selects->children.rend(); - ++child) - selects.push_back(std::move(*child)); - } - else - selects.push_back(std::move(select_list[i + 1])); - } - /// flatten all left nodes and current node to a UNION DISTINCT list - else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT) - { - auto distinct_list = std::make_shared(); - distinct_list->list_of_selects = std::make_shared(); - distinct_list->children.push_back(distinct_list->list_of_selects); - - for (int j = 0; j <= i + 1; ++j) - { - getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); - } - - distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT; - distinct_list->is_normalized = true; - selects.push_back(std::move(distinct_list)); - break; - } - } - - /// No UNION DISTINCT or only one child in select_list - if (i == -1) - { - if (auto * inner_union = select_list[0]->as()) - { - /// Inner_union is an UNION ALL list, just lift it up - for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); - ++child) - selects.push_back(std::move(*child)); - } - else - selects.push_back(std::move(select_list[0])); - } - - // reverse children list - std::reverse(selects.begin(), selects.end()); - - ast.is_normalized = true; - ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; - - ast.list_of_selects->children = std::move(selects); - } -}; - -/// We need normalize children first, so we should visit AST tree bottom up -using CustomizeASTSelectWithUnionQueryNormalizeVisitor - = InDepthNodeVisitor, false>; - InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( const ASTPtr & query_ptr_, const Context & context_, const SelectQueryOptions & options_, const Names & required_result_column_names) : IInterpreterUnionOrSelectQuery(query_ptr_, context_, options_) @@ -138,21 +36,6 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( if (options.subquery_depth == 0 && (settings.limit > 0 || settings.offset > 0)) settings_limit_offset_needed = true; - /// Normalize AST Tree - if (!ast->is_normalized) - { - CustomizeASTSelectWithUnionQueryNormalizeVisitor::Data union_default_mode{settings.union_default_mode}; - CustomizeASTSelectWithUnionQueryNormalizeVisitor(union_default_mode).visit(query_ptr); - - /// After normalization, if it only has one ASTSelectWithUnionQuery child, - /// we can lift it up, this can reduce one unnecessary recursion later. - if (ast->list_of_selects->children.size() == 1 && ast->list_of_selects->children.at(0)->as()) - { - query_ptr = std::move(ast->list_of_selects->children.at(0)); - ast = query_ptr->as(); - } - } - size_t num_children = ast->list_of_selects->children.size(); if (!num_children) throw Exception("Logical error: no children in ASTSelectWithUnionQuery", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp new file mode 100644 index 00000000000..31d33d0781a --- /dev/null +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -0,0 +1,107 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int EXPECTED_ALL_OR_DISTINCT; +} + +void NormalizeSelectWithUnionQueryMatcher::getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects) +{ + if (auto * inner_union = ast_select->as()) + { + for (auto & child : inner_union->list_of_selects->children) + getSelectsFromUnionListNode(child, selects); + + return; + } + + selects.push_back(std::move(ast_select)); +} + +void NormalizeSelectWithUnionQueryMatcher::visit(ASTPtr & ast, Data & data) +{ + if (auto * select_union = ast->as()) + visit(*select_union, data); +} + +void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, Data & data) +{ + auto & union_modes = ast.list_of_modes; + ASTs selects; + auto & select_list = ast.list_of_selects->children; + + int i; + for (i = union_modes.size() - 1; i >= 0; --i) + { + /// Rewrite UNION Mode + if (union_modes[i] == ASTSelectWithUnionQuery::Mode::Unspecified) + { + if (data.union_default_mode == UnionMode::ALL) + union_modes[i] = ASTSelectWithUnionQuery::Mode::ALL; + else if (data.union_default_mode == UnionMode::DISTINCT) + union_modes[i] = ASTSelectWithUnionQuery::Mode::DISTINCT; + else + throw Exception( + "Expected ALL or DISTINCT in SelectWithUnion query, because setting (union_default_mode) is empty", + DB::ErrorCodes::EXPECTED_ALL_OR_DISTINCT); + } + + if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL) + { + if (auto * inner_union = select_list[i + 1]->as()) + { + /// Inner_union is an UNION ALL list, just lift up + for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); + ++child) + selects.push_back(std::move(*child)); + } + else + selects.push_back(std::move(select_list[i + 1])); + } + /// flatten all left nodes and current node to a UNION DISTINCT list + else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT) + { + auto distinct_list = std::make_shared(); + distinct_list->list_of_selects = std::make_shared(); + distinct_list->children.push_back(distinct_list->list_of_selects); + + for (int j = 0; j <= i + 1; ++j) + { + getSelectsFromUnionListNode(select_list[j], distinct_list->list_of_selects->children); + } + + distinct_list->union_mode = ASTSelectWithUnionQuery::Mode::DISTINCT; + distinct_list->is_normalized = true; + selects.push_back(std::move(distinct_list)); + break; + } + } + + /// No UNION DISTINCT or only one child in select_list + if (i == -1) + { + if (auto * inner_union = select_list[0]->as()) + { + /// Inner_union is an UNION ALL list, just lift it up + for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); + ++child) + selects.push_back(std::move(*child)); + } + else + selects.push_back(std::move(select_list[0])); + } + + // reverse children list + std::reverse(selects.begin(), selects.end()); + + ast.is_normalized = true; + ast.union_mode = ASTSelectWithUnionQuery::Mode::ALL; + + ast.list_of_selects->children = std::move(selects); +} +} diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h new file mode 100644 index 00000000000..cec2e4265e2 --- /dev/null +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.h @@ -0,0 +1,34 @@ +#pragma once + +#include + +#include +#include + +#include +#include + +namespace DB +{ + +class ASTFunction; + +class NormalizeSelectWithUnionQueryMatcher +{ +public: + struct Data + { + const UnionMode & union_default_mode; + }; + + static void getSelectsFromUnionListNode(ASTPtr & ast_select, ASTs & selects); + + static void visit(ASTPtr & ast, Data &); + static void visit(ASTSelectWithUnionQuery &, Data &); + static bool needChildVisit(const ASTPtr &, const ASTPtr &) { return true; } +}; + +/// We need normalize children first, so we should visit AST tree bottom up +using NormalizeSelectWithUnionQueryVisitor + = InDepthNodeVisitor; +} diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index d786e1146be..3d6332b45a0 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -39,16 +39,17 @@ #include #include -#include -#include -#include -#include -#include #include +#include +#include +#include +#include +#include +#include +#include #include #include #include -#include #include #include @@ -475,6 +476,19 @@ static std::tuple executeQueryImpl( query = serializeAST(*ast); } + /// Normalize SelectWithUnionQuery + NormalizeSelectWithUnionQueryVisitor::Data data{context.getSettingsRef().union_default_mode}; + NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); + + /// After normalization, if it only has one ASTSelectWithUnionQuery child, + /// we can lift it up, this can reduce one unnecessary recursion later in interpreter phase + auto select_union = ast->as(); + if (select_union && select_union->list_of_selects->children.size() == 1 + && select_union->list_of_selects->children.at(0)->as()) + ast = std::move(select_union->list_of_selects->children.at(0)); + + query = serializeAST(*ast); + /// Check the limits. checkASTSizeLimits(*ast, settings); @@ -874,7 +888,7 @@ static std::tuple executeQueryImpl( LOG_DEBUG(&Poco::Logger::get("executeQuery"), "Query pipeline:\n{}", msg_buf.str()); } } - } + } catch (...) { if (!internal) From a9f7ab15f6bce95b93f39f77e8f71a0141d64a7c Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 Feb 2021 08:44:26 +0000 Subject: [PATCH 381/510] Empty commit From 2f70e895fccf67eadbc8ba5a2e5e4b7673037780 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Feb 2021 12:31:22 +0300 Subject: [PATCH 382/510] Update StorageJoin locking Move joinGet into StorageJoin Protect JoinSource with lock, add test Add comments about locking logic --- src/Functions/FunctionJoinGet.cpp | 12 +++--- src/Functions/FunctionJoinGet.h | 16 ++++---- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/HashJoin.cpp | 29 +++++--------- src/Interpreters/HashJoin.h | 8 ++-- src/Storages/StorageJoin.cpp | 38 +++++++++++++------ src/Storages/StorageJoin.h | 25 ++++-------- ...ace_condition_storage_join_long.reference} | 0 ...01732_race_condition_storage_join_long.sh} | 11 ++++++ 9 files changed, 73 insertions(+), 68 deletions(-) rename tests/queries/0_stateless/{01732_race_condition_storage_long.reference => 01732_race_condition_storage_join_long.reference} (100%) rename tests/queries/0_stateless/{01732_race_condition_storage_long.sh => 01732_race_condition_storage_join_long.sh} (83%) diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index 3a2649c11a8..a2e4e2d1790 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -25,16 +25,17 @@ ColumnPtr ExecutableFunctionJoinGet::execute(const ColumnsWithTypeAndNa auto key = arguments[i]; keys.emplace_back(std::move(key)); } - return join->join->joinGet(keys, result_columns).column; + return storage_join->joinGet(keys, result_columns).column; } template ExecutableFunctionImplPtr FunctionJoinGet::prepare(const ColumnsWithTypeAndName &) const { - return std::make_unique>(join, DB::Block{{return_type->createColumn(), return_type, attr_name}}); + return std::make_unique>(storage_join, DB::Block{{return_type->createColumn(), return_type, attr_name}}); } -static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) +static std::pair, String> +getJoin(const ColumnsWithTypeAndName & arguments, const Context & context) { String join_name; if (const auto * name_col = checkAndGetColumnConst(arguments[0].column.get())) @@ -87,13 +88,12 @@ FunctionBaseImplPtr JoinGetOverloadResolver::build(const ColumnsWithTyp + ", should be greater or equal to 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); auto [storage_join, attr_name] = getJoin(arguments, context); - auto join_holder = storage_join->getJoin(); DataTypes data_types(arguments.size() - 2); for (size_t i = 2; i < arguments.size(); ++i) data_types[i - 2] = arguments[i].type; - auto return_type = join_holder->join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); + auto return_type = storage_join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null); auto table_lock = storage_join->lockForShare(context.getInitialQueryId(), context.getSettingsRef().lock_acquire_timeout); - return std::make_unique>(table_lock, join_holder, attr_name, data_types, return_type); + return std::make_unique>(table_lock, storage_join, attr_name, data_types, return_type); } void registerFunctionJoinGet(FunctionFactory & factory) diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index 820c6cd3fa2..f7d3858e902 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -9,15 +9,15 @@ namespace DB class Context; class HashJoin; -class HashJoinHolder; -using HashJoinPtr = std::shared_ptr; +class StorageJoin; +using StorageJoinPtr = std::shared_ptr; template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(std::shared_ptr join_, const DB::Block & result_columns_) - : join(std::move(join_)), result_columns(result_columns_) {} + ExecutableFunctionJoinGet(StorageJoinPtr storage_join_, const DB::Block & result_columns_) + : storage_join(std::move(storage_join_)), result_columns(result_columns_) {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; @@ -30,7 +30,7 @@ public: String getName() const override { return name; } private: - std::shared_ptr join; + StorageJoinPtr storage_join; DB::Block result_columns; }; @@ -41,10 +41,10 @@ public: static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; FunctionJoinGet(TableLockHolder table_lock_, - std::shared_ptr join_, String attr_name_, + StorageJoinPtr storage_join_, String attr_name_, DataTypes argument_types_, DataTypePtr return_type_) : table_lock(std::move(table_lock_)) - , join(join_) + , storage_join(storage_join_) , attr_name(std::move(attr_name_)) , argument_types(std::move(argument_types_)) , return_type(std::move(return_type_)) @@ -60,7 +60,7 @@ public: private: TableLockHolder table_lock; - std::shared_ptr join; + StorageJoinPtr storage_join; const String attr_name; DataTypes argument_types; DataTypePtr return_type; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 660718549b3..55dc622a976 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -739,7 +739,7 @@ static JoinPtr tryGetStorageJoin(std::shared_ptr analyzed_join) { if (auto * table = analyzed_join->joined_storage.get()) if (auto * storage_join = dynamic_cast(table)) - return storage_join->getJoin(analyzed_join); + return storage_join->getJoinLocked(analyzed_join); return {}; } diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 9c64b9522b9..942be9d172d 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1203,7 +1203,6 @@ void HashJoin::joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) block = block.cloneWithColumns(std::move(dst_columns)); } - DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const { size_t num_keys = data_types.size(); @@ -1235,10 +1234,15 @@ DataTypePtr HashJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, return elem.type; } - -template -ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const +/// TODO: return multiple columns as named tuple +/// TODO: return array of values when strictness == ASTTableJoin::Strictness::All +ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const { + bool is_valid = (strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) + && kind == ASTTableJoin::Kind::Left; + if (!is_valid) + throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); + /// Assemble the key block with correct names. Block keys; for (size_t i = 0; i < block.columns(); ++i) @@ -1249,25 +1253,10 @@ ColumnWithTypeAndName HashJoin::joinGetImpl(const Block & block, const Block & b } joinBlockImpl( - keys, key_names_right, block_with_columns_to_add, maps_); + keys, key_names_right, block_with_columns_to_add, std::get(data->maps)); return keys.getByPosition(keys.columns() - 1); } - -/// TODO: return multiple columns as named tuple -/// TODO: return array of values when strictness == ASTTableJoin::Strictness::All -ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const -{ - if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && - kind == ASTTableJoin::Kind::Left) - { - return joinGetImpl(block, block_with_columns_to_add, std::get(data->maps)); - } - else - throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::INCOMPATIBLE_TYPE_OF_JOIN); -} - - void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) { const Names & key_names_left = table_join->keyNamesLeft(); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index d212e16b175..075634b348d 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -318,6 +318,8 @@ public: Arena pool; }; + /// We keep correspondence between used_flags and hash table internal buffer. + /// Hash table cannot be modified during HashJoin lifetime and must be protected with lock. void setLock(std::shared_mutex & rwlock) { storage_join_lock = std::shared_lock(rwlock); @@ -354,6 +356,8 @@ private: /// Flags that indicate that particular row already used in join. /// Flag is stored for every record in hash map. /// Number of this flags equals to hashtable buffer size (plus one for zero value). + /// Changes in hash table broke correspondence, + /// so we must guarantee constantness of hash table during HashJoin lifetime (using method setLock) mutable JoinStuff::JoinUsedFlags used_flags; Sizes key_sizes; @@ -372,6 +376,7 @@ private: Block totals; + /// Should be set via setLock to protect hash table from modification from StorageJoin std::shared_lock storage_join_lock; void init(Type type_); @@ -391,9 +396,6 @@ private: void joinBlockImplCross(Block & block, ExtraBlockPtr & not_processed) const; - template - ColumnWithTypeAndName joinGetImpl(const Block & block, const Block & block_with_columns_to_add, const Maps & maps_) const; - static Type chooseMethod(const ColumnRawPtrs & key_columns, Sizes & key_sizes); bool empty() const; diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp index bfe866ed223..a449cebba51 100644 --- a/src/Storages/StorageJoin.cpp +++ b/src/Storages/StorageJoin.cpp @@ -79,7 +79,7 @@ void StorageJoin::truncate( } -HashJoinPtr StorageJoin::getJoin(std::shared_ptr analyzed_join) const +HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) const { auto metadata_snapshot = getInMemoryMetadataPtr(); if (!analyzed_join->sameStrictnessAndKind(strictness, kind)) @@ -127,6 +127,16 @@ std::optional StorageJoin::totalBytes(const Settings &) const return join->getTotalByteCount(); } +DataTypePtr StorageJoin::joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const +{ + return join->joinGetCheckAndGetReturnType(data_types, column_name, or_null); +} + +ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const +{ + std::shared_lock lock(rwlock); + return join->joinGet(block, block_with_columns_to_add); +} void registerStorageJoin(StorageFactory & factory) { @@ -284,23 +294,24 @@ size_t rawSize(const StringRef & t) class JoinSource : public SourceWithProgress { public: - JoinSource(const HashJoin & parent_, UInt64 max_block_size_, Block sample_block_) + JoinSource(HashJoinPtr join_, std::shared_mutex & rwlock, UInt64 max_block_size_, Block sample_block_) : SourceWithProgress(sample_block_) - , parent(parent_) + , join(join_) + , lock(rwlock) , max_block_size(max_block_size_) , sample_block(std::move(sample_block_)) { column_indices.resize(sample_block.columns()); - auto & saved_block = parent.getJoinedData()->sample_block; + auto & saved_block = join->getJoinedData()->sample_block; for (size_t i = 0; i < sample_block.columns(); ++i) { auto & [_, type, name] = sample_block.getByPosition(i); - if (parent.right_table_keys.has(name)) + if (join->right_table_keys.has(name)) { key_pos = i; - const auto & column = parent.right_table_keys.getByName(name); + const auto & column = join->right_table_keys.getByName(name); restored_block.insert(column); } else @@ -319,18 +330,20 @@ public: protected: Chunk generate() override { - if (parent.data->blocks.empty()) + if (join->data->blocks.empty()) return {}; Chunk chunk; - if (!joinDispatch(parent.kind, parent.strictness, parent.data->maps, + if (!joinDispatch(join->kind, join->strictness, join->data->maps, [&](auto kind, auto strictness, auto & map) { chunk = createChunk(map); })) throw Exception("Logical error: unknown JOIN strictness", ErrorCodes::LOGICAL_ERROR); return chunk; } private: - const HashJoin & parent; + HashJoinPtr join; + std::shared_lock lock; + UInt64 max_block_size; Block sample_block; Block restored_block; /// sample_block with parent column types @@ -348,7 +361,7 @@ private: size_t rows_added = 0; - switch (parent.data->type) + switch (join->data->type) { #define M(TYPE) \ case HashJoin::Type::TYPE: \ @@ -358,7 +371,7 @@ private: #undef M default: - throw Exception("Unsupported JOIN keys in StorageJoin. Type: " + toString(static_cast(parent.data->type)), + throw Exception("Unsupported JOIN keys in StorageJoin. Type: " + toString(static_cast(join->data->type)), ErrorCodes::UNSUPPORTED_JOIN_KEYS); } @@ -486,7 +499,8 @@ Pipe StorageJoin::read( { metadata_snapshot->check(column_names, getVirtuals(), getStorageID()); - return Pipe(std::make_shared(*join, max_block_size, metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()))); + Block source_sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID()); + return Pipe(std::make_shared(join, rwlock, max_block_size, source_sample_block)); } } diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 7e4dea5d223..49f2be825b2 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -14,19 +14,6 @@ class TableJoin; class HashJoin; using HashJoinPtr = std::shared_ptr; -class HashJoinHolder -{ - std::shared_lock lock; -public: - HashJoinPtr join; - - HashJoinHolder(std::shared_mutex & rwlock, HashJoinPtr join_) - : lock(rwlock) - , join(join_) - { - } -}; - /** Allows you save the state for later use on the right side of the JOIN. * When inserted into a table, the data will be inserted into the state, * and also written to the backup file, to restore after the restart. @@ -42,9 +29,12 @@ public: void truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, const Context &, TableExclusiveLockHolder &) override; - /// Access the innards. - std::shared_ptr getJoin() { return std::make_shared(rwlock, join); } - HashJoinPtr getJoin(std::shared_ptr analyzed_join) const; + /// Return instance of HashJoin holding lock that protects from insertions to StorageJoin. + /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. + HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join) const; + + DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; + ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; Pipe read( const Names & column_names, @@ -71,8 +61,7 @@ private: HashJoinPtr join; /// Protect state for concurrent use in insertFromBlock and joinBlock. - /// Lock hold via HashJoin instance (or HashJoinHolder for joinGet) - /// during all query and block insertions. + /// Lock is stored in HashJoin instance during query and blocks concurrent insertions. mutable std::shared_mutex rwlock; void insertBlock(const Block & block) override; diff --git a/tests/queries/0_stateless/01732_race_condition_storage_long.reference b/tests/queries/0_stateless/01732_race_condition_storage_join_long.reference similarity index 100% rename from tests/queries/0_stateless/01732_race_condition_storage_long.reference rename to tests/queries/0_stateless/01732_race_condition_storage_join_long.reference diff --git a/tests/queries/0_stateless/01732_race_condition_storage_long.sh b/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh similarity index 83% rename from tests/queries/0_stateless/01732_race_condition_storage_long.sh rename to tests/queries/0_stateless/01732_race_condition_storage_join_long.sh index 0ce6f2b0c7a..b7dd76760d4 100755 --- a/tests/queries/0_stateless/01732_race_condition_storage_long.sh +++ b/tests/queries/0_stateless/01732_race_condition_storage_join_long.sh @@ -32,14 +32,25 @@ function read_thread_small() done } +function read_thread_select() +{ + while true; do + echo " + SELECT * FROM storage_join_race FORMAT Null; + " | $CLICKHOUSE_CLIENT -n + done +} + # https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout export -f read_thread_big; export -f read_thread_small; +export -f read_thread_select; TIMEOUT=20 timeout $TIMEOUT bash -c read_thread_big 2> /dev/null & timeout $TIMEOUT bash -c read_thread_small 2> /dev/null & +timeout $TIMEOUT bash -c read_thread_select 2> /dev/null & echo " INSERT INTO storage_join_race SELECT number AS x, number AS y FROM numbers (10000000); From f1c60109f69676e0a200d218a24714200d51a4b9 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Feb 2021 12:42:20 +0300 Subject: [PATCH 383/510] Add static_assert to joinGet to keep information about current locking logic --- src/Interpreters/HashJoin.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 942be9d172d..c97c746cbf1 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1252,6 +1252,8 @@ ColumnWithTypeAndName HashJoin::joinGet(const Block & block, const Block & block keys.insert(std::move(key)); } + static_assert(!MapGetter::flagged, + "joinGet are not protected from hash table changes between block processing"); joinBlockImpl( keys, key_names_right, block_with_columns_to_add, std::get(data->maps)); return keys.getByPosition(keys.columns() - 1); From 5c6c318737ab9fb8fbcc2f853a76f0c69ea9d173 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Feb 2021 13:07:48 +0300 Subject: [PATCH 384/510] Restrict mutations for engines which doesn't support them --- src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Storages/IStorage.cpp | 5 ++++ src/Storages/IStorage.h | 5 ++++ src/Storages/MergeTree/MergeTreeData.cpp | 6 ++++ src/Storages/MergeTree/MergeTreeData.h | 4 +++ src/Storages/StorageMaterializedView.cpp | 6 ++++ src/Storages/StorageMaterializedView.h | 2 ++ src/Storages/StorageMemory.cpp | 5 ++++ src/Storages/StorageMemory.h | 1 + .../01745_alter_delete_view.reference | 4 +++ .../0_stateless/01745_alter_delete_view.sql | 28 +++++++++++++++++++ 11 files changed, 67 insertions(+) create mode 100644 tests/queries/0_stateless/01745_alter_delete_view.reference create mode 100644 tests/queries/0_stateless/01745_alter_delete_view.sql diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index bf624507574..6294b31cc8c 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -104,6 +104,7 @@ BlockIO InterpreterAlterQuery::execute() if (!mutation_commands.empty()) { + table->checkMutationIsPossible(mutation_commands, context.getSettingsRef()); MutationsInterpreter(table, metadata_snapshot, mutation_commands, context, false).validate(); table->mutate(mutation_commands, context); } diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 5f500518516..2400b0587ba 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -145,6 +145,11 @@ void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settin } } +void IStorage::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const +{ + throw Exception("Table engine " + getName() + " doesn't support mutations", ErrorCodes::NOT_IMPLEMENTED); +} + void IStorage::checkAlterPartitionIsPossible( const PartitionCommands & /*commands*/, const StorageMetadataPtr & /*metadata_snapshot*/, const Settings & /*settings*/) const { diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 651688f41bb..1a27dbd637f 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -364,6 +364,11 @@ public: */ virtual void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const; + /** + * Checks that mutation commands can be applied to storage. + */ + virtual void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const; + /** ALTER tables with regard to its partitions. * Should handle locks for each command on its own. */ diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index b09f068f509..2d841b98c59 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1670,6 +1670,12 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S } } + +void MergeTreeData::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const +{ + /// Some validation will be added +} + MergeTreeDataPartType MergeTreeData::choosePartType(size_t bytes_uncompressed, size_t rows_count) const { const auto settings = getSettings(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2aefa66ac58..09cf017d220 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -519,6 +519,10 @@ public: /// If something is wrong, throws an exception. void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; + /// Checks if the Mutation can be performed. + /// (currenly no additional checks: always ok) + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; + /// Checks that partition name in all commands is valid void checkAlterPartitionIsPossible(const PartitionCommands & commands, const StorageMetadataPtr & metadata_snapshot, const Settings & settings) const override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index 325bf3d2f74..2d211c8061b 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -320,6 +320,12 @@ void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & command } } +void StorageMaterializedView::checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const +{ + checkStatementCanBeForwarded(); + getTargetTable()->checkMutationIsPossible(commands, settings); +} + Pipe StorageMaterializedView::alterPartition( const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) { diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h index 94e4295cd34..4b10cf7a9b5 100644 --- a/src/Storages/StorageMaterializedView.h +++ b/src/Storages/StorageMaterializedView.h @@ -52,6 +52,8 @@ public: void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override; + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; + void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override; diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index d7b0ae055ab..d98cd4212e9 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -253,6 +253,11 @@ static inline void updateBlockData(Block & old_block, const Block & new_block) } } +void StorageMemory::checkMutationIsPossible(const MutationCommands & /*commands*/, const Settings & /*settings*/) const +{ + /// Some validation will be added +} + void StorageMemory::mutate(const MutationCommands & commands, const Context & context) { std::lock_guard lock(mutex); diff --git a/src/Storages/StorageMemory.h b/src/Storages/StorageMemory.h index db71c13ca99..b7fa4d7b222 100644 --- a/src/Storages/StorageMemory.h +++ b/src/Storages/StorageMemory.h @@ -51,6 +51,7 @@ public: void drop() override; + void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; void mutate(const MutationCommands & commands, const Context & context) override; void truncate(const ASTPtr &, const StorageMetadataPtr &, const Context &, TableExclusiveLockHolder &) override; diff --git a/tests/queries/0_stateless/01745_alter_delete_view.reference b/tests/queries/0_stateless/01745_alter_delete_view.reference new file mode 100644 index 00000000000..dc3ab50ab0d --- /dev/null +++ b/tests/queries/0_stateless/01745_alter_delete_view.reference @@ -0,0 +1,4 @@ +1 1 +2 1 +1 1 +2 1 diff --git a/tests/queries/0_stateless/01745_alter_delete_view.sql b/tests/queries/0_stateless/01745_alter_delete_view.sql new file mode 100644 index 00000000000..c242f1be63e --- /dev/null +++ b/tests/queries/0_stateless/01745_alter_delete_view.sql @@ -0,0 +1,28 @@ +DROP VIEW IF EXISTS test_view; +DROP TABLE IF EXISTS test_table; + +CREATE TABLE test_table +( + f1 Int32, + f2 Int32, + pk Int32 +) +ENGINE = MergeTree() +ORDER BY f1 +PARTITION BY pk; + +CREATE VIEW test_view AS +SELECT f1, f2 +FROM test_table +WHERE pk = 2; + +INSERT INTO test_table (f1, f2, pk) VALUES (1,1,1), (1,1,2), (2,1,1), (2,1,2); + +SELECT * FROM test_view ORDER BY f1, f2; + +ALTER TABLE test_view DELETE WHERE pk = 2; --{serverError 48} + +SELECT * FROM test_view ORDER BY f1, f2; + +DROP VIEW IF EXISTS test_view; +DROP TABLE IF EXISTS test_table; From 33364f6bdf15fe164321bff7da0fdd9e000a3947 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 25 Feb 2021 13:50:35 +0300 Subject: [PATCH 385/510] Remove debug output --- src/Processors/Formats/LazyOutputFormat.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 7188458dd82..15ea5022f82 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -38,7 +38,6 @@ public: void finalize() override { - std::cerr << StackTrace().toString() << std::endl; finished_processing = true; /// In case we are waiting for result. From 70b8d6d5657efe0157babeabf107453dad07e42a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Feb 2021 14:19:26 +0300 Subject: [PATCH 386/510] Update encryption-functions.md --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index e2c5560e4f6..4388f327eab 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -41,7 +41,7 @@ encrypt('mode', 'plaintext', 'key' [, iv, aad]) **Возвращаемое значение** -- Двоичная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). +- Бинарная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). **Примеры** From d4e48f44f5e69d553e4c67f8a158307206cca132 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 25 Feb 2021 14:20:42 +0300 Subject: [PATCH 387/510] Update encryption-functions.md --- docs/ru/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/encryption-functions.md b/docs/ru/sql-reference/functions/encryption-functions.md index 4388f327eab..0216a6b2356 100644 --- a/docs/ru/sql-reference/functions/encryption-functions.md +++ b/docs/ru/sql-reference/functions/encryption-functions.md @@ -136,7 +136,7 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv]) **Возвращаемое значение** -- Двоичная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). +- Бинарная зашифрованная строка. [String](../../sql-reference/data-types/string.md#string). **Примеры** From 6f864ddb424b9b985e535d817ee35b13fd94f75d Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 25 Feb 2021 14:21:06 +0300 Subject: [PATCH 388/510] Minor changes in Hash/StorageJoin, add TableLockHolder to ExecutableFunctionJoinGet --- src/Functions/FunctionJoinGet.cpp | 3 ++- src/Functions/FunctionJoinGet.h | 10 ++++++++-- src/Interpreters/HashJoin.cpp | 4 +++- src/Interpreters/HashJoin.h | 1 + src/Storages/StorageJoin.h | 5 +++++ 5 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp index a2e4e2d1790..49f155c687b 100644 --- a/src/Functions/FunctionJoinGet.cpp +++ b/src/Functions/FunctionJoinGet.cpp @@ -31,7 +31,8 @@ ColumnPtr ExecutableFunctionJoinGet::execute(const ColumnsWithTypeAndNa template ExecutableFunctionImplPtr FunctionJoinGet::prepare(const ColumnsWithTypeAndName &) const { - return std::make_unique>(storage_join, DB::Block{{return_type->createColumn(), return_type, attr_name}}); + Block result_columns {{return_type->createColumn(), return_type, attr_name}}; + return std::make_unique>(table_lock, storage_join, result_columns); } static std::pair, String> diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h index f7d3858e902..f88b58e961a 100644 --- a/src/Functions/FunctionJoinGet.h +++ b/src/Functions/FunctionJoinGet.h @@ -16,8 +16,13 @@ template class ExecutableFunctionJoinGet final : public IExecutableFunctionImpl { public: - ExecutableFunctionJoinGet(StorageJoinPtr storage_join_, const DB::Block & result_columns_) - : storage_join(std::move(storage_join_)), result_columns(result_columns_) {} + ExecutableFunctionJoinGet(TableLockHolder table_lock_, + StorageJoinPtr storage_join_, + const DB::Block & result_columns_) + : table_lock(std::move(table_lock_)) + , storage_join(std::move(storage_join_)) + , result_columns(result_columns_) + {} static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet"; @@ -30,6 +35,7 @@ public: String getName() const override { return name; } private: + TableLockHolder table_lock; StorageJoinPtr storage_join; DB::Block result_columns; }; diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index c97c746cbf1..7d8e78a4e7e 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -639,7 +639,9 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits) size_t total_bytes = 0; { - assert(storage_join_lock.mutex() == nullptr); + if (storage_join_lock.mutex()) + throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates", + ErrorCodes::LOGICAL_ERROR); data->blocks.emplace_back(std::move(structured_block)); Block * stored_block = &data->blocks.back(); diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h index 075634b348d..b726de44f3a 100644 --- a/src/Interpreters/HashJoin.h +++ b/src/Interpreters/HashJoin.h @@ -377,6 +377,7 @@ private: Block totals; /// Should be set via setLock to protect hash table from modification from StorageJoin + /// If set HashJoin instance is not available for modification (addJoinedBlock) std::shared_lock storage_join_lock; void init(Type type_); diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h index 49f2be825b2..5f0f9f92404 100644 --- a/src/Storages/StorageJoin.h +++ b/src/Storages/StorageJoin.h @@ -33,7 +33,12 @@ public: /// HashJoin relies on structure of hash table that's why we need to return it with locked mutex. HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join) const; + /// Get result type for function "joinGet(OrNull)" DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const; + + /// Execute function "joinGet(OrNull)" on data block. + /// Takes rwlock for read to prevent parallel StorageJoin updates during processing data block + /// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin) ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const; Pipe read( From 8fec34af12f6d3373c53e8792a0ecb49bf983ac3 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 19 Feb 2021 23:00:33 +0300 Subject: [PATCH 389/510] Constraints complex types support --- .../CheckConstraintsBlockOutputStream.cpp | 114 +++++++++--------- .../01720_constraints_complex_types.reference | 3 + .../01720_constraints_complex_types.sql | 47 ++++++++ ...constraints_constant_expressions.reference | 1 + ...01721_constraints_constant_expressions.sql | 27 +++++ 5 files changed, 136 insertions(+), 56 deletions(-) create mode 100644 tests/queries/0_stateless/01720_constraints_complex_types.reference create mode 100644 tests/queries/0_stateless/01720_constraints_complex_types.sql create mode 100644 tests/queries/0_stateless/01721_constraints_constant_expressions.reference create mode 100644 tests/queries/0_stateless/01721_constraints_constant_expressions.sql diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index a967ee28502..84c31de99b4 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -1,12 +1,15 @@ -#include -#include -#include -#include -#include -#include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include +#include namespace DB @@ -15,7 +18,7 @@ namespace DB namespace ErrorCodes { extern const int VIOLATED_CONSTRAINT; - extern const int LOGICAL_ERROR; + extern const int UNSUPPORTED_METHOD; } @@ -48,62 +51,61 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName()); - if (!isUInt8(res_column.type)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Constraint {} does not return a value of type UInt8", + auto result_type = removeNullable(removeLowCardinality(res_column.type)); + auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality(); + + if (const auto * column_nullable = checkAndGetColumn(*result_column)) + result_column = column_nullable->getNestedColumnPtr(); + + if (!isUInt8(result_type)) + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Constraint {} does not return a value of type UInt8", backQuote(constraint_ptr->name)); - if (const ColumnConst * res_const = typeid_cast(res_column.column.get())) + const ColumnUInt8 & res_column_uint8 = assert_cast(*result_column); + + const UInt8 * data = res_column_uint8.getData().data(); + size_t size = res_column_uint8.size(); + + std::cerr << std::endl; + for (size_t j = 0; j < size; ++j) + std::cerr << data[j] << " "; + std::cerr << std::endl; + + /// Is violated. + if (!memoryIsByte(data, size, 1)) { - UInt8 value = res_const->getValue(); + size_t row_idx = 0; + for (; row_idx < size; ++row_idx) + if (data[row_idx] != 1) + break; - /// Is violated. - if (!value) + Names related_columns = constraint_expr->getRequiredColumns(); + + bool first = true; + String column_values_msg; + constexpr size_t approx_bytes_for_col = 32; + column_values_msg.reserve(approx_bytes_for_col * related_columns.size()); + for (const auto & name : related_columns) { - throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, - "Constraint {} for table {} is violated, because it is a constant expression returning 0. " - "It is most likely an error in table definition.", - backQuote(constraint_ptr->name), table_id.getNameForLogs()); + const IColumn & column = *block.getByName(name).column; + assert(row_idx < column.size()); + + if (!first) + column_values_msg.append(", "); + column_values_msg.append(backQuoteIfNeed(name)); + column_values_msg.append(" = "); + column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx])); + first = false; } - } - else - { - const ColumnUInt8 & res_column_uint8 = assert_cast(*res_column.column); - const UInt8 * data = res_column_uint8.getData().data(); - size_t size = res_column_uint8.size(); - - /// Is violated. - if (!memoryIsByte(data, size, 1)) - { - size_t row_idx = 0; - for (; row_idx < size; ++row_idx) - if (data[row_idx] != 1) - break; - - Names related_columns = constraint_expr->getRequiredColumns(); - - bool first = true; - String column_values_msg; - constexpr size_t approx_bytes_for_col = 32; - column_values_msg.reserve(approx_bytes_for_col * related_columns.size()); - for (const auto & name : related_columns) - { - const IColumn & column = *block.getByName(name).column; - assert(row_idx < column.size()); - - if (!first) - column_values_msg.append(", "); - column_values_msg.append(backQuoteIfNeed(name)); - column_values_msg.append(" = "); - column_values_msg.append(applyVisitor(FieldVisitorToString(), column[row_idx])); - first = false; - } - - throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, - "Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}", - backQuote(constraint_ptr->name), table_id.getNameForLogs(), rows_written + row_idx + 1, - serializeAST(*(constraint_ptr->expr), true), column_values_msg); - } + throw Exception( + ErrorCodes::VIOLATED_CONSTRAINT, + "Constraint {} for table {} is violated at row {}. Expression: ({}). Column values: {}", + backQuote(constraint_ptr->name), + table_id.getNameForLogs(), + rows_written + row_idx + 1, + serializeAST(*(constraint_ptr->expr), true), + column_values_msg); } } } diff --git a/tests/queries/0_stateless/01720_constraints_complex_types.reference b/tests/queries/0_stateless/01720_constraints_complex_types.reference new file mode 100644 index 00000000000..01e79c32a8c --- /dev/null +++ b/tests/queries/0_stateless/01720_constraints_complex_types.reference @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/tests/queries/0_stateless/01720_constraints_complex_types.sql b/tests/queries/0_stateless/01720_constraints_complex_types.sql new file mode 100644 index 00000000000..7e400c4931e --- /dev/null +++ b/tests/queries/0_stateless/01720_constraints_complex_types.sql @@ -0,0 +1,47 @@ +DROP TABLE IF EXISTS constraint_on_nullable_type; +CREATE TABLE constraint_on_nullable_type +( + `id` Nullable(UInt64), + CONSTRAINT `c0` CHECK `id` = 1 +) +ENGINE = TinyLog(); + +INSERT INTO constraint_on_nullable_type VALUES (0); -- {serverError 469} +INSERT INTO constraint_on_nullable_type VALUES (1); + +SELECT * FROM constraint_on_nullable_type; + +DROP TABLE constraint_on_nullable_type; + +SET allow_suspicious_low_cardinality_types = 1; + +DROP TABLE IF EXISTS constraint_on_low_cardinality_type; +CREATE TABLE constraint_on_low_cardinality_type +( + `id` LowCardinality(UInt64), + CONSTRAINT `c0` CHECK `id` = 2 +) +ENGINE = TinyLog; + +INSERT INTO constraint_on_low_cardinality_type VALUES (0); -- {serverError 469} +INSERT INTO constraint_on_low_cardinality_type VALUES (2); + +SELECT * FROM constraint_on_low_cardinality_type; + +DROP TABLE constraint_on_low_cardinality_type; + +DROP TABLE IF EXISTS constraint_on_low_cardinality_nullable_type; + +CREATE TABLE constraint_on_low_cardinality_nullable_type +( + `id` LowCardinality(Nullable(UInt64)), + CONSTRAINT `c0` CHECK `id` = 3 +) +ENGINE = TinyLog; + +INSERT INTO constraint_on_low_cardinality_nullable_type VALUES (0); -- {serverError 469} +INSERT INTO constraint_on_low_cardinality_nullable_type VALUES (3); + +SELECT * FROM constraint_on_low_cardinality_nullable_type; + +DROP TABLE constraint_on_low_cardinality_nullable_type; diff --git a/tests/queries/0_stateless/01721_constraints_constant_expressions.reference b/tests/queries/0_stateless/01721_constraints_constant_expressions.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01721_constraints_constant_expressions.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01721_constraints_constant_expressions.sql b/tests/queries/0_stateless/01721_constraints_constant_expressions.sql new file mode 100644 index 00000000000..c38ed442eef --- /dev/null +++ b/tests/queries/0_stateless/01721_constraints_constant_expressions.sql @@ -0,0 +1,27 @@ +DROP TABLE IF EXISTS constraint_constant_number_expression; +CREATE TABLE constraint_constant_number_expression +( + id UInt64, + CONSTRAINT `c0` CHECK 1, + CONSTRAINT `c1` CHECK 1 < 2, + CONSTRAINT `c2` CHECK isNull(cast(NULL, 'Nullable(UInt8)')) +) ENGINE = TinyLog(); + +INSERT INTO constraint_constant_number_expression VALUES (1); + +SELECT * FROM constraint_constant_number_expression; + +DROP TABLE constraint_constant_number_expression; + +DROP TABLE IF EXISTS constraint_constant_number_expression_non_uint8; +CREATE TABLE constraint_constant_number_expression_non_uint8 +( + id UInt64, + CONSTRAINT `c0` CHECK toUInt64(1) +) ENGINE = TinyLog(); + +INSERT INTO constraint_constant_number_expression_non_uint8 VALUES (1); -- {serverError 1} + +SELECT * FROM constraint_constant_number_expression_non_uint8; + +DROP TABLE constraint_constant_number_expression_non_uint8; From 2eec1d021b4ac417cc625ece47d5c0f86cc90d0d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 20 Feb 2021 00:37:52 +0300 Subject: [PATCH 390/510] Fixed unused code --- src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 84c31de99b4..9bb80cfa4e1 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -66,11 +66,6 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) const UInt8 * data = res_column_uint8.getData().data(); size_t size = res_column_uint8.size(); - std::cerr << std::endl; - for (size_t j = 0; j < size; ++j) - std::cerr << data[j] << " "; - std::cerr << std::endl; - /// Is violated. if (!memoryIsByte(data, size, 1)) { From 23af53067d17e5dd1aa0fe844fc195e5fa154470 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Feb 2021 11:35:31 +0300 Subject: [PATCH 391/510] Updated support for Nullable column --- .../CheckConstraintsBlockOutputStream.cpp | 27 ++++++++++++++++--- ...01721_constraints_constant_expressions.sql | 15 ++++++++++- 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 9bb80cfa4e1..b42ff4b4e1d 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -52,15 +52,34 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName()); auto result_type = removeNullable(removeLowCardinality(res_column.type)); - auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality(); - - if (const auto * column_nullable = checkAndGetColumn(*result_column)) - result_column = column_nullable->getNestedColumnPtr(); if (!isUInt8(result_type)) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Constraint {} does not return a value of type UInt8", backQuote(constraint_ptr->name)); + auto result_column = res_column.column->convertToFullColumnIfConst()->convertToFullColumnIfLowCardinality(); + + if (const auto * column_nullable = checkAndGetColumn(*result_column)) + { + const auto & nested_column = column_nullable->getNestedColumnPtr(); + + /// Check if constraint value is nullable + const auto & null_map = column_nullable->getNullMapColumn(); + const auto & data = null_map.getData(); + bool null_map_contain_null = std::find(data.begin(), data.end(), true); + + if (null_map_contain_null) + throw Exception( + ErrorCodes::VIOLATED_CONSTRAINT, + "Constraint {} for table {} is violated. Expression: ({})."\ + "Constraint expression returns nullable column that contains null value", + backQuote(constraint_ptr->name), + table_id.getNameForLogs(), + serializeAST(*(constraint_ptr->expr), true)); + + result_column = nested_column; + } + const ColumnUInt8 & res_column_uint8 = assert_cast(*result_column); const UInt8 * data = res_column_uint8.getData().data(); diff --git a/tests/queries/0_stateless/01721_constraints_constant_expressions.sql b/tests/queries/0_stateless/01721_constraints_constant_expressions.sql index c38ed442eef..d70c0cd4dc0 100644 --- a/tests/queries/0_stateless/01721_constraints_constant_expressions.sql +++ b/tests/queries/0_stateless/01721_constraints_constant_expressions.sql @@ -20,8 +20,21 @@ CREATE TABLE constraint_constant_number_expression_non_uint8 CONSTRAINT `c0` CHECK toUInt64(1) ) ENGINE = TinyLog(); -INSERT INTO constraint_constant_number_expression_non_uint8 VALUES (1); -- {serverError 1} +INSERT INTO constraint_constant_number_expression_non_uint8 VALUES (2); -- {serverError 1} SELECT * FROM constraint_constant_number_expression_non_uint8; DROP TABLE constraint_constant_number_expression_non_uint8; + +DROP TABLE IF EXISTS constraint_constant_nullable_expression_that_contains_null; +CREATE TABLE constraint_constant_nullable_expression_that_contains_null +( + id UInt64, + CONSTRAINT `c0` CHECK nullIf(1 % 2, 1) +) ENGINE = TinyLog(); + +INSERT INTO constraint_constant_nullable_expression_that_contains_null VALUES (3); -- {serverError 469} + +SELECT * FROM constraint_constant_nullable_expression_that_contains_null; + +DROP TABLE constraint_constant_nullable_expression_that_contains_null; From da321c2bfecaaafce225e4df0a711ae6a6fe45a4 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Feb 2021 16:08:04 +0300 Subject: [PATCH 392/510] Fixed check for null value in null map --- src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index b42ff4b4e1d..90d6560b9a8 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -66,9 +66,11 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) /// Check if constraint value is nullable const auto & null_map = column_nullable->getNullMapColumn(); const auto & data = null_map.getData(); - bool null_map_contain_null = std::find(data.begin(), data.end(), true); + const auto * it = std::find(data.begin(), data.end(), true); - if (null_map_contain_null) + bool null_map_contains_null = it != data.end(); + + if (null_map_contains_null) throw Exception( ErrorCodes::VIOLATED_CONSTRAINT, "Constraint {} for table {} is violated. Expression: ({})."\ From f077a25a7add0c02da295f25ee1578649367315e Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Feb 2021 17:02:34 +0300 Subject: [PATCH 393/510] StorageRabbitMQ added UVLoop --- src/Storages/RabbitMQ/StorageRabbitMQ.cpp | 10 ++--- src/Storages/RabbitMQ/StorageRabbitMQ.h | 3 +- src/Storages/RabbitMQ/UVLoop.h | 44 +++++++++++++++++++ .../WriteBufferToRabbitMQProducer.cpp | 4 +- .../RabbitMQ/WriteBufferToRabbitMQProducer.h | 3 +- 5 files changed, 53 insertions(+), 11 deletions(-) create mode 100644 src/Storages/RabbitMQ/UVLoop.h diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp index 48305ab1b61..0ecf85e5c3d 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.cpp +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.cpp @@ -99,9 +99,7 @@ StorageRabbitMQ::StorageRabbitMQ( , unique_strbase(getRandomName()) , queue_size(std::max(QUEUE_SIZE, static_cast(getMaxBlockSize()))) { - loop = std::make_unique(); - uv_loop_init(loop.get()); - event_handler = std::make_shared(loop.get(), log); + event_handler = std::make_shared(loop.getLoop(), log); restoreConnection(false); StorageInMemoryMetadata storage_metadata; @@ -482,7 +480,7 @@ bool StorageRabbitMQ::restoreConnection(bool reconnecting) /* Connection is not closed immediately (firstly, all pending operations are completed, and then * an AMQP closing-handshake is performed). But cannot open a new connection until previous one is properly closed */ - while (!connection->closed() && ++cnt_retries != RETRIES_MAX) + while (!connection->closed() && cnt_retries++ != RETRIES_MAX) event_handler->iterateLoop(); /// This will force immediate closure if not yet closed @@ -498,7 +496,7 @@ bool StorageRabbitMQ::restoreConnection(bool reconnecting) AMQP::Login(login_password.first, login_password.second), vhost)); cnt_retries = 0; - while (!connection->ready() && !stream_cancelled && ++cnt_retries != RETRIES_MAX) + while (!connection->ready() && !stream_cancelled && cnt_retries++ != RETRIES_MAX) { event_handler->iterateLoop(); std::this_thread::sleep_for(std::chrono::milliseconds(CONNECT_SLEEP)); @@ -653,7 +651,7 @@ void StorageRabbitMQ::shutdown() connection->close(); size_t cnt_retries = 0; - while (!connection->closed() && ++cnt_retries != RETRIES_MAX) + while (!connection->closed() && cnt_retries++ != RETRIES_MAX) event_handler->iterateLoop(); /// Should actually force closure, if not yet closed, but it generates distracting error logs diff --git a/src/Storages/RabbitMQ/StorageRabbitMQ.h b/src/Storages/RabbitMQ/StorageRabbitMQ.h index aa316e7a842..9f573ea4a3e 100644 --- a/src/Storages/RabbitMQ/StorageRabbitMQ.h +++ b/src/Storages/RabbitMQ/StorageRabbitMQ.h @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -96,7 +97,7 @@ private: std::pair login_password; String vhost; - std::unique_ptr loop; + UVLoop loop; std::shared_ptr event_handler; std::unique_ptr connection; /// Connection for all consumers diff --git a/src/Storages/RabbitMQ/UVLoop.h b/src/Storages/RabbitMQ/UVLoop.h new file mode 100644 index 00000000000..c7258aee7a4 --- /dev/null +++ b/src/Storages/RabbitMQ/UVLoop.h @@ -0,0 +1,44 @@ +#pragma once + +#include + +#include +#include + +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYSTEM_ERROR; +} + +/// RAII wrapper arount uv event loop +class UVLoop: public boost::noncopyable +{ +public: + UVLoop(): loop_ptr(new uv_loop_t()) + { + int res = uv_loop_init(loop_ptr.get()); + + if (res != 0) + throw Exception("UVLoop could not initialize", ErrorCodes::SYSTEM_ERROR); + } + + ~UVLoop() + { + if (loop_ptr) + uv_loop_close(loop_ptr.get()); + } + + inline uv_loop_t * getLoop() { return loop_ptr.get(); } + + inline const uv_loop_t * getLoop() const { return loop_ptr.get(); } + +private: + std::unique_ptr loop_ptr; +}; + +} diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index ac1b253b4bb..17e3dfab9ef 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -57,9 +57,7 @@ WriteBufferToRabbitMQProducer::WriteBufferToRabbitMQProducer( , max_rows(rows_per_message) , chunk_size(chunk_size_) { - loop = std::make_unique(); - uv_loop_init(loop.get()); - event_handler = std::make_unique(loop.get(), log); + event_handler = std::make_unique(loop.getLoop(), log); if (setupConnection(false)) { diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h index e88f92239ca..d307ad9cc72 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include @@ -69,7 +70,7 @@ private: AMQP::Table key_arguments; BackgroundSchedulePool::TaskHolder writing_task; - std::unique_ptr loop; + UVLoop loop; std::unique_ptr event_handler; std::unique_ptr connection; std::unique_ptr producer_channel; From c9ab71e6c1b9100196fb73236dbfecf662faa430 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 25 Feb 2021 14:39:33 +0000 Subject: [PATCH 394/510] Bump CI From 29e783999cee676aa1dd1390110142697e9622b9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 25 Feb 2021 17:43:32 +0300 Subject: [PATCH 395/510] Fixed typo --- src/Storages/RabbitMQ/UVLoop.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/RabbitMQ/UVLoop.h b/src/Storages/RabbitMQ/UVLoop.h index c7258aee7a4..4de67cbc206 100644 --- a/src/Storages/RabbitMQ/UVLoop.h +++ b/src/Storages/RabbitMQ/UVLoop.h @@ -15,8 +15,8 @@ namespace ErrorCodes extern const int SYSTEM_ERROR; } -/// RAII wrapper arount uv event loop -class UVLoop: public boost::noncopyable +/// RAII wrapper around uv event loop +class UVLoop : public boost::noncopyable { public: UVLoop(): loop_ptr(new uv_loop_t()) From 653d962cdd04746e351604265ae2269ac5c81112 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Feb 2021 17:43:58 +0300 Subject: [PATCH 396/510] Fix style and add low-level check --- src/Storages/MergeTree/MergeTreeData.h | 2 +- src/Storages/StorageView.cpp | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 09cf017d220..f03f3f1dd8c 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -520,7 +520,7 @@ public: void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override; /// Checks if the Mutation can be performed. - /// (currenly no additional checks: always ok) + /// (currently no additional checks: always ok) void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override; /// Checks that partition name in all commands is valid diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 38349ef8df9..22a1992d922 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -105,6 +105,9 @@ void StorageView::read( static ASTTableExpression * getFirstTableExpression(ASTSelectQuery & select_query) { + if (!select_query.tables() || select_query.tables()->children.empty()) + throw Exception("Logical error: no table expression in view select AST", ErrorCodes::LOGICAL_ERROR); + auto * select_element = select_query.tables()->children[0]->as(); if (!select_element->table_expression) From 64327eafc84310803ce5b5236943bb1d657c43c6 Mon Sep 17 00:00:00 2001 From: John Hummel Date: Thu, 25 Feb 2021 10:50:22 -0500 Subject: [PATCH 397/510] Update encryption-functions.md Updated Optinal to Optional line 136. --- docs/en/sql-reference/functions/encryption-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/encryption-functions.md b/docs/en/sql-reference/functions/encryption-functions.md index 0dd7469b25e..c7b26b1f288 100644 --- a/docs/en/sql-reference/functions/encryption-functions.md +++ b/docs/en/sql-reference/functions/encryption-functions.md @@ -132,7 +132,7 @@ aes_encrypt_mysql('mode', 'plaintext', 'key' [, iv]) - `mode` — Encryption mode. [String](../../sql-reference/data-types/string.md#string). - `plaintext` — Text that needs to be encrypted. [String](../../sql-reference/data-types/string.md#string). - `key` — Encryption key. If key is longer than required by mode, MySQL-specific key folding is performed. [String](../../sql-reference/data-types/string.md#string). -- `iv` — Initialization vector. Optinal, only first 16 bytes are taken into account [String](../../sql-reference/data-types/string.md#string). +- `iv` — Initialization vector. Optional, only first 16 bytes are taken into account [String](../../sql-reference/data-types/string.md#string). **Returned value** From 32fa3dbc99c2600fb4571e1dc11985d81e34d6c8 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 23 Feb 2021 11:18:24 +0100 Subject: [PATCH 398/510] Integration tests: add IPC_LOCK and SYS_NICE caps It is needed to make integration tests work with a installed binary (/usr/bin/clickhouse) mounted in. Otherwise the binary mounted in have the capability, while the container is not allowing that and you get cryptic message: standard_init_linux.go:211: exec user process caused "operation not permitted" --- tests/integration/helpers/cluster.py | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 16ceb823f2e..0fa6f80e6ba 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -868,6 +868,8 @@ services: cap_add: - SYS_PTRACE - NET_ADMIN + - IPC_LOCK + - SYS_NICE depends_on: {depends_on} user: '{user}' env_file: From a4c9e62d6dd774b0ff300abf9bcfc3ce872b08f7 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 23 Feb 2021 17:53:14 +0100 Subject: [PATCH 399/510] Integration tests: introduce wait_for_log_line. It uses tail -f executed in container, this way i don't need to pull the file many times, rememeber positions etc. --- tests/integration/helpers/cluster.py | 8 ++++++++ tests/integration/test_storage_kafka/test.py | 18 +++++------------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0fa6f80e6ba..9e09016c250 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -13,6 +13,7 @@ import subprocess import time import traceback import urllib.parse +import shlex import cassandra.cluster import docker @@ -1081,6 +1082,13 @@ class ClickHouseInstance: ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) return len(result) > 0 + def wait_for_log_line(self, regexp, filename='/var/log/clickhouse-server/clickhouse-server.log', timeout=30, repetitions=1, look_behind_lines=100): + start_time = time.time() + result = self.exec_in_container( + ["bash", "-c", 'timeout {} tail -Fn{} "{}" | grep -Eqm {} {}'.format(timeout, look_behind_lines, filename, repetitions, shlex.quote(regexp))]) + current_time = time.time() + print('Log line matching "{}" appeared in a {} seconds'.format(regexp, current_time - start_time)) + def file_exists(self, path): return self.exec_in_container( ["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n' diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 5f2726832cc..d8b9684c98a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2042,7 +2042,7 @@ def test_kafka_rebalance(kafka_cluster): assert result == 1, 'Messages from kafka get duplicated!' -@pytest.mark.timeout(1200) +@pytest.mark.timeout(120) def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] kafka_produce('no_holes_when_write_suffix_failed', messages) @@ -2076,23 +2076,15 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.kafka - WHERE NOT sleepEachRow(1); + WHERE NOT sleepEachRow(0.1); ''') # the tricky part here is that disconnect should happen after write prefix, but before write suffix - # so i use sleepEachRow - - time.sleep(3) + instance.wait_for_log_line("Polled batch of 20 messages") pm.drop_instance_zk_connections(instance) - time.sleep(20) + instance.wait_for_log_line("Coordination.*while write prefix to view") pm.heal_all() - - # connection restored and it will take a while until next block will be flushed - # it takes years on CI :\ - time.sleep(45) - - # as it's a bit tricky to hit the proper moment - let's check in logs if we did it correctly - assert instance.contains_in_log("ZooKeeper session has been expired.: while write prefix to view") + instance.wait_for_log_line("Committed offset 23") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) From 6c6eaf2a608ce482b80385068ab34ddd52113834 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 24 Feb 2021 16:08:58 +0100 Subject: [PATCH 400/510] Fixes --- tests/integration/helpers/cluster.py | 18 +- tests/integration/test_storage_kafka/test.py | 180 +++++++++---------- 2 files changed, 99 insertions(+), 99 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 9e09016c250..8c3cc2e592b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -574,7 +574,7 @@ class ClickHouseCluster: raise Exception("Can't wait Minio to start") def wait_schema_registry_to_start(self, timeout=10): - sr_client = CachedSchemaRegistryClient('http://localhost:8081') + sr_client = CachedSchemaRegistryClient({"url":'http://localhost:8081'}) start = time.time() while time.time() - start < timeout: try: @@ -1085,9 +1085,19 @@ class ClickHouseInstance: def wait_for_log_line(self, regexp, filename='/var/log/clickhouse-server/clickhouse-server.log', timeout=30, repetitions=1, look_behind_lines=100): start_time = time.time() result = self.exec_in_container( - ["bash", "-c", 'timeout {} tail -Fn{} "{}" | grep -Eqm {} {}'.format(timeout, look_behind_lines, filename, repetitions, shlex.quote(regexp))]) - current_time = time.time() - print('Log line matching "{}" appeared in a {} seconds'.format(regexp, current_time - start_time)) + ["bash", "-c", 'timeout {} tail -Fn{} "{}" | grep -Em {} {}'.format(timeout, look_behind_lines, filename, repetitions, shlex.quote(regexp))]) + + # if repetitions>1 grep will return success even if not enough lines were collected, + if repetitions>1 and len(result.splitlines()) < repetitions: + print("wait_for_log_line: those lines were founded during {} sec.".format(timeout)) + print(result) + raise Exception("wait_for_log_line: Not enough repetitions: {} found, while {} expected".format(len(result.splitlines()), repetitions)) + + wait_duration = time.time() - start_time + + print('Log line matching "{}" appeared in a {} seconds'.format(regexp, wait_duration)) + return wait_duration + def file_exists(self, path): return self.exec_in_container( diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index d8b9684c98a..b7548d39d8a 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -183,7 +183,7 @@ def test_kafka_json_as_string(kafka_cluster): "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") -@pytest.mark.timeout(300) +@pytest.mark.timeout(120) def test_kafka_formats(kafka_cluster): # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -311,7 +311,7 @@ def test_kafka_formats(kafka_cluster): # On empty message exception happens: Line "" doesn't match the regexp.: (at row 1) # /src/Processors/Formats/Impl/RegexpRowInputFormat.cpp:140: DB::RegexpRowInputFormat::readRow(std::__1::vector::mutable_ptr, std::__1::allocator::mutable_ptr > >&, DB::RowReadExtension&) @ 0x1df82fcb in /usr/bin/clickhouse ], - 'extra_settings': ", format_regexp='\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)', format_regexp_escaping_rule='Escaped'" + 'extra_settings': r", format_regexp='\(id = (.+?), blockNo = (.+?), val1 = \"(.+?)\", val2 = (.+?), val3 = (.+?)\)', format_regexp_escaping_rule='Escaped'" }, ## BINARY FORMATS @@ -545,7 +545,7 @@ def test_kafka_formats(kafka_cluster): '''.format(topic_name=topic_name, format_name=format_name, extra_settings=format_opts.get('extra_settings') or '')) - time.sleep(12) + instance.wait_for_log_line('kafka.*Committed offset [0-9]+.*format_tests_', repetitions=len(all_formats.keys()), look_behind_lines=12000) for format_name, format_opts in list(all_formats.items()): print(('Checking {}'.format(format_name))) @@ -1170,7 +1170,7 @@ def test_kafka_materialized_view(kafka_cluster): kafka_check_result(result, True) @pytest.mark.timeout(180) -def test_librdkafka_snappy_regression(kafka_cluster): +def test_librdkafka_compression(kafka_cluster): """ Regression for UB in snappy-c (that is used in librdkafka), backport pr is [1]. @@ -1180,55 +1180,63 @@ def test_librdkafka_snappy_regression(kafka_cluster): Example of corruption: 2020.12.10 09:59:56.831507 [ 20 ] {} void DB::StorageKafka::threadFunc(size_t): Code: 27, e.displayText() = DB::Exception: Cannot parse input: expected '"' before: 'foo"}': (while reading the value of key value): (at row 1) -, Stack trace (when copying this message, always include the lines below): + + To trigger this regression there should duplicated messages + + Orignal reproducer is: + $ gcc --version |& fgrep gcc + gcc (GCC) 10.2.0 + $ yes foobarbaz | fold -w 80 | head -n10 >| in-… + $ make clean && make CFLAGS='-Wall -g -O2 -ftree-loop-vectorize -DNDEBUG=1 -DSG=1 -fPIC' + $ ./verify in + final comparision of in failed at 20 of 100 + """ - # create topic with snappy compression - admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:9092'}) - topic_snappy = admin.NewTopic(topic='snappy_regression', num_partitions=1, replication_factor=1, config={ - 'compression.type': 'snappy', - }) - admin_client.create_topics(new_topics=[topic_snappy], validate_only=False) - - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value String) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'snappy_regression', - kafka_group_name = 'ch_snappy_regression', - kafka_format = 'JSONEachRow'; - ''') + supported_compression_types = ['gzip', 'snappy', 'lz4', 'zstd', 'uncompressed'] messages = [] expected = [] - # To trigger this regression there should duplicated messages - # Orignal reproducer is: - # - # $ gcc --version |& fgrep gcc - # gcc (GCC) 10.2.0 - # $ yes foobarbaz | fold -w 80 | head -n10 >| in-… - # $ make clean && make CFLAGS='-Wall -g -O2 -ftree-loop-vectorize -DNDEBUG=1 -DSG=1 -fPIC' - # $ ./verify in - # final comparision of in failed at 20 of 100 + value = 'foobarbaz'*10 number_of_messages = 50 for i in range(number_of_messages): messages.append(json.dumps({'key': i, 'value': value})) expected.append(f'{i}\t{value}') - kafka_produce('snappy_regression', messages) expected = '\n'.join(expected) - while True: - result = instance.query('SELECT * FROM test.kafka') - rows = len(result.strip('\n').split('\n')) - print(rows) - if rows == number_of_messages: - break + for compression_type in supported_compression_types: + print(('Check compression {}'.format(compression_type))) - assert TSV(result) == TSV(expected) + topic_name = 'test_librdkafka_compression_{}'.format(compression_type) + admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:9092'}) + topic = admin.NewTopic(topic=topic_name, num_partitions=1, replication_factor=1, config={ + 'compression.type': compression_type, + }) + admin_client.create_topics(new_topics=[topic], validate_only=False) - instance.query('DROP TABLE test.kafka') + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value String) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = '{topic_name}', + kafka_group_name = '{topic_name}_group', + kafka_format = 'JSONEachRow', + kafka_flush_interval_ms = 1000; + CREATE MATERIALIZED VIEW test.consumer Engine=Log AS + SELECT * FROM test.kafka; + '''.format(topic_name=topic_name) ) + + kafka_produce(topic_name, messages) + + instance.wait_for_log_line("Committed offset {}".format(number_of_messages)) + + result = instance.query('SELECT * FROM test.consumer') + assert TSV(result) == TSV(expected) + + instance.query('DROP TABLE test.kafka SYNC') + instance.query('DROP TABLE test.consumer SYNC') @pytest.mark.timeout(180) def test_kafka_materialized_view_with_subquery(kafka_cluster): @@ -1577,9 +1585,6 @@ def test_kafka_commit_on_block_write(kafka_cluster): DROP TABLE test.kafka; ''') - while int(instance.query("SELECT count() FROM system.tables WHERE database='test' AND name='kafka'")) == 1: - time.sleep(1) - instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka @@ -1874,7 +1879,8 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): kafka_topic_list = 'topic_with_multiple_partitions2', kafka_group_name = 'topic_with_multiple_partitions2', kafka_format = 'JSONEachRow', - kafka_max_block_size = 211; + kafka_max_block_size = 211, + kafka_flush_interval_ms = 500; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; @@ -1892,7 +1898,7 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): messages.append("\n".join(rows)) kafka_produce('topic_with_multiple_partitions2', messages) - time.sleep(30) + instance.wait_for_log_line('kafka.*Stalled', repetitions=20) result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) @@ -1961,7 +1967,8 @@ def test_kafka_rebalance(kafka_cluster): kafka_topic_list = 'topic_with_multiple_partitions', kafka_group_name = 'rebalance_test_group', kafka_format = 'JSONEachRow', - kafka_max_block_size = 33; + kafka_max_block_size = 33, + kafka_flush_interval_ms = 500; CREATE MATERIALIZED VIEW test.{0}_mv TO test.destination AS SELECT key, @@ -1975,21 +1982,15 @@ def test_kafka_rebalance(kafka_cluster): FROM test.{0}; '''.format(table_name)) # kafka_cluster.open_bash_shell('instance') - while int( - instance.query("SELECT count() FROM test.destination WHERE _consumed_by='{}'".format(table_name))) == 0: - print(("Waiting for test.kafka_consumer{} to start consume".format(consumer_index))) - time.sleep(1) + # Waiting for test.kafka_consumerX to start consume ... + instance.wait_for_log_line('kafka_consumer{}.*Polled offset [0-9]+'.format(consumer_index)) cancel.set() # I leave last one working by intent (to finish consuming after all rebalances) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): print(("Dropping test.kafka_consumer{}".format(consumer_index))) - instance.query('DROP TABLE IF EXISTS test.kafka_consumer{}'.format(consumer_index)) - while int(instance.query( - "SELECT count() FROM system.tables WHERE database='test' AND name='kafka_consumer{}'".format( - consumer_index))) == 1: - time.sleep(1) + instance.query('DROP TABLE IF EXISTS test.kafka_consumer{} SYNC'.format(consumer_index)) # print(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) # kafka_cluster.open_bash_shell('instance') @@ -2044,7 +2045,7 @@ def test_kafka_rebalance(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): - messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] + messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] kafka_produce('no_holes_when_write_suffix_failed', messages) instance.query(''' @@ -2060,31 +2061,28 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): kafka_max_block_size = 20, kafka_flush_interval_ms = 2000; - SELECT * FROM test.kafka LIMIT 1; /* do subscription & assignment in advance (it can take different time, test rely on timing, so can flap otherwise) */ + CREATE TABLE test.view (key UInt64, value String) + ENGINE = ReplicatedMergeTree('/clickhouse/kafkatest/tables/no_holes_when_write_suffix_failed', 'node1') + ORDER BY key; ''') - messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('no_holes_when_write_suffix_failed', messages) - # init PartitionManager (it starts container) earlier pm = PartitionManager() instance.query(''' - CREATE TABLE test.view (key UInt64, value String) - ENGINE = ReplicatedMergeTree('/clickhouse/kafkatest/tables/no_holes_when_write_suffix_failed', 'node1') - ORDER BY key; - CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.kafka - WHERE NOT sleepEachRow(0.1); + WHERE NOT sleepEachRow(0.25); ''') - # the tricky part here is that disconnect should happen after write prefix, but before write suffix instance.wait_for_log_line("Polled batch of 20 messages") + # the tricky part here is that disconnect should happen after write prefix, but before write suffix + # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" + # while materialized view is working to inject zookeeper failure pm.drop_instance_zk_connections(instance) - instance.wait_for_log_line("Coordination.*while write prefix to view") + instance.wait_for_log_line("Error.*(session has been expired|Connection loss).*while write prefix to view") pm.heal_all() - instance.wait_for_log_line("Committed offset 23") + instance.wait_for_log_line("Committed offset 22") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) @@ -2261,7 +2259,7 @@ def test_bad_reschedule(kafka_cluster): @pytest.mark.timeout(300) def test_kafka_duplicates_when_commit_failed(kafka_cluster): - messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] + messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] kafka_produce('duplicates_when_commit_failed', messages) instance.query(''' @@ -2277,44 +2275,35 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): kafka_max_block_size = 20, kafka_flush_interval_ms = 1000; - SELECT * FROM test.kafka LIMIT 1; /* do subscription & assignment in advance (it can take different time, test rely on timing, so can flap otherwise) */ - ''') - - messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('duplicates_when_commit_failed', messages) - - instance.query(''' CREATE TABLE test.view (key UInt64, value String) ENGINE = MergeTree() ORDER BY key; - - CREATE MATERIALIZED VIEW test.consumer TO test.view AS - SELECT * FROM test.kafka - WHERE NOT sleepEachRow(0.5); ''') - # print time.strftime("%m/%d/%Y %H:%M:%S") - time.sleep(3) # MV will work for 10 sec, after that commit should happen, we want to pause before + instance.query(''' + CREATE MATERIALIZED VIEW test.consumer TO test.view AS + SELECT * FROM test.kafka + WHERE NOT sleepEachRow(0.25); + ''') - # print time.strftime("%m/%d/%Y %H:%M:%S") + instance.wait_for_log_line("Polled batch of 20 messages") + # the tricky part here is that disconnect should happen after write prefix, but before we do commit + # we have 0.25 (sleepEachRow) * 20 ( Rows ) = 5 sec window after "Polled batch of 20 messages" + # while materialized view is working to inject zookeeper failure kafka_cluster.pause_container('kafka1') - # that timeout it VERY important, and picked after lot of experiments - # when too low (<30sec) librdkafka will not report any timeout (alternative is to decrease the default session timeouts for librdkafka) - # when too high (>50sec) broker will decide to remove us from the consumer group, and will start answering "Broker: Unknown member" - time.sleep(42) - # print time.strftime("%m/%d/%Y %H:%M:%S") + # if we restore the connection too fast (<30sec) librdkafka will not report any timeout + # (alternative is to decrease the default session timeouts for librdkafka) + # + # when the delay is too long (>50sec) broker will decide to remove us from the consumer group, + # and will start answering "Broker: Unknown member" + instance.wait_for_log_line("Exception during commit attempt: Local: Waiting for coordinator", timeout=45) + instance.wait_for_log_line("All commit attempts failed", look_behind_lines=500) + kafka_cluster.unpause_container('kafka1') # kafka_cluster.open_bash_shell('instance') - - # connection restored and it will take a while until next block will be flushed - # it takes years on CI :\ - time.sleep(30) - - # as it's a bit tricky to hit the proper moment - let's check in logs if we did it correctly - assert instance.contains_in_log("Local: Waiting for coordinator") - assert instance.contains_in_log("All commit attempts failed") + instance.wait_for_log_line("Committed offset 22") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) @@ -2420,7 +2409,8 @@ def test_kafka_unavailable(kafka_cluster): kafka_topic_list = 'test_bad_reschedule', kafka_group_name = 'test_bad_reschedule', kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000; + kafka_max_block_size = 1000, + kafka_flush_interval_ms = 1000; CREATE MATERIALIZED VIEW test.destination Engine=Log AS SELECT From 2bcfff257c28d76f58304b6a3ad7534dad0afca7 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Thu, 25 Feb 2021 17:01:35 +0100 Subject: [PATCH 401/510] Remove most of sleeps in tests --- tests/integration/helpers/cluster.py | 4 +- tests/integration/test_storage_kafka/test.py | 169 ++++++++++--------- 2 files changed, 91 insertions(+), 82 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 8c3cc2e592b..3872234d36c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1089,13 +1089,13 @@ class ClickHouseInstance: # if repetitions>1 grep will return success even if not enough lines were collected, if repetitions>1 and len(result.splitlines()) < repetitions: - print("wait_for_log_line: those lines were founded during {} sec.".format(timeout)) + print("wait_for_log_line: those lines were found during {} seconds:".format(timeout)) print(result) raise Exception("wait_for_log_line: Not enough repetitions: {} found, while {} expected".format(len(result.splitlines()), repetitions)) wait_duration = time.time() - start_time - print('Log line matching "{}" appeared in a {} seconds'.format(regexp, wait_duration)) + print('{} log line matching "{}" appeared in a {} seconds'.format(repetitions, regexp, wait_duration)) return wait_duration diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b7548d39d8a..fe1b94adf75 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -796,6 +796,12 @@ def test_kafka_issue4116(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + + topic_list = [] + topic_list.append(NewTopic(name="consumer_hang", num_partitions=8, replication_factor=1)) + admin_client.create_topics(new_topics=topic_list, validate_only=False) + instance.query(''' DROP TABLE IF EXISTS test.kafka; DROP TABLE IF EXISTS test.view; @@ -807,20 +813,18 @@ def test_kafka_consumer_hang(kafka_cluster): kafka_topic_list = 'consumer_hang', kafka_group_name = 'consumer_hang', kafka_format = 'JSONEachRow', - kafka_num_consumers = 8, - kafka_row_delimiter = '\\n'; + kafka_num_consumers = 8; CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = Memory(); CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.kafka; ''') - time.sleep(10) - instance.query('SELECT * FROM test.view') + instance.wait_for_log_line('kafka.*Stalled', repetitions=20) # This should trigger heartbeat fail, # which will trigger REBALANCE_IN_PROGRESS, # and which can lead to consumer hang. kafka_cluster.pause_container('kafka1') - time.sleep(0.5) + instance.wait_for_log_line('heartbeat error') kafka_cluster.unpause_container('kafka1') # print("Attempt to drop") @@ -844,6 +848,12 @@ def test_kafka_consumer_hang(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang2(kafka_cluster): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + + topic_list = [] + topic_list.append(NewTopic(name="consumer_hang2", num_partitions=1, replication_factor=1)) + admin_client.create_topics(new_topics=topic_list, validate_only=False) + instance.query(''' DROP TABLE IF EXISTS test.kafka; @@ -884,23 +894,22 @@ def test_kafka_consumer_hang2(kafka_cluster): assert int(instance.query("select count() from system.processes where position(lower(query),'dr'||'op')>0")) == 0 -@pytest.mark.timeout(180) +@pytest.mark.timeout(120) def test_kafka_csv_with_delimiter(kafka_cluster): + messages = [] + for i in range(50): + messages.append('{i}, {i}'.format(i=i)) + kafka_produce('csv', messages) + instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'csv', kafka_group_name = 'csv', - kafka_format = 'CSV', - kafka_row_delimiter = '\\n'; + kafka_format = 'CSV'; ''') - messages = [] - for i in range(50): - messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) - result = '' while True: result += instance.query('SELECT * FROM test.kafka', ignore_error=True) @@ -910,23 +919,22 @@ def test_kafka_csv_with_delimiter(kafka_cluster): kafka_check_result(result, True) -@pytest.mark.timeout(180) +@pytest.mark.timeout(120) def test_kafka_tsv_with_delimiter(kafka_cluster): + messages = [] + for i in range(50): + messages.append('{i}\t{i}'.format(i=i)) + kafka_produce('tsv', messages) + instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'tsv', kafka_group_name = 'tsv', - kafka_format = 'TSV', - kafka_row_delimiter = '\\n'; + kafka_format = 'TSV'; ''') - messages = [] - for i in range(50): - messages.append('{i}\t{i}'.format(i=i)) - kafka_produce('tsv', messages) - result = '' while True: result += instance.query('SELECT * FROM test.kafka', ignore_error=True) @@ -936,8 +944,13 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): kafka_check_result(result, True) -@pytest.mark.timeout(180) +@pytest.mark.timeout(120) def test_kafka_select_empty(kafka_cluster): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + topic_list = [] + topic_list.append(NewTopic(name="empty", num_partitions=1, replication_factor=1)) + admin_client.create_topics(new_topics=topic_list, validate_only=False) + instance.query(''' CREATE TABLE test.kafka (key UInt64) ENGINE = Kafka @@ -953,15 +966,6 @@ def test_kafka_select_empty(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_json_without_delimiter(kafka_cluster): - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'json', - kafka_group_name = 'json', - kafka_format = 'JSONEachRow'; - ''') - messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' @@ -972,6 +976,15 @@ def test_kafka_json_without_delimiter(kafka_cluster): messages += json.dumps({'key': i, 'value': i}) + '\n' kafka_produce('json', [messages]) + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = 'kafka1:19092', + kafka_topic_list = 'json', + kafka_group_name = 'json', + kafka_format = 'JSONEachRow'; + ''') + result = '' while True: result += instance.query('SELECT * FROM test.kafka', ignore_error=True) @@ -983,6 +996,10 @@ def test_kafka_json_without_delimiter(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_protobuf(kafka_cluster): + kafka_produce_protobuf_messages('pb', 0, 20) + kafka_produce_protobuf_messages('pb', 20, 1) + kafka_produce_protobuf_messages('pb', 21, 29) + instance.query(''' CREATE TABLE test.kafka (key UInt64, value String) ENGINE = Kafka @@ -993,10 +1010,6 @@ def test_kafka_protobuf(kafka_cluster): kafka_schema = 'kafka.proto:KeyValuePair'; ''') - kafka_produce_protobuf_messages('pb', 0, 20) - kafka_produce_protobuf_messages('pb', 20, 1) - kafka_produce_protobuf_messages('pb', 21, 29) - result = '' while True: result += instance.query('SELECT * FROM test.kafka', ignore_error=True) @@ -1009,6 +1022,9 @@ def test_kafka_protobuf(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_string_field_on_first_position_in_protobuf(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/12615 + kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) instance.query(''' CREATE TABLE test.kafka ( @@ -1021,14 +1037,8 @@ SETTINGS kafka_group_name = 'string_field_on_first_position_in_protobuf', kafka_format = 'Protobuf', kafka_schema = 'social:User'; - - SELECT * FROM test.kafka; ''') - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) - result = instance.query('SELECT * FROM test.kafka', ignore_error=True) expected = '''\ John Doe 0 1000000 @@ -1898,7 +1908,7 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): messages.append("\n".join(rows)) kafka_produce('topic_with_multiple_partitions2', messages) - instance.wait_for_log_line('kafka.*Stalled', repetitions=20) + instance.wait_for_log_line('kafka.*Stalled', repetitions=5) result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') print(result) @@ -2136,7 +2146,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): kafka_produce('commits_of_unprocessed_messages_on_drop', messages) instance.query(''' - DROP TABLE IF EXISTS test.destination; + DROP TABLE IF EXISTS test.destination SYNC; CREATE TABLE test.destination ( key UInt64, value UInt64, @@ -2156,7 +2166,8 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): kafka_topic_list = 'commits_of_unprocessed_messages_on_drop', kafka_group_name = 'commits_of_unprocessed_messages_on_drop_test_group', kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000; + kafka_max_block_size = 1000, + kafka_flush_interval_ms = 1000; CREATE MATERIALIZED VIEW test.kafka_consumer TO test.destination AS SELECT @@ -2170,9 +2181,8 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): FROM test.kafka; ''') - while int(instance.query("SELECT count() FROM test.destination")) == 0: - print("Waiting for test.kafka_consumer to start consume") - time.sleep(1) + # Waiting for test.kafka_consumer to start consume + instance.wait_for_log_line('Committed offset [0-9]+') cancel = threading.Event() @@ -2185,14 +2195,14 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 kafka_produce('commits_of_unprocessed_messages_on_drop', messages) - time.sleep(1) + time.sleep(0.5) kafka_thread = threading.Thread(target=produce) kafka_thread.start() - time.sleep(12) + time.sleep(4) instance.query(''' - DROP TABLE test.kafka; + DROP TABLE test.kafka SYNC; ''') instance.query(''' @@ -2202,11 +2212,12 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): kafka_topic_list = 'commits_of_unprocessed_messages_on_drop', kafka_group_name = 'commits_of_unprocessed_messages_on_drop_test_group', kafka_format = 'JSONEachRow', - kafka_max_block_size = 10000; + kafka_max_block_size = 10000, + kafka_flush_interval_ms = 1000; ''') cancel.set() - time.sleep(15) + instance.wait_for_log_line('kafka.*Stalled', repetitions=5) # kafka_cluster.open_bash_shell('instance') # SELECT key, _timestamp, _offset FROM test.destination where runningDifference(key) <> 1 ORDER BY key; @@ -2215,8 +2226,8 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): print(result) instance.query(''' - DROP TABLE test.kafka_consumer; - DROP TABLE test.destination; + DROP TABLE test.kafka_consumer SYNC; + DROP TABLE test.destination SYNC; ''') kafka_thread.join() @@ -2235,7 +2246,8 @@ def test_bad_reschedule(kafka_cluster): kafka_topic_list = 'test_bad_reschedule', kafka_group_name = 'test_bad_reschedule', kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000; + kafka_max_block_size = 1000, + kafka_flush_interval_ms = 1000; CREATE MATERIALIZED VIEW test.destination Engine=Log AS SELECT @@ -2250,9 +2262,7 @@ def test_bad_reschedule(kafka_cluster): FROM test.kafka; ''') - while int(instance.query("SELECT count() FROM test.destination")) < 20000: - print("Waiting for consume") - time.sleep(1) + instance.wait_for_log_line("Committed offset 20000") assert int(instance.query("SELECT max(consume_ts) - min(consume_ts) FROM test.destination")) < 8 @@ -2263,8 +2273,8 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): kafka_produce('duplicates_when_commit_failed', messages) instance.query(''' - DROP TABLE IF EXISTS test.view; - DROP TABLE IF EXISTS test.consumer; + DROP TABLE IF EXISTS test.view SYNC; + DROP TABLE IF EXISTS test.consumer SYNC; CREATE TABLE test.kafka (key UInt64, value String) ENGINE = Kafka @@ -2309,8 +2319,8 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): print(result) instance.query(''' - DROP TABLE test.consumer; - DROP TABLE test.view; + DROP TABLE test.consumer SYNC; + DROP TABLE test.view SYNC; ''') # After https://github.com/edenhill/librdkafka/issues/2631 @@ -2373,9 +2383,8 @@ def test_premature_flush_on_eof(kafka_cluster): # all subscriptions/assignments done during select, so it start sending data to test.destination # immediately after creation of MV - time.sleep(1.5) # that sleep is needed to ensure that first poll finished, and at least one 'empty' polls happened. - # Empty poll before the fix were leading to premature flush. - # TODO: wait for messages in log: "Polled batch of 1 messages", followed by "Stalled" + instance.wait_for_log_line("Polled batch of 1 messages") + instance.wait_for_log_line("Stalled") # produce more messages after delay kafka_produce('premature_flush_on_eof', messages) @@ -2383,7 +2392,7 @@ def test_premature_flush_on_eof(kafka_cluster): # data was not flushed yet (it will be flushed 7.5 sec after creating MV) assert int(instance.query("SELECT count() FROM test.destination")) == 0 - time.sleep(9) # TODO: wait for messages in log: "Committed offset ..." + instance.wait_for_log_line("Committed offset 2") # it should be single part, i.e. single insert result = instance.query('SELECT _part, count() FROM test.destination group by _part') @@ -2395,10 +2404,10 @@ def test_premature_flush_on_eof(kafka_cluster): ''') -@pytest.mark.timeout(180) +@pytest.mark.timeout(120) def test_kafka_unavailable(kafka_cluster): - messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce('test_bad_reschedule', messages) + messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(2000)] + kafka_produce('test_kafka_unavailable', messages) kafka_cluster.pause_container('kafka1') @@ -2406,8 +2415,8 @@ def test_kafka_unavailable(kafka_cluster): CREATE TABLE test.kafka (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_bad_reschedule', - kafka_group_name = 'test_bad_reschedule', + kafka_topic_list = 'test_kafka_unavailable', + kafka_group_name = 'test_kafka_unavailable', kafka_format = 'JSONEachRow', kafka_max_block_size = 1000, kafka_flush_interval_ms = 1000; @@ -2426,19 +2435,21 @@ def test_kafka_unavailable(kafka_cluster): ''') instance.query("SELECT * FROM test.kafka") - instance.query("SELECT count() FROM test.destination") - # enough to trigger issue - time.sleep(30) + instance.wait_for_log_line('brokers are down') + instance.wait_for_log_line('stalled. Reschedule', repetitions=2) + kafka_cluster.unpause_container('kafka1') - while int(instance.query("SELECT count() FROM test.destination")) < 20000: - print("Waiting for consume") - time.sleep(1) - + instance.wait_for_log_line("Committed offset 2000") + assert int(instance.query("SELECT count() FROM test.destination")) == 2000 @pytest.mark.timeout(180) def test_kafka_issue14202(kafka_cluster): + """ + INSERT INTO Kafka Engine from an empty SELECT sub query was leading to failure + """ + instance.query(''' CREATE TABLE test.empty_table ( dt Date, @@ -2456,8 +2467,6 @@ def test_kafka_issue14202(kafka_cluster): kafka_format = 'JSONEachRow'; ''') - time.sleep(3) - instance.query( 'INSERT INTO test.kafka_q SELECT t, some_string FROM ( SELECT dt AS t, some_string FROM test.empty_table )') # check instance is alive From f42beb4e6cb3c544b441c05a3235f134437dcaec Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 25 Feb 2021 20:09:13 +0300 Subject: [PATCH 402/510] add test --- tests/integration/test_distributed_ddl/test.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/integration/test_distributed_ddl/test.py b/tests/integration/test_distributed_ddl/test.py index 58e1d0d06f7..9af27738eed 100755 --- a/tests/integration/test_distributed_ddl/test.py +++ b/tests/integration/test_distributed_ddl/test.py @@ -343,6 +343,12 @@ def test_replicated_without_arguments(test_cluster): "EXCHANGE TABLES test_atomic.rmt AND test_atomic.rmt_renamed ON CLUSTER cluster") assert instance.query("SELECT countDistinct(uuid) from clusterAllReplicas('cluster', 'system', 'databases') WHERE uuid != 0 AND name='test_atomic'") == "1\n" assert instance.query("SELECT countDistinct(uuid) from clusterAllReplicas('cluster', 'system', 'tables') WHERE uuid != 0 AND name='rmt'") == "1\n" + test_cluster.ddl_check_query(instance, + "CREATE TABLE test_atomic.rrmt ON CLUSTER cluster (n UInt64, m UInt64) ENGINE=ReplicatedReplacingMergeTree(m) ORDER BY n") + test_cluster.ddl_check_query(instance, + "CREATE TABLE test_atomic.rsmt ON CLUSTER cluster (n UInt64, m UInt64, k UInt64) ENGINE=ReplicatedSummingMergeTree((m, k)) ORDER BY n") + test_cluster.ddl_check_query(instance, + "CREATE TABLE test_atomic.rvcmt ON CLUSTER cluster (n UInt64, m Int8, k UInt64) ENGINE=ReplicatedVersionedCollapsingMergeTree(m, k) ORDER BY n") test_cluster.ddl_check_query(instance, "DROP DATABASE test_atomic ON CLUSTER cluster") test_cluster.ddl_check_query(instance, "CREATE DATABASE test_ordinary ON CLUSTER cluster ENGINE=Ordinary") From 90527556ad8ceeb7cff42f961d27aa4887b999d9 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 25 Feb 2021 22:21:11 +0300 Subject: [PATCH 403/510] Fix by the comments MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил на основании комментарий разработчика. --- docs/en/sql-reference/table-functions/url.md | 2 +- docs/ru/operations/caches.md | 2 +- docs/ru/sql-reference/table-functions/file.md | 6 +++--- docs/ru/sql-reference/table-functions/remote.md | 2 +- docs/ru/sql-reference/table-functions/url.md | 2 +- 5 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 0d004f9601a..4698c852431 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -17,7 +17,7 @@ url(URL, format, structure) **Parameters** -- `URL` — HTTP or HTTPS server address, which can accept `GET` (for `SELECT`) or `POST` (for `INSERT`) queries. Type: [String](../../sql-reference/data-types/string.md). +- `URL` — HTTP or HTTPS server address, which can accept `GET` or `POST` queries (for `SELECT` or `INSERT` queries correspondingly). Type: [String](../../sql-reference/data-types/string.md). - `format` — [Format](../../interfaces/formats.md#formats) of the data. Type: [String](../../sql-reference/data-types/string.md). - `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). diff --git a/docs/ru/operations/caches.md b/docs/ru/operations/caches.md index 9a8092c3c39..7744c596cd9 100644 --- a/docs/ru/operations/caches.md +++ b/docs/ru/operations/caches.md @@ -9,7 +9,7 @@ toc_title: Кеши Основные типы кеша: -- `mark_cache` — кеш меток, используемых движками таблиц семейства [MergeTree](../engines/table-engines/mergetree-family/mergetree.md). +- `mark_cache` — кеш засечек, используемых движками таблиц семейства [MergeTree](../engines/table-engines/mergetree-family/mergetree.md). - `uncompressed_cache` — кеш несжатых данных, используемых движками таблиц семейства [MergeTree](../engines/table-engines/mergetree-family/mergetree.md). Дополнительные типы кеша: diff --git a/docs/ru/sql-reference/table-functions/file.md b/docs/ru/sql-reference/table-functions/file.md index 3cb7043929a..f9bdf902ad8 100644 --- a/docs/ru/sql-reference/table-functions/file.md +++ b/docs/ru/sql-reference/table-functions/file.md @@ -5,7 +5,7 @@ toc_title: file # file {#file} -Создаёт таблицу из файла. Данная табличная функция похожа на табличные функции [url](../../sql-reference/table-functions/url.md) and [hdfs](../../sql-reference/table-functions/hdfs.md). +Создаёт таблицу из файла. Данная табличная функция похожа на табличные функции [url](../../sql-reference/table-functions/url.md) и [hdfs](../../sql-reference/table-functions/hdfs.md). Функция `file` может использоваться в запросах `SELECT` и `INSERT` при работе с движком таблиц [File](../../engines/table-engines/special/file.md). @@ -72,9 +72,9 @@ SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 U └─────────┴─────────┴─────────┘ ``` -## Шаблоны в компонентах пути {#globs-in-path} +## Шаблоны поиска в компонентах пути {#globs-in-path} -При описании пути к файлу могут использоваться шаблоны. Обрабатываются только те файлы, у которых путь и название соответствуют шаблону полностью (а не только префикс или суффикс). +При описании пути к файлу могут использоваться шаблоны поиска. Обрабатываются только те файлы, у которых путь и название соответствуют шаблону полностью (а не только префикс или суффикс). - `*` — заменяет любое количество любых символов кроме `/`, включая отсутствие символов. - `?` — заменяет ровно один любой символ. diff --git a/docs/ru/sql-reference/table-functions/remote.md b/docs/ru/sql-reference/table-functions/remote.md index 0cb32861d1e..83b3687f61d 100644 --- a/docs/ru/sql-reference/table-functions/remote.md +++ b/docs/ru/sql-reference/table-functions/remote.md @@ -22,7 +22,7 @@ remoteSecure('addresses_expr', db.table[, 'user'[, 'password']]) - `addresses_expr` — выражение, генерирующее адреса удалённых серверов. Это может быть просто один адрес сервера. Адрес сервера — это `host:port` или только `host`. - Вместо параметра `host' может быть указано имя сервера или его адрес в формате IPv4 или IPv6. IPv6 адрес указывается в квадратных скобках. + Вместо параметра `host` может быть указано имя сервера или его адрес в формате IPv4 или IPv6. IPv6 адрес указывается в квадратных скобках. `port` — TCP-порт удалённого сервера. Если порт не указан, используется [tcp_port](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port) из конфигурационного файла сервера, к которому обратились через функцию `remote` (по умолчанию - 9000), и [tcp_port_secure](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-tcp_port_secure), к которому обратились через функцию `remoteSecure` (по умолчанию — 9440). diff --git a/docs/ru/sql-reference/table-functions/url.md b/docs/ru/sql-reference/table-functions/url.md index fe80f466f54..043a9231e75 100644 --- a/docs/ru/sql-reference/table-functions/url.md +++ b/docs/ru/sql-reference/table-functions/url.md @@ -17,7 +17,7 @@ url(URL, format, structure) **Параметры** -- `URL` — HTTP или HTTPS-адрес сервера, который может принимать запросы `GET` (для `SELECT`) или `POST` (для `INSERT`). Тип: [String](../../sql-reference/data-types/string.md). +- `URL` — HTTP или HTTPS-адрес сервера, который может принимать запросы `GET` или `POST` (для запросов `SELECT` или `INSERT` соответственно). Тип: [String](../../sql-reference/data-types/string.md). - `format` — [формат](../../interfaces/formats.md#formats) данных. Тип: [String](../../sql-reference/data-types/string.md). - `structure` — структура таблицы в формате `'UserID UInt64, Name String'`. Определяет имена и типы столбцов. Тип: [String](../../sql-reference/data-types/string.md). From 84102a22e7868be918df3bef92681c028e2750ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 25 Feb 2021 23:30:55 +0300 Subject: [PATCH 404/510] Trying to make testkeeper better in single server mode --- docker/test/stress/run.sh | 2 + src/Coordination/NuKeeperServer.cpp | 47 ++++++++++++++++++++--- src/Coordination/NuKeeperStateManager.cpp | 1 + src/Coordination/NuKeeperStateManager.h | 5 +++ 4 files changed, 50 insertions(+), 5 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index dc1e4db4477..3a6491323f2 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -81,6 +81,8 @@ clickhouse-client --query "SHOW TABLES FROM test" ./stress --hung-check --output-folder test_output --skip-func-tests "$SKIP_TESTS_OPTION" && echo "OK" > /test_output/script_exit_code.txt || echo "FAIL" > /test_output/script_exit_code.txt stop +# TODO remove me when persistent snapshots will be ready +rm -fr /var/lib/clickhouse/coordination :|| start clickhouse-client --query "SELECT 'Server successfuly started'" > /test_output/alive_check.txt || echo 'Server failed to start' > /test_output/alive_check.txt diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index c0dc3f85343..43bea42a5b8 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -35,11 +35,23 @@ void NuKeeperServer::startup() { state_manager->loadLogStore(state_machine->last_commit_index()); + bool single_server = state_manager->getTotalServers() == 1; nuraft::raft_params params; - params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); - params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); - params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(); + if (single_server) + { + /// Don't make sence in single server mode + params.heart_beat_interval_ = 0; + params.election_timeout_lower_bound_ = 0; + params.election_timeout_upper_bound_ = 0; + } + else + { + params.heart_beat_interval_ = coordination_settings->heart_beat_interval_ms.totalMilliseconds(); + params.election_timeout_lower_bound_ = coordination_settings->election_timeout_lower_bound_ms.totalMilliseconds(); + params.election_timeout_upper_bound_ = coordination_settings->election_timeout_upper_bound_ms.totalMilliseconds(); + } + params.reserved_log_items_ = coordination_settings->reserved_log_items; params.snapshot_distance_ = coordination_settings->snapshot_distance; params.client_req_timeout_ = coordination_settings->operation_timeout_ms.totalMilliseconds(); @@ -161,13 +173,38 @@ bool NuKeeperServer::isLeaderAlive() const nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type type, nuraft::cb_func::Param * /* param */) { - if ((type == nuraft::cb_func::InitialBatchCommited && isLeader()) || type == nuraft::cb_func::BecomeFresh) + /// Only initial record + bool empty_store = state_manager->getLogStore()->size() == 1; + + auto set_initialized = [this] () { std::unique_lock lock(initialized_mutex); initialized_flag = true; initialized_cv.notify_all(); + }; + + switch (type) + { + case nuraft::cb_func::BecomeLeader: + { + if (empty_store) /// We become leader and store is empty, ready to serve requests + set_initialized(); + return nuraft::cb_func::ReturnCode::Ok; + } + case nuraft::cb_func::BecomeFresh: + { + set_initialized(); /// We are fresh follower, ready to serve requests. + return nuraft::cb_func::ReturnCode::Ok; + } + case nuraft::cb_func::InitialBatchCommited: + { + if (isLeader()) /// We have commited our log store and we are leader, ready to serve requests. + set_initialized(); + return nuraft::cb_func::ReturnCode::Ok; + } + default: /// ignore other events + return nuraft::cb_func::ReturnCode::Ok; } - return nuraft::cb_func::ReturnCode::Ok; } void NuKeeperServer::waitInit() diff --git a/src/Coordination/NuKeeperStateManager.cpp b/src/Coordination/NuKeeperStateManager.cpp index 83cb18e27fe..a7d8b345fee 100644 --- a/src/Coordination/NuKeeperStateManager.cpp +++ b/src/Coordination/NuKeeperStateManager.cpp @@ -33,6 +33,7 @@ NuKeeperStateManager::NuKeeperStateManager( Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_prefix + ".raft_configuration", keys); + total_servers = keys.size(); for (const auto & server_key : keys) { diff --git a/src/Coordination/NuKeeperStateManager.h b/src/Coordination/NuKeeperStateManager.h index 66229a3b8d1..c84b0918beb 100644 --- a/src/Coordination/NuKeeperStateManager.h +++ b/src/Coordination/NuKeeperStateManager.h @@ -52,9 +52,14 @@ public: return start_as_follower_servers.count(my_server_id); } + nuraft::ptr getLogStore() const { return log_store; } + + size_t getTotalServers() const { return total_servers; } + private: int my_server_id; int my_port; + size_t total_servers{0}; std::unordered_set start_as_follower_servers; nuraft::ptr log_store; nuraft::ptr my_server_config; From 83cf614c55e454c8a05e578cd00263a86ad6b661 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Fri, 26 Feb 2021 00:37:22 +0300 Subject: [PATCH 405/510] Fix the link MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Правлю битые ссылки. --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 7d607e79246..37e97ca0b6a 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -361,7 +361,7 @@ Result: These functions accept a string and interpret the bytes placed at the beginning of the string as a number in host order (little endian). If the string isn’t long enough, the functions work as if the string is padded with the necessary number of null bytes. If the string is longer than needed, the extra bytes are ignored. A date is interpreted as the number of days since the beginning of the Unix Epoch, and a date with time is interpreted as the number of seconds since the beginning of the Unix Epoch. -## reinterpretAsString {#type_conversion_functions-reinterpretAsString} +## reinterpretAsString {#type_conversion_functions-reinterpretasstring} This function accepts a number or date or date with time, and returns a string containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a string that is one byte long. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 319d51d607d..4e0d1f56729 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -361,7 +361,7 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; Функции принимают строку и интерпретируют байты, расположенные в начале строки, как число в host order (little endian). Если строка имеет недостаточную длину, то функции работают так, как будто строка дополнена необходимым количеством нулевых байт. Если строка длиннее, чем нужно, то лишние байты игнорируются. Дата интерпретируется, как число дней с начала unix-эпохи, а дата-с-временем - как число секунд с начала unix-эпохи. -## reinterpretAsString {#type_conversion_functions-reinterpretAsString} +## reinterpretAsString {#type_conversion_functions-reinterpretasstring} Функция принимает число или дату или дату-с-временем и возвращает строку, содержащую байты, представляющие соответствующее значение в host order (little endian). При этом, отбрасываются нулевые байты с конца. Например, значение 255 типа UInt32 будет строкой длины 1 байт. From c4470ff2835768a086b1b82bdb7aaf4160ea64fb Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 26 Feb 2021 06:38:14 +0100 Subject: [PATCH 406/510] proper placement for test --- tests/integration/test_storage_kafka/test.py | 282 +++++++++---------- 1 file changed, 140 insertions(+), 142 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index fe1b94adf75..a04b5a7e369 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -156,6 +156,146 @@ def avro_confluent_message(schema_registry_client, value): }) return serializer.encode_record_with_schema('test_subject', schema, value) +# Since everything is async and shaky when receiving messages from Kafka, +# we may want to try and check results multiple times in a loop. +def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference'): + fpath = p.join(p.dirname(__file__), ref_file) + with open(fpath) as reference: + if check: + assert TSV(result) == TSV(reference) + else: + return TSV(result) == TSV(reference) + + +# https://stackoverflow.com/a/57692111/1555175 +def describe_consumer_group(name): + client = BrokerConnection('localhost', 9092, socket.AF_INET) + client.connect_blocking() + + list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) + future = client.send(list_members_in_groups) + while not future.is_done: + for resp, f in client.recv(): + f.success(resp) + + (error_code, group_id, state, protocol_type, protocol, members) = future.value.groups[0] + + res = [] + for member in members: + (member_id, client_id, client_host, member_metadata, member_assignment) = member + member_info = {} + member_info['member_id'] = member_id + member_info['client_id'] = client_id + member_info['client_host'] = client_host + member_topics_assignment = [] + for (topic, partitions) in MemberAssignment.decode(member_assignment).assignment: + member_topics_assignment.append({'topic': topic, 'partitions': partitions}) + member_info['assignment'] = member_topics_assignment + res.append(member_info) + return res + + +# Fixtures + +@pytest.fixture(scope="module") +def kafka_cluster(): + try: + global kafka_id + cluster.start() + kafka_id = instance.cluster.kafka_docker_id + print(("kafka_id is {}".format(kafka_id))) + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def kafka_setup_teardown(): + instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') + wait_kafka_is_available() + # print("kafka is available - running test") + yield # run test + + +# Tests + +@pytest.mark.timeout(180) +def test_kafka_settings_old_syntax(kafka_cluster): + assert TSV(instance.query("SELECT * FROM system.macros WHERE macro like 'kafka%' ORDER BY macro", + ignore_error=True)) == TSV('''kafka_broker kafka1 +kafka_client_id instance +kafka_format_json_each_row JSONEachRow +kafka_group_name_new new +kafka_group_name_old old +kafka_topic_new new +kafka_topic_old old +''') + + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); + ''') + + # Don't insert malformed messages since old settings syntax + # doesn't support skipping of broken messages. + messages = [] + for i in range(50): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce('old', messages) + + result = '' + while True: + result += instance.query('SELECT * FROM test.kafka', ignore_error=True) + if kafka_check_result(result): + break + + kafka_check_result(result, True) + + members = describe_consumer_group('old') + assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' + # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) + + +@pytest.mark.timeout(180) +def test_kafka_settings_new_syntax(kafka_cluster): + instance.query(''' + CREATE TABLE test.kafka (key UInt64, value UInt64) + ENGINE = Kafka + SETTINGS kafka_broker_list = '{kafka_broker}:19092', + kafka_topic_list = '{kafka_topic_new}', + kafka_group_name = '{kafka_group_name_new}', + kafka_format = '{kafka_format_json_each_row}', + kafka_row_delimiter = '\\n', + kafka_client_id = '{kafka_client_id} test 1234', + kafka_skip_broken_messages = 1; + ''') + + messages = [] + for i in range(25): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce('new', messages) + + # Insert couple of malformed messages. + kafka_produce('new', ['}{very_broken_message,']) + kafka_produce('new', ['}another{very_broken_message,']) + + messages = [] + for i in range(25, 50): + messages.append(json.dumps({'key': i, 'value': i})) + kafka_produce('new', messages) + + result = '' + while True: + result += instance.query('SELECT * FROM test.kafka', ignore_error=True) + if kafka_check_result(result): + break + + kafka_check_result(result, True) + + members = describe_consumer_group('new') + assert members[0]['client_id'] == 'instance test 1234' + @pytest.mark.timeout(180) def test_kafka_json_as_string(kafka_cluster): @@ -574,148 +714,6 @@ def test_kafka_formats(kafka_cluster): '''.format(topic_name=topic_name, offset_0=offsets[0], offset_1=offsets[1], offset_2=offsets[2]) assert TSV(result) == TSV(expected), 'Proper result for format: {}'.format(format_name) - -# Since everything is async and shaky when receiving messages from Kafka, -# we may want to try and check results multiple times in a loop. -def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference'): - fpath = p.join(p.dirname(__file__), ref_file) - with open(fpath) as reference: - if check: - assert TSV(result) == TSV(reference) - else: - return TSV(result) == TSV(reference) - - -# https://stackoverflow.com/a/57692111/1555175 -def describe_consumer_group(name): - client = BrokerConnection('localhost', 9092, socket.AF_INET) - client.connect_blocking() - - list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) - future = client.send(list_members_in_groups) - while not future.is_done: - for resp, f in client.recv(): - f.success(resp) - - (error_code, group_id, state, protocol_type, protocol, members) = future.value.groups[0] - - res = [] - for member in members: - (member_id, client_id, client_host, member_metadata, member_assignment) = member - member_info = {} - member_info['member_id'] = member_id - member_info['client_id'] = client_id - member_info['client_host'] = client_host - member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode(member_assignment).assignment: - member_topics_assignment.append({'topic': topic, 'partitions': partitions}) - member_info['assignment'] = member_topics_assignment - res.append(member_info) - return res - - -# Fixtures - -@pytest.fixture(scope="module") -def kafka_cluster(): - try: - global kafka_id - cluster.start() - kafka_id = instance.cluster.kafka_docker_id - print(("kafka_id is {}".format(kafka_id))) - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture(autouse=True) -def kafka_setup_teardown(): - instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() - # print("kafka is available - running test") - yield # run test - - -# Tests - -@pytest.mark.timeout(180) -def test_kafka_settings_old_syntax(kafka_cluster): - assert TSV(instance.query("SELECT * FROM system.macros WHERE macro like 'kafka%' ORDER BY macro", - ignore_error=True)) == TSV('''kafka_broker kafka1 -kafka_client_id instance -kafka_format_json_each_row JSONEachRow -kafka_group_name_new new -kafka_group_name_old old -kafka_topic_new new -kafka_topic_old old -''') - - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka('{kafka_broker}:19092', '{kafka_topic_old}', '{kafka_group_name_old}', '{kafka_format_json_each_row}', '\\n'); - ''') - - # Don't insert malformed messages since old settings syntax - # doesn't support skipping of broken messages. - messages = [] - for i in range(50): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('old', messages) - - result = '' - while True: - result += instance.query('SELECT * FROM test.kafka', ignore_error=True) - if kafka_check_result(result): - break - - kafka_check_result(result, True) - - members = describe_consumer_group('old') - assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' - # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) - - -@pytest.mark.timeout(180) -def test_kafka_settings_new_syntax(kafka_cluster): - instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) - ENGINE = Kafka - SETTINGS kafka_broker_list = '{kafka_broker}:19092', - kafka_topic_list = '{kafka_topic_new}', - kafka_group_name = '{kafka_group_name_new}', - kafka_format = '{kafka_format_json_each_row}', - kafka_row_delimiter = '\\n', - kafka_client_id = '{kafka_client_id} test 1234', - kafka_skip_broken_messages = 1; - ''') - - messages = [] - for i in range(25): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) - - # Insert couple of malformed messages. - kafka_produce('new', ['}{very_broken_message,']) - kafka_produce('new', ['}another{very_broken_message,']) - - messages = [] - for i in range(25, 50): - messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) - - result = '' - while True: - result += instance.query('SELECT * FROM test.kafka', ignore_error=True) - if kafka_check_result(result): - break - - kafka_check_result(result, True) - - members = describe_consumer_group('new') - assert members[0]['client_id'] == 'instance test 1234' - - @pytest.mark.timeout(180) def test_kafka_issue11308(kafka_cluster): # Check that matview does respect Kafka SETTINGS From 60ede360681b2a9a85fa4cb86525d8495b6b0258 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 26 Feb 2021 06:41:11 +0100 Subject: [PATCH 407/510] simple fix for CI failure --- tests/integration/test_storage_kafka/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index a04b5a7e369..c31f34975d3 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2441,6 +2441,7 @@ def test_kafka_unavailable(kafka_cluster): instance.wait_for_log_line("Committed offset 2000") assert int(instance.query("SELECT count() FROM test.destination")) == 2000 + time.sleep(5) # needed to give time for kafka client in python test to recovery @pytest.mark.timeout(180) def test_kafka_issue14202(kafka_cluster): From 9e3547681ba386c1b8768103a374a0f1d3a116be Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 26 Feb 2021 07:17:31 +0100 Subject: [PATCH 408/510] try to address python kafka client disconnects better --- tests/integration/test_storage_kafka/test.py | 44 +++++++------------- 1 file changed, 16 insertions(+), 28 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index c31f34975d3..2a73375c5ea 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -18,10 +18,8 @@ from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer from kafka.admin import NewTopic -from kafka.protocol.admin import DescribeGroupsRequest_v1 -from kafka.protocol.group import MemberAssignment """ protoc --version @@ -83,12 +81,16 @@ def wait_kafka_is_available(max_retries=50): def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) +def kafka_produce(topic, messages, timestamp=None, retries=2): + producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer, retries=retries, max_in_flight_requests_per_connection=1) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() +## just to ensure the python client / producer is working properly +def kafka_producer_send_heartbeat_msg(max_retries=50): + kafka_produce('test_heartbeat_topic', ['test'], retries=max_retries) + def kafka_consume(topic): consumer = KafkaConsumer(bootstrap_servers="localhost:9092", auto_offset_reset="earliest") consumer.subscribe(topics=(topic)) @@ -166,35 +168,22 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' else: return TSV(result) == TSV(reference) - -# https://stackoverflow.com/a/57692111/1555175 def describe_consumer_group(name): - client = BrokerConnection('localhost', 9092, socket.AF_INET) - client.connect_blocking() - - list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) - future = client.send(list_members_in_groups) - while not future.is_done: - for resp, f in client.recv(): - f.success(resp) - - (error_code, group_id, state, protocol_type, protocol, members) = future.value.groups[0] - + admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + consumer_groups = admin_client.describe_consumer_groups([name]) res = [] - for member in members: - (member_id, client_id, client_host, member_metadata, member_assignment) = member + for member in consumer_groups[0].members: member_info = {} - member_info['member_id'] = member_id - member_info['client_id'] = client_id - member_info['client_host'] = client_host + member_info['member_id'] = member.member_id + member_info['client_id'] = member.client_id + member_info['client_host'] = member.client_host member_topics_assignment = [] - for (topic, partitions) in MemberAssignment.decode(member_assignment).assignment: + for (topic, partitions) in member.member_assignment.assignment: member_topics_assignment.append({'topic': topic, 'partitions': partitions}) member_info['assignment'] = member_topics_assignment res.append(member_info) return res - # Fixtures @pytest.fixture(scope="module") @@ -209,15 +198,14 @@ def kafka_cluster(): finally: cluster.shutdown() - @pytest.fixture(autouse=True) def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() + wait_kafka_is_available() # ensure kafka is alive + kafka_producer_send_heartbeat_msg() # ensure python kafka client is ok # print("kafka is available - running test") yield # run test - # Tests @pytest.mark.timeout(180) From 4bcaed98d8bd1b138b45b21a8577a69a5d9438b2 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 26 Feb 2021 09:49:49 +0300 Subject: [PATCH 409/510] Added "opt_reconnect" parameter to config for controlling MYSQL_OPT_RECONNECT option (#19998) * Support for MYSQL_OPT_RECONNECT option It can now be controlled by .opt_reconnect = {true|false} parameter in the mysql replica config section --- base/mysqlxx/Connection.cpp | 13 +++++++------ base/mysqlxx/Connection.h | 12 +++++++++--- base/mysqlxx/Pool.cpp | 7 ++++++- base/mysqlxx/Pool.h | 9 ++++++--- 4 files changed, 28 insertions(+), 13 deletions(-) diff --git a/base/mysqlxx/Connection.cpp b/base/mysqlxx/Connection.cpp index 8a15115cb06..2dbbc0c73f3 100644 --- a/base/mysqlxx/Connection.cpp +++ b/base/mysqlxx/Connection.cpp @@ -51,10 +51,11 @@ Connection::Connection( const char* ssl_key, unsigned timeout, unsigned rw_timeout, - bool enable_local_infile) + bool enable_local_infile, + bool opt_reconnect) : Connection() { - connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout, enable_local_infile); + connect(db, server, user, password, port, socket, ssl_ca, ssl_cert, ssl_key, timeout, rw_timeout, enable_local_infile, opt_reconnect); } Connection::Connection(const std::string & config_name) @@ -80,7 +81,8 @@ void Connection::connect(const char* db, const char * ssl_key, unsigned timeout, unsigned rw_timeout, - bool enable_local_infile) + bool enable_local_infile, + bool opt_reconnect) { if (is_connected) disconnect(); @@ -104,9 +106,8 @@ void Connection::connect(const char* db, if (mysql_options(driver.get(), MYSQL_OPT_LOCAL_INFILE, &enable_local_infile_arg)) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); - /// Enables auto-reconnect. - bool reconnect = true; - if (mysql_options(driver.get(), MYSQL_OPT_RECONNECT, reinterpret_cast(&reconnect))) + /// See C API Developer Guide: Automatic Reconnection Control + if (mysql_options(driver.get(), MYSQL_OPT_RECONNECT, reinterpret_cast(&opt_reconnect))) throw ConnectionFailed(errorMessage(driver.get()), mysql_errno(driver.get())); /// Specifies particular ssl key and certificate if it needs diff --git a/base/mysqlxx/Connection.h b/base/mysqlxx/Connection.h index ca67db0e0c6..65955136eb1 100644 --- a/base/mysqlxx/Connection.h +++ b/base/mysqlxx/Connection.h @@ -14,6 +14,8 @@ /// Disable LOAD DATA LOCAL INFILE because it is insecure #define MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE false +/// See https://dev.mysql.com/doc/c-api/5.7/en/c-api-auto-reconnect.html +#define MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT true namespace mysqlxx @@ -76,7 +78,8 @@ public: const char * ssl_key = "", unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, - bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); + bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, + bool opt_reconnect = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT); /// Creates connection. Can be used if Poco::Util::Application is using. /// All settings will be got from config_name section of configuration. @@ -96,7 +99,8 @@ public: const char* ssl_key, unsigned timeout = MYSQLXX_DEFAULT_TIMEOUT, unsigned rw_timeout = MYSQLXX_DEFAULT_RW_TIMEOUT, - bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); + bool enable_local_infile = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, + bool opt_reconnect = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT); void connect(const std::string & config_name) { @@ -112,6 +116,7 @@ public: std::string ssl_cert = cfg.getString(config_name + ".ssl_cert", ""); std::string ssl_key = cfg.getString(config_name + ".ssl_key", ""); bool enable_local_infile = cfg.getBool(config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); + bool opt_reconnect = cfg.getBool(config_name + ".opt_reconnect", MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT); unsigned timeout = cfg.getInt(config_name + ".connect_timeout", @@ -135,7 +140,8 @@ public: ssl_key.c_str(), timeout, rw_timeout, - enable_local_infile); + enable_local_infile, + opt_reconnect); } /// If MySQL connection was established. diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index 2cb3e62db84..835a3c19587 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -78,6 +78,9 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co enable_local_infile = cfg.getBool(config_name + ".enable_local_infile", cfg.getBool(parent_config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE)); + + opt_reconnect = cfg.getBool(config_name + ".opt_reconnect", + cfg.getBool(parent_config_name + ".opt_reconnect", MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT)); } else { @@ -96,6 +99,8 @@ Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & co enable_local_infile = cfg.getBool( config_name + ".enable_local_infile", MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE); + + opt_reconnect = cfg.getBool(config_name + ".opt_reconnect", MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT); } connect_timeout = cfg.getInt(config_name + ".connect_timeout", @@ -248,7 +253,7 @@ bool Pool::Entry::tryForceConnected() const if (prev_connection_id != current_connection_id) { auto & logger = Poco::Util::Application::instance().logger(); - logger.information("Connection to mysql server has been reestablished. Connection id changed: %lu -> %lu", + logger.information("Reconnected to mysql server. Connection id changed: %lu -> %lu", prev_connection_id, current_connection_id); } return true; diff --git a/base/mysqlxx/Pool.h b/base/mysqlxx/Pool.h index 83b00e0081a..0f64e58054c 100644 --- a/base/mysqlxx/Pool.h +++ b/base/mysqlxx/Pool.h @@ -165,10 +165,12 @@ public: unsigned rw_timeout_ = MYSQLXX_DEFAULT_RW_TIMEOUT, unsigned default_connections_ = MYSQLXX_POOL_DEFAULT_START_CONNECTIONS, unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS, - unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE) + unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, + bool opt_reconnect_ = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT) : default_connections(default_connections_), max_connections(max_connections_), db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_), - connect_timeout(connect_timeout_), rw_timeout(rw_timeout_), enable_local_infile(enable_local_infile_) {} + connect_timeout(connect_timeout_), rw_timeout(rw_timeout_), enable_local_infile(enable_local_infile_), + opt_reconnect(opt_reconnect_) {} Pool(const Pool & other) : default_connections{other.default_connections}, @@ -177,7 +179,7 @@ public: user{other.user}, password{other.password}, port{other.port}, socket{other.socket}, connect_timeout{other.connect_timeout}, rw_timeout{other.rw_timeout}, - enable_local_infile{other.enable_local_infile} + enable_local_infile{other.enable_local_infile}, opt_reconnect(other.opt_reconnect) {} Pool & operator=(const Pool &) = delete; @@ -231,6 +233,7 @@ private: std::string ssl_cert; std::string ssl_key; bool enable_local_infile; + bool opt_reconnect; /// True if connection was established at least once. bool was_successful{false}; From 56d9d99d1c95525ce91e58afe66fba7fb9d138cc Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Feb 2021 09:59:40 +0300 Subject: [PATCH 410/510] Fix style and bayan --- docker/test/stress/run.sh | 2 +- src/Coordination/NuKeeperServer.cpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/test/stress/run.sh b/docker/test/stress/run.sh index 3a6491323f2..2e1c546ce8c 100755 --- a/docker/test/stress/run.sh +++ b/docker/test/stress/run.sh @@ -82,7 +82,7 @@ clickhouse-client --query "SHOW TABLES FROM test" stop # TODO remove me when persistent snapshots will be ready -rm -fr /var/lib/clickhouse/coordination :|| +rm -fr /var/lib/clickhouse/coordination ||: start clickhouse-client --query "SELECT 'Server successfuly started'" > /test_output/alive_check.txt || echo 'Server failed to start' > /test_output/alive_check.txt diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 43bea42a5b8..400ef8fd58b 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -40,7 +40,7 @@ void NuKeeperServer::startup() nuraft::raft_params params; if (single_server) { - /// Don't make sence in single server mode + /// Don't make sense in single server mode params.heart_beat_interval_ = 0; params.election_timeout_lower_bound_ = 0; params.election_timeout_upper_bound_ = 0; @@ -198,7 +198,7 @@ nuraft::cb_func::ReturnCode NuKeeperServer::callbackFunc(nuraft::cb_func::Type t } case nuraft::cb_func::InitialBatchCommited: { - if (isLeader()) /// We have commited our log store and we are leader, ready to serve requests. + if (isLeader()) /// We have committed our log store and we are leader, ready to serve requests. set_initialized(); return nuraft::cb_func::ReturnCode::Ok; } From 66f603512c2d860ff0834722450b3e95a109c722 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 26 Feb 2021 11:42:37 +0300 Subject: [PATCH 411/510] A followup correction to #19998 (#21221) --- base/mysqlxx/Pool.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index 835a3c19587..b524087836d 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -238,7 +238,8 @@ void Pool::Entry::forceConnected() const pool->ssl_key.c_str(), pool->connect_timeout, pool->rw_timeout, - pool->enable_local_infile); + pool->enable_local_infile, + pool->opt_reconnect); } } @@ -299,7 +300,8 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) ssl_key.c_str(), connect_timeout, rw_timeout, - enable_local_infile); + enable_local_infile, + opt_reconnect); } catch (mysqlxx::ConnectionFailed & e) { From c51773d2b3435d2a4f4009616858aa744249fe17 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Feb 2021 12:11:30 +0300 Subject: [PATCH 412/510] Updated test --- tests/queries/0_stateless/01720_constraints_complex_types.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01720_constraints_complex_types.sql b/tests/queries/0_stateless/01720_constraints_complex_types.sql index 7e400c4931e..273f509b6eb 100644 --- a/tests/queries/0_stateless/01720_constraints_complex_types.sql +++ b/tests/queries/0_stateless/01720_constraints_complex_types.sql @@ -1,3 +1,5 @@ +SET allow_suspicious_low_cardinality_types = 1; + DROP TABLE IF EXISTS constraint_on_nullable_type; CREATE TABLE constraint_on_nullable_type ( @@ -13,8 +15,6 @@ SELECT * FROM constraint_on_nullable_type; DROP TABLE constraint_on_nullable_type; -SET allow_suspicious_low_cardinality_types = 1; - DROP TABLE IF EXISTS constraint_on_low_cardinality_type; CREATE TABLE constraint_on_low_cardinality_type ( From 2b401cfa7c1f2886f40bc77f37ee4b38b86c5eae Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 26 Feb 2021 11:50:01 +0000 Subject: [PATCH 413/510] Update brotli to a version with a recent origin fix: "encoder: fix rare access to uninitialized data in ring-buffer" --- contrib/brotli | 2 +- contrib/brotli-cmake/CMakeLists.txt | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/brotli b/contrib/brotli index 5805f99a533..63be8a99401 160000 --- a/contrib/brotli +++ b/contrib/brotli @@ -1 +1 @@ -Subproject commit 5805f99a533a8f8118699c0100d8c102f3605f65 +Subproject commit 63be8a99401992075c23e99f7c84de1c653e39e2 diff --git a/contrib/brotli-cmake/CMakeLists.txt b/contrib/brotli-cmake/CMakeLists.txt index e22f4593c02..4c5f584de9d 100644 --- a/contrib/brotli-cmake/CMakeLists.txt +++ b/contrib/brotli-cmake/CMakeLists.txt @@ -2,6 +2,8 @@ set(BROTLI_SOURCE_DIR ${ClickHouse_SOURCE_DIR}/contrib/brotli/c) set(BROTLI_BINARY_DIR ${ClickHouse_BINARY_DIR}/contrib/brotli/c) set(SRCS + ${BROTLI_SOURCE_DIR}/enc/command.c + ${BROTLI_SOURCE_DIR}/enc/fast_log.c ${BROTLI_SOURCE_DIR}/dec/bit_reader.c ${BROTLI_SOURCE_DIR}/dec/state.c ${BROTLI_SOURCE_DIR}/dec/huffman.c @@ -26,6 +28,9 @@ set(SRCS ${BROTLI_SOURCE_DIR}/enc/memory.c ${BROTLI_SOURCE_DIR}/common/dictionary.c ${BROTLI_SOURCE_DIR}/common/transform.c + ${BROTLI_SOURCE_DIR}/common/platform.c + ${BROTLI_SOURCE_DIR}/common/context.c + ${BROTLI_SOURCE_DIR}/common/constants.c ) add_library(brotli ${SRCS}) From 17d94065018028773a034a240987bb354966b12a Mon Sep 17 00:00:00 2001 From: Vladimir Date: Fri, 26 Feb 2021 15:13:41 +0300 Subject: [PATCH 414/510] Apply suggestions from code review --- docs/ru/sql-reference/data-types/simpleaggregatefunction.md | 2 +- docs/ru/sql-reference/functions/date-time-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md index 7677b64e924..668b579ff78 100644 --- a/docs/ru/sql-reference/data-types/simpleaggregatefunction.md +++ b/docs/ru/sql-reference/data-types/simpleaggregatefunction.md @@ -36,4 +36,4 @@ CREATE TABLE simple (id UInt64, val SimpleAggregateFunction(sum, Double)) ENGINE=AggregatingMergeTree ORDER BY id; ``` -[Оригинальная статья](https://clickhouse.tech/docs/en/data_types/simpleaggregatefunction/) +[Оригинальная статья](https://clickhouse.tech/docs/en/sql-reference/data-types/simpleaggregatefunction/) diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index bb4c49e898e..9b3eade9d79 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -305,7 +305,7 @@ WITH toDateTime64('2020-01-01 10:20:30.999', 3) AS dt64 SELECT toStartOfSecond(d Переводит дату-с-временем или дату в число типа UInt16, содержащее номер ISO года. ISO год отличается от обычного года, потому что в соответствии с [ISO 8601:1988](https://en.wikipedia.org/wiki/ISO_8601) ISO год начинается необязательно первого января. -**Пример:** +**Пример** Запрос: From 4c30c1009206de439d363c6562b24adf0a1e2fc3 Mon Sep 17 00:00:00 2001 From: feng lv Date: Fri, 26 Feb 2021 12:04:11 +0000 Subject: [PATCH 415/510] add test fix --- .../NormalizeSelectWithUnionQueryVisitor.cpp | 23 +++-- src/Interpreters/executeQuery.cpp | 9 +- src/Interpreters/ya.make | 1 + ...01732_explain_syntax_union_query.reference | 66 ++++++++++++++ .../01732_explain_syntax_union_query.sql | 86 +++++++++++++++++++ 5 files changed, 170 insertions(+), 15 deletions(-) create mode 100644 tests/queries/0_stateless/01732_explain_syntax_union_query.reference create mode 100644 tests/queries/0_stateless/01732_explain_syntax_union_query.sql diff --git a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp index 31d33d0781a..d65755f98ba 100644 --- a/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp +++ b/src/Interpreters/NormalizeSelectWithUnionQueryVisitor.cpp @@ -20,7 +20,7 @@ void NormalizeSelectWithUnionQueryMatcher::getSelectsFromUnionListNode(ASTPtr & return; } - selects.push_back(std::move(ast_select)); + selects.push_back(ast_select); } void NormalizeSelectWithUnionQueryMatcher::visit(ASTPtr & ast, Data & data) @@ -53,15 +53,16 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, if (union_modes[i] == ASTSelectWithUnionQuery::Mode::ALL) { - if (auto * inner_union = select_list[i + 1]->as()) + if (auto * inner_union = select_list[i + 1]->as(); + inner_union && inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL) { /// Inner_union is an UNION ALL list, just lift up for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); ++child) - selects.push_back(std::move(*child)); + selects.push_back(*child); } else - selects.push_back(std::move(select_list[i + 1])); + selects.push_back(select_list[i + 1]); } /// flatten all left nodes and current node to a UNION DISTINCT list else if (union_modes[i] == ASTSelectWithUnionQuery::Mode::DISTINCT) @@ -85,15 +86,23 @@ void NormalizeSelectWithUnionQueryMatcher::visit(ASTSelectWithUnionQuery & ast, /// No UNION DISTINCT or only one child in select_list if (i == -1) { - if (auto * inner_union = select_list[0]->as()) + if (auto * inner_union = select_list[0]->as(); + inner_union && inner_union->union_mode == ASTSelectWithUnionQuery::Mode::ALL) { /// Inner_union is an UNION ALL list, just lift it up for (auto child = inner_union->list_of_selects->children.rbegin(); child != inner_union->list_of_selects->children.rend(); ++child) - selects.push_back(std::move(*child)); + selects.push_back(*child); } else - selects.push_back(std::move(select_list[0])); + selects.push_back(select_list[0]); + } + + /// Just one union type child, lift it up + if (selects.size() == 1 && selects[0]->as()) + { + ast = *(selects[0]->as()); + return; } // reverse children list diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 3d6332b45a0..62986793376 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -480,13 +480,6 @@ static std::tuple executeQueryImpl( NormalizeSelectWithUnionQueryVisitor::Data data{context.getSettingsRef().union_default_mode}; NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); - /// After normalization, if it only has one ASTSelectWithUnionQuery child, - /// we can lift it up, this can reduce one unnecessary recursion later in interpreter phase - auto select_union = ast->as(); - if (select_union && select_union->list_of_selects->children.size() == 1 - && select_union->list_of_selects->children.at(0)->as()) - ast = std::move(select_union->list_of_selects->children.at(0)); - query = serializeAST(*ast); /// Check the limits. @@ -888,7 +881,7 @@ static std::tuple executeQueryImpl( LOG_DEBUG(&Poco::Logger::get("executeQuery"), "Query pipeline:\n{}", msg_buf.str()); } } - } + } catch (...) { if (!internal) diff --git a/src/Interpreters/ya.make b/src/Interpreters/ya.make index 879333db507..3eab077df86 100644 --- a/src/Interpreters/ya.make +++ b/src/Interpreters/ya.make @@ -111,6 +111,7 @@ SRCS( MetricLog.cpp MutationsInterpreter.cpp MySQL/InterpretersMySQLDDLQuery.cpp + NormalizeSelectWithUnionQueryVisitor.cpp NullableUtils.cpp OpenTelemetrySpanLog.cpp OptimizeIfChains.cpp diff --git a/tests/queries/0_stateless/01732_explain_syntax_union_query.reference b/tests/queries/0_stateless/01732_explain_syntax_union_query.reference new file mode 100644 index 00000000000..fe5eb01a7ed --- /dev/null +++ b/tests/queries/0_stateless/01732_explain_syntax_union_query.reference @@ -0,0 +1,66 @@ +SELECT 1 +UNION ALL +SELECT 1 +UNION ALL +SELECT 1 +UNION ALL +SELECT 1 +UNION ALL +SELECT 1 + +SELECT 1 +UNION ALL +( + SELECT 1 + UNION DISTINCT + SELECT 1 + UNION DISTINCT + SELECT 1 +) +UNION ALL +SELECT 1 + +SELECT x +FROM +( + SELECT 1 AS x + UNION ALL + ( + SELECT 1 + UNION DISTINCT + SELECT 1 + UNION DISTINCT + SELECT 1 + ) + UNION ALL + SELECT 1 +) + +SELECT x +FROM +( + SELECT 1 AS x + UNION ALL + SELECT 1 + UNION ALL + SELECT 1 +) + +SELECT 1 +UNION DISTINCT +SELECT 1 +UNION DISTINCT +SELECT 1 + +SELECT 1 + + +( + SELECT 1 + UNION DISTINCT + SELECT 1 + UNION DISTINCT + SELECT 1 +) +UNION ALL +SELECT 1 diff --git a/tests/queries/0_stateless/01732_explain_syntax_union_query.sql b/tests/queries/0_stateless/01732_explain_syntax_union_query.sql new file mode 100644 index 00000000000..0dd1e19e765 --- /dev/null +++ b/tests/queries/0_stateless/01732_explain_syntax_union_query.sql @@ -0,0 +1,86 @@ +EXPLAIN SYNTAX +SELECT 1 +UNION ALL +( + SELECT 1 + UNION ALL + ( + SELECT 1 + UNION ALL + SELECT 1 + ) + UNION ALL + SELECT 1 +); + +SELECT ' '; + +EXPLAIN SYNTAX +SELECT 1 +UNION ALL +( + SELECT 1 + UNION DISTINCT + ( + SELECT 1 + UNION ALL + SELECT 1 + ) + UNION ALL + SELECT 1 +); + +SELECT ' '; + +EXPLAIN SYNTAX +SELECT x +FROM +( + SELECT 1 AS x + UNION ALL + ( + SELECT 1 + UNION DISTINCT + ( + SELECT 1 + UNION ALL + SELECT 1 + ) + UNION ALL + SELECT 1 + ) +); + +SELECT ' '; + +EXPLAIN SYNTAX +SELECT x +FROM +( + SELECT 1 AS x + UNION ALL + ( + SELECT 1 + UNION ALL + SELECT 1 + ) +); + +SELECT ' '; + +EXPLAIN SYNTAX +SELECT 1 +UNION ALL +SELECT 1 +UNION DISTINCT +SELECT 1; + +SELECT ' '; + +EXPLAIN SYNTAX +(((((((((((((((SELECT 1))))))))))))))); + +SELECT ' '; + +EXPLAIN SYNTAX +(((((((((((((((SELECT 1 UNION DISTINCT SELECT 1))) UNION DISTINCT SELECT 1)))) UNION ALL SELECT 1)))))))); From 932286df1ff07550841902b5f878a9482ca48045 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 26 Feb 2021 16:32:34 +0300 Subject: [PATCH 416/510] Convert columns to nullable for totals in join --- src/Interpreters/HashJoin.cpp | 2 +- src/Interpreters/IJoin.h | 2 ++ src/Interpreters/MergeJoin.cpp | 2 +- src/Interpreters/join_common.cpp | 14 ++++++++++++-- src/Interpreters/join_common.h | 2 +- .../0_stateless/01637_nullable_fuzz3.reference | 4 ++++ tests/queries/0_stateless/01637_nullable_fuzz3.sql | 2 ++ 7 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp index 142ab07927f..44d1b434bc0 100644 --- a/src/Interpreters/HashJoin.cpp +++ b/src/Interpreters/HashJoin.cpp @@ -1337,7 +1337,7 @@ void HashJoin::joinBlock(Block & block, ExtraBlockPtr & not_processed) void HashJoin::joinTotals(Block & block) const { - JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, key_names_right, block); + JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, *table_join, block); } diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 3d313adcb6c..ade6eaa0cc9 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -28,7 +28,9 @@ public: virtual void joinBlock(Block & block, std::shared_ptr & not_processed) = 0; virtual bool hasTotals() const = 0; + /// Set totals for right table virtual void setTotals(const Block & block) = 0; + /// Add totals to block from left table virtual void joinTotals(Block & block) const = 0; virtual size_t getTotalRowCount() const = 0; diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index e1e4cd9e8a3..ddeaf053225 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -496,7 +496,7 @@ void MergeJoin::setTotals(const Block & totals_block) void MergeJoin::joinTotals(Block & block) const { - JoinCommon::joinTotals(totals, right_columns_to_add, table_join->keyNamesRight(), block); + JoinCommon::joinTotals(totals, right_columns_to_add, *table_join, block); } void MergeJoin::mergeRightBlocks() diff --git a/src/Interpreters/join_common.cpp b/src/Interpreters/join_common.cpp index a4c39a45efa..69727a2a6bb 100644 --- a/src/Interpreters/join_common.cpp +++ b/src/Interpreters/join_common.cpp @@ -251,13 +251,23 @@ void createMissedColumns(Block & block) } } -void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block) +/// Append totals from right to left block, correct types if needed +void joinTotals(const Block & totals, const Block & columns_to_add, const TableJoin & table_join, Block & block) { + if (table_join.forceNullableLeft()) + convertColumnsToNullable(block); + if (Block totals_without_keys = totals) { - for (const auto & name : key_names_right) + for (const auto & name : table_join.keyNamesRight()) totals_without_keys.erase(totals_without_keys.getPositionByName(name)); + for (auto & col : totals_without_keys) + { + if (table_join.rightBecomeNullable(col.type)) + JoinCommon::convertColumnToNullable(col); + } + for (size_t i = 0; i < totals_without_keys.columns(); ++i) block.insert(totals_without_keys.safeGetByPosition(i)); } diff --git a/src/Interpreters/join_common.h b/src/Interpreters/join_common.h index 6f9f7dd1210..76fbcf8d87d 100644 --- a/src/Interpreters/join_common.h +++ b/src/Interpreters/join_common.h @@ -32,7 +32,7 @@ ColumnRawPtrs extractKeysForJoin(const Block & block_keys, const Names & key_nam void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, const Block & block_right, const Names & key_names_right); void createMissedColumns(Block & block); -void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block); +void joinTotals(const Block & totals, const Block & columns_to_add, const TableJoin & table_join, Block & block); void addDefaultValues(IColumn & column, const DataTypePtr & type, size_t count); diff --git a/tests/queries/0_stateless/01637_nullable_fuzz3.reference b/tests/queries/0_stateless/01637_nullable_fuzz3.reference index d9cf16b9d2a..795a0159ac8 100644 --- a/tests/queries/0_stateless/01637_nullable_fuzz3.reference +++ b/tests/queries/0_stateless/01637_nullable_fuzz3.reference @@ -1,2 +1,6 @@ 0 + +0 + +0 0 0 diff --git a/tests/queries/0_stateless/01637_nullable_fuzz3.sql b/tests/queries/0_stateless/01637_nullable_fuzz3.sql index 21bf8999eae..6cfd0fc7d1c 100644 --- a/tests/queries/0_stateless/01637_nullable_fuzz3.sql +++ b/tests/queries/0_stateless/01637_nullable_fuzz3.sql @@ -1,4 +1,6 @@ DROP TABLE IF EXISTS t; CREATE TABLE t (`item_id` UInt64, `price_sold` Float32, `date` Date) ENGINE = MergeTree ORDER BY item_id; SELECT item_id FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id); +SELECT item_id FROM (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id) SETTINGS join_use_nulls = '1'; +SELECT * FROM (SELECT item_id, sum(price_sold) as price_sold FROM t GROUP BY item_id WITH TOTALS) AS l FULL OUTER JOIN (SELECT item_id, sum(price_sold) as price_sold FROM t GROUP BY item_id WITH TOTALS) AS r USING (item_id) SETTINGS join_use_nulls = '1'; DROP TABLE t; From b8418533a39c69bded8166291934d9a8944aca45 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 26 Feb 2021 17:28:56 +0300 Subject: [PATCH 417/510] Update docs/en/sql-reference/table-functions/url.md --- docs/en/sql-reference/table-functions/url.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/table-functions/url.md b/docs/en/sql-reference/table-functions/url.md index 4698c852431..63b0ff0e152 100644 --- a/docs/en/sql-reference/table-functions/url.md +++ b/docs/en/sql-reference/table-functions/url.md @@ -17,7 +17,7 @@ url(URL, format, structure) **Parameters** -- `URL` — HTTP or HTTPS server address, which can accept `GET` or `POST` queries (for `SELECT` or `INSERT` queries correspondingly). Type: [String](../../sql-reference/data-types/string.md). +- `URL` — HTTP or HTTPS server address, which can accept `GET` or `POST` requests (for `SELECT` or `INSERT` queries correspondingly). Type: [String](../../sql-reference/data-types/string.md). - `format` — [Format](../../interfaces/formats.md#formats) of the data. Type: [String](../../sql-reference/data-types/string.md). - `structure` — Table structure in `'UserID UInt64, Name String'` format. Determines column names and types. Type: [String](../../sql-reference/data-types/string.md). From 2679613503b020402ff1efc0750f8b31f08cf705 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Fri, 26 Feb 2021 18:37:57 +0300 Subject: [PATCH 418/510] Added test to skip list DatabaseReplicated --- tests/queries/skip_list.json | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 475b87ae36a..36cca55779d 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -260,7 +260,8 @@ "00121_drop_column_zookeeper", "00116_storage_set", "00083_create_merge_tree_zookeeper", - "00062_replicated_merge_tree_alter_zookeeper" + "00062_replicated_merge_tree_alter_zookeeper", + "01720_constraints_complex_types" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed From 91f6f9227b7d25ec6597e6129004df00c7a5edcd Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 26 Feb 2021 18:53:40 +0300 Subject: [PATCH 419/510] Small fixes and tests updates --- src/Client/ConnectionEstablisher.cpp | 6 +- src/Client/HedgedConnections.cpp | 57 ++++++---- src/Client/HedgedConnections.h | 9 ++ src/Client/HedgedConnectionsFactory.cpp | 44 +++++--- src/Client/HedgedConnectionsFactory.h | 35 ++++-- src/Client/PacketReceiver.h | 58 ---------- .../test_hedged_requests/configs/users.xml | 2 + .../integration/test_hedged_requests/test.py | 100 +++++++++++++++--- .../configs/users.xml | 2 + 9 files changed, 196 insertions(+), 117 deletions(-) diff --git a/src/Client/ConnectionEstablisher.cpp b/src/Client/ConnectionEstablisher.cpp index 17b8832aab4..2532035fabd 100644 --- a/src/Client/ConnectionEstablisher.cpp +++ b/src/Client/ConnectionEstablisher.cpp @@ -118,11 +118,9 @@ ConnectionEstablisherAsync::ConnectionEstablisherAsync( void ConnectionEstablisherAsync::Routine::ReadCallback::operator()(int fd, const Poco::Timespan & timeout, const std::string &) { - if (connection_establisher_async.socket_fd != fd) + /// Check if it's the first time and we need to add socket fd to epoll. + if (connection_establisher_async.socket_fd == -1) { - if (connection_establisher_async.socket_fd != -1) - connection_establisher_async.epoll.remove(connection_establisher_async.socket_fd); - connection_establisher_async.epoll.add(fd); connection_establisher_async.socket_fd = fd; } diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 9601c035f07..8048d5cf1d4 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -146,11 +146,7 @@ void HedgedConnections::sendQuery( if (!disable_two_level_aggregation) { /// Tell hedged_connections_factory to skip replicas that doesn't support two-level aggregation. - hedged_connections_factory.setSkipPredicate( - [timeouts](Connection * connection) - { - return connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; - }); + hedged_connections_factory.skipReplicasWithTwoLevelAggregationIncompatibility(); } auto send_query = [this, timeouts, query, query_id, stage, client_info, with_pending_data](ReplicaState & replica) @@ -295,6 +291,15 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback) { + /// Firstly, resume replica with the last received packet if needed. + if (replica_with_last_received_packet) + { + ReplicaLocation location = replica_with_last_received_packet.value(); + replica_with_last_received_packet.reset(); + if (resumePacketReceiver(location)) + return location; + } + int event_fd; while (true) { @@ -306,22 +311,8 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As else if (fd_to_replica_location.contains(event_fd)) { ReplicaLocation location = fd_to_replica_location[event_fd]; - ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; - auto res = replica_state.packet_receiver->resume(); - - if (std::holds_alternative(res)) - { - last_received_packet = std::move(std::get(res)); + if (resumePacketReceiver(location)) return location; - } - else if (std::holds_alternative(res)) - { - finishProcessReplica(replica_state, true); - - /// Check if there is no more active connections with the same offset and there is no new replica in process. - if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process) - throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); - } } else if (timeout_fd_to_replica_location.contains(event_fd)) { @@ -336,6 +327,28 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As } }; +bool HedgedConnections::resumePacketReceiver(const HedgedConnections::ReplicaLocation & location) +{ + ReplicaState & replica_state = offset_states[location.offset].replicas[location.index]; + auto res = replica_state.packet_receiver->resume(); + + if (std::holds_alternative(res)) + { + last_received_packet = std::move(std::get(res)); + return true; + } + else if (std::holds_alternative(res)) + { + finishProcessReplica(replica_state, true); + + /// Check if there is no more active connections with the same offset and there is no new replica in process. + if (offset_states[location.offset].active_connection_count == 0 && !offset_states[location.offset].next_replica_in_process) + throw NetException("Receive timeout expired", ErrorCodes::SOCKET_TIMEOUT); + } + + return false; +} + int HedgedConnections::getReadyFileDescriptor(AsyncCallback async_callback) { epoll_event event; @@ -359,6 +372,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli case Protocol::Server::Data: if (!offset_states[replica_location.offset].first_packet_of_data_received) processReceivedFirstDataPacket(replica_location); + replica_with_last_received_packet = replica_location; break; case Protocol::Server::PartUUIDs: case Protocol::Server::Progress: @@ -366,6 +380,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli case Protocol::Server::Totals: case Protocol::Server::Extremes: case Protocol::Server::Log: + replica_with_last_received_packet = replica_location; break; case Protocol::Server::EndOfStream: @@ -423,7 +438,7 @@ void HedgedConnections::startNewReplica() void HedgedConnections::checkNewReplica() { Connection * connection = nullptr; - HedgedConnectionsFactory::State state = hedged_connections_factory.waitForReadyConnections(/*blocking = */false, connection); + HedgedConnectionsFactory::State state = hedged_connections_factory.waitForReadyConnections(connection); processNewReplicaState(state, connection); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 555f601b901..cd47e01db2f 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -3,6 +3,8 @@ #include #include +#include + #include #include #include @@ -118,6 +120,8 @@ private: ReplicaLocation getReadyReplicaLocation(AsyncCallback async_callback = {}); + bool resumePacketReceiver(const ReplicaLocation & replica_location); + void processReceivedFirstDataPacket(const ReplicaLocation & replica_location); void startNewReplica(); @@ -162,6 +166,11 @@ private: /// If we didn't disabled it, we need to skip this replica. bool disable_two_level_aggregation = false; + /// We will save replica with last received packet + /// (except cases when packet type is EndOfStream or Exception) + /// to resume it's packet receiver when new packet is needed. + std::optional replica_with_last_received_packet; + Packet last_received_packet; Epoll epoll; diff --git a/src/Client/HedgedConnectionsFactory.cpp b/src/Client/HedgedConnectionsFactory.cpp index 78031e86c0b..a7f3bfb04d7 100644 --- a/src/Client/HedgedConnectionsFactory.cpp +++ b/src/Client/HedgedConnectionsFactory.cpp @@ -40,17 +40,25 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode size_t min_entries = (settings && settings->skip_unavailable_shards) ? 0 : 1; size_t max_entries; - if (pool_mode == PoolMode::GET_ALL) + switch (pool_mode) { - min_entries = shuffled_pools.size(); - max_entries = shuffled_pools.size(); + case PoolMode::GET_ALL: + { + min_entries = shuffled_pools.size(); + max_entries = shuffled_pools.size(); + break; + } + case PoolMode::GET_ONE: + { + max_entries = 1; + break; + } + case PoolMode::GET_MANY: + { + max_entries = settings ? size_t(settings->max_parallel_replicas) : 1; + break; + } } - else if (pool_mode == PoolMode::GET_ONE) - max_entries = 1; - else if (pool_mode == PoolMode::GET_MANY) - max_entries = settings ? size_t(settings->max_parallel_replicas) : 1; - else - throw DB::Exception("Unknown pool allocation mode", DB::ErrorCodes::LOGICAL_ERROR); std::vector connections; connections.reserve(max_entries); @@ -74,7 +82,7 @@ std::vector HedgedConnectionsFactory::getManyConnections(PoolMode while (connections.size() < max_entries) { /// Set blocking = true to avoid busy-waiting here. - auto state = waitForReadyConnections(/*blocking = */true, connection); + auto state = waitForReadyConnectionsImpl(/*blocking = */true, connection); if (state == State::READY) connections.push_back(connection); else if (state == State::CANNOT_CHOOSE) @@ -111,7 +119,12 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::startNewConnection(Con return state; } -HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnections(bool blocking, Connection *& connection_out) +HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnections(Connection *& connection_out) +{ + return waitForReadyConnectionsImpl(false, connection_out); +} + +HedgedConnectionsFactory::State HedgedConnectionsFactory::waitForReadyConnectionsImpl(bool blocking, Connection *& connection_out) { State state = processEpollEvents(blocking, connection_out); if (state != State::CANNOT_CHOOSE) @@ -254,7 +267,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::processFinishedConnect if (result.is_up_to_date) { ++up_to_date_count; - if (!skip_predicate || !skip_predicate(&*result.entry)) + if (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry)) { replicas[index].is_ready = true; ++ready_replicas_count; @@ -343,7 +356,7 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C if (!result.entry.isNull() && result.is_usable && !replicas[i].is_ready - && (!skip_predicate || !skip_predicate(&*result.entry))) + && (!skip_replicas_with_two_level_aggregation_incompatibility || !isTwoLevelAggregationIncompatible(&*result.entry))) indexes.push_back(i); } @@ -365,5 +378,10 @@ HedgedConnectionsFactory::State HedgedConnectionsFactory::setBestUsableReplica(C return State::READY; } +bool HedgedConnectionsFactory::isTwoLevelAggregationIncompatible(Connection * connection) +{ + return connection->getServerRevision(timeouts) < DBMS_MIN_REVISION_WITH_CURRENT_AGGREGATION_VARIANT_SELECTION_METHOD; +} + } #endif diff --git a/src/Client/HedgedConnectionsFactory.h b/src/Client/HedgedConnectionsFactory.h index f49b368bc4b..c5e8d493efa 100644 --- a/src/Client/HedgedConnectionsFactory.h +++ b/src/Client/HedgedConnectionsFactory.h @@ -53,12 +53,11 @@ public: /// Create and return active connections according to pool_mode. std::vector getManyConnections(PoolMode pool_mode); - /// Try to get connection to the new replica. If start_new_connection is true, we start establishing connection - /// with the new replica. Process all current events in epoll (connections, timeouts), - /// if there is no events in epoll and blocking is false, return NOT_READY. - /// Returned state might be READY, NOT_READY and CANNOT_CHOOSE. + /// Try to get connection to the new replica without blocking. Process all current events in epoll (connections, timeouts), + /// Returned state might be READY (connection established successfully), + /// NOT_READY (there are no ready events now) and CANNOT_CHOOSE (cannot produce new connection anymore). /// If state is READY, replica connection will be written in connection_out. - State waitForReadyConnections(bool blocking, Connection *& connection_out); + State waitForReadyConnections(Connection *& connection_out); State startNewConnection(Connection *& connection_out); @@ -73,11 +72,14 @@ public: int numberOfProcessingReplicas() const; - void setSkipPredicate(std::function pred) { skip_predicate = std::move(pred); } + /// Tell Factory to not return connections with two level aggregation incompatibility. + void skipReplicasWithTwoLevelAggregationIncompatibility() { skip_replicas_with_two_level_aggregation_incompatibility = true; } ~HedgedConnectionsFactory(); private: + State waitForReadyConnectionsImpl(bool blocking, Connection *& connection_out); + /// Try to start establishing connection to the new replica. Return /// the index of the new replica or -1 if cannot start new connection. State startNewConnectionImpl(Connection *& connection_out); @@ -104,6 +106,8 @@ private: State setBestUsableReplica(Connection *& connection_out); + bool isTwoLevelAggregationIncompatible(Connection * connection); + const ConnectionPoolWithFailoverPtr pool; const Settings * settings; const ConnectionTimeouts timeouts; @@ -117,7 +121,9 @@ private: /// Map timeout for changing replica to replica index. std::unordered_map timeout_fd_to_replica_index; - std::function skip_predicate; + /// If this flag is true, don't return connections with + /// two level aggregation incompatibility + bool skip_replicas_with_two_level_aggregation_incompatibility = false; std::shared_ptr table_to_check; int last_used_index = -1; @@ -125,14 +131,27 @@ private: Epoll epoll; Poco::Logger * log; std::string fail_messages; + + /// The maximum number of attempts to connect to replicas. size_t max_tries; + /// Total number of established connections. size_t entries_count = 0; + /// The number of established connections that are usable. size_t usable_count = 0; + /// The number of established connections that are up to date. size_t up_to_date_count = 0; + /// The number of failed connections (replica is considered failed after max_tries attempts to connect). size_t failed_pools_count= 0; + + /// The number of replicas that are in process of connection. size_t replicas_in_process_count = 0; - size_t requested_connections_count = 0; + /// The number of ready replicas (replica is considered ready when it's + /// connection returns outside). size_t ready_replicas_count = 0; + + /// The number of requested in startNewConnection replicas (it's needed for + /// checking the number of requested replicas that are still in process). + size_t requested_connections_count = 0; }; } diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index e82591f9444..39384cb4041 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -33,19 +33,9 @@ public: epoll.add(receive_timeout.getDescriptor()); epoll.add(connection->getSocket()->impl()->sockfd()); - if (-1 == pipe2(pipe_fd, O_NONBLOCK)) - throwFromErrno("Cannot create pipe", ErrorCodes::CANNOT_OPEN_FILE); - epoll.add(pipe_fd[0]); - fiber = boost::context::fiber(std::allocator_arg_t(), fiber_stack, Routine{*this}); } - ~PacketReceiver() - { - close(pipe_fd[0]); - close(pipe_fd[1]); - } - /// Resume packet receiving. std::variant resume() { @@ -64,22 +54,6 @@ public: if (is_read_in_process) return epoll.getFileDescriptor(); - /// Write something in pipe when buffer has pending data, because - /// in this case socket won't be ready in epoll but we need to tell - /// outside that there is more data in buffer. - if (connection->hasReadPendingData()) - { - uint64_t buf = 0; - while (-1 == write(pipe_fd[1], &buf, sizeof(buf))) - { - if (errno == EAGAIN) - break; - - if (errno != EINTR) - throwFromErrno("Cannot write to pipe", ErrorCodes::CANNOT_READ_FROM_SOCKET); - } - } - /// Receiving packet was finished. return std::move(packet); } @@ -98,7 +72,6 @@ private: bool checkReceiveTimeout() { bool is_socket_ready = false; - bool is_pipe_ready = false; bool is_receive_timeout_expired = false; epoll_event events[2]; @@ -109,18 +82,10 @@ private: { if (events[i].data.fd == connection->getSocket()->impl()->sockfd()) is_socket_ready = true; - if (events[i].data.fd == pipe_fd[0]) - is_pipe_ready = true; if (events[i].data.fd == receive_timeout.getDescriptor()) is_receive_timeout_expired = true; } - if (is_pipe_ready) - { - drainPipe(); - return true; - } - if (is_receive_timeout_expired && !is_socket_ready) { receive_timeout.reset(); @@ -130,23 +95,6 @@ private: return true; } - void drainPipe() - { - uint64_t buf; - while (true) - { - ssize_t res = read(pipe_fd[0], &buf, sizeof(buf)); - if (res < 0) - { - if (errno == EAGAIN) - break; - - if (errno != EINTR) - throwFromErrno("Cannot drain pipe_fd", ErrorCodes::CANNOT_READ_FROM_SOCKET); - } - } - } - struct Routine { PacketReceiver & receiver; @@ -209,12 +157,6 @@ private: /// in epoll, so we can return epoll file descriptor outside for polling. Epoll epoll; - /// Pipe is used when there is pending data in buffer - /// after receiving packet socket won't be ready in epoll in this case), - /// so we add pipe_fd in epoll and write something in it to tell - /// outside that we are ready to receive new packet. - int pipe_fd[2]; - /// If and exception occurred in fiber resume, we save it and rethrow. std::exception_ptr exception; diff --git a/tests/integration/test_hedged_requests/configs/users.xml b/tests/integration/test_hedged_requests/configs/users.xml index c95d73a92ed..509d3d12508 100644 --- a/tests/integration/test_hedged_requests/configs/users.xml +++ b/tests/integration/test_hedged_requests/configs/users.xml @@ -3,6 +3,8 @@ in_order + 100 + 2 diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index dc64b8a7ba9..de16b8d361a 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -8,6 +8,7 @@ sys.path.insert(0, os.path.dirname(os.path.dirname(os.path.abspath(__file__)))) from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager +from helpers.test_tools import TSV cluster = ClickHouseCluster(__file__) NODES = {'node_' + str(i): None for i in (1, 2, 3)} @@ -18,7 +19,6 @@ sleep_time = 30 @pytest.fixture(scope="module") def started_cluster(): - cluster = ClickHouseCluster(__file__) NODES['node'] = cluster.add_instance( 'node', with_zookeeper=True, stay_alive=True, main_configs=['configs/remote_servers.xml'], user_configs=['configs/users.xml']) @@ -54,28 +54,43 @@ config = ''' ''' -def check_query(): +def check_query(expected_replica, receive_timeout=300): NODES['node'].restart_clickhouse() # Without hedged requests select query will last more than 30 seconds, # with hedged requests it will last just around 1-2 second start = time.time() - NODES['node'].query("SELECT * FROM distributed"); + result = NODES['node'].query("SELECT hostName(), id FROM distributed SETTINGS receive_timeout={}".format(receive_timeout)); query_time = time.time() - start + + assert TSV(result) == TSV(expected_replica + "\t1") + print("Query time:", query_time) - assert query_time < 10 +def test_stuck_replica(started_cluster): + cluster.pause_container("node_1") + check_query(expected_replica="node_2") + cluster.unpause_container("node_1") + + def test_send_table_status_sleep(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) - time.sleep(2) - check_query() + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + + time.sleep(2) + check_query(expected_replica="node_2") def test_send_table_status_sleep2(started_cluster): NODES['node_1'].replace_config( @@ -86,8 +101,12 @@ def test_send_table_status_sleep2(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + time.sleep(2) - check_query() + check_query(expected_replica="node_3") def test_send_data(started_cluster): @@ -95,9 +114,17 @@ def test_send_data(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + time.sleep(2) - check_query() + check_query(expected_replica="node_2") def test_send_data2(started_cluster): @@ -109,8 +136,12 @@ def test_send_data2(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + time.sleep(2) - check_query() + check_query(expected_replica="node_3") def test_combination1(started_cluster): @@ -123,7 +154,7 @@ def test_combination1(started_cluster): config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) time.sleep(2) - check_query() + check_query(expected_replica="node_3") def test_combination2(started_cluster): @@ -135,8 +166,12 @@ def test_combination2(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + time.sleep(2) - check_query() + check_query(expected_replica="node_3") def test_combination3(started_cluster): @@ -153,7 +188,7 @@ def test_combination3(started_cluster): config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) time.sleep(2) - check_query() + check_query(expected_replica="node_2") def test_combination4(started_cluster): @@ -170,5 +205,44 @@ def test_combination4(started_cluster): config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0)) time.sleep(2) - check_query() + check_query(expected_replica="node_2") + + +def test_receive_timeout1(started_cluster): + # Check the situation when first two replicas get receive timeout + # in establishing connection, but the third replica is ok. + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=3, sleep_in_send_data=0)) + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=3, sleep_in_send_data=0)) + + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=1)) + + time.sleep(2) + check_query(expected_replica="node_3", receive_timeout=2) + + +def test_receive_timeout2(started_cluster): + # Check the situation when first replica get receive timeout + # in packet receiving but there are replicas in process of + # connection establishing. + NODES['node_1'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=4)) + + NODES['node_2'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0)) + + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0)) + + time.sleep(2) + check_query(expected_replica="node_2", receive_timeout=3) diff --git a/tests/integration/test_hedged_requests_parallel/configs/users.xml b/tests/integration/test_hedged_requests_parallel/configs/users.xml index c3ba59294a5..af9d6d96e60 100644 --- a/tests/integration/test_hedged_requests_parallel/configs/users.xml +++ b/tests/integration/test_hedged_requests_parallel/configs/users.xml @@ -4,6 +4,8 @@ in_order 2 + 100 + 2 From d328bfa41f7a4c6d5796b2d8b8747dbe077984a8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Feb 2021 19:29:56 +0300 Subject: [PATCH 420/510] Review fixes. Add setting max_optimizations_to_apply. --- src/Common/ErrorCodes.cpp | 3 ++- src/Core/Settings.h | 1 + src/Interpreters/ActionsDAG.cpp | 9 ++++---- src/Interpreters/ActionsDAG.h | 3 +++ .../ClusterProxy/SelectStreamFactory.cpp | 2 +- src/Interpreters/InterpreterExplainQuery.cpp | 6 +++--- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../InterpreterSelectWithUnionQuery.cpp | 2 +- src/Interpreters/MutationsInterpreter.cpp | 2 +- .../QueryPlan/Optimizations/Optimizations.h | 2 +- .../QueryPlanOptimizationSettings.cpp | 12 +++++++++++ .../QueryPlanOptimizationSettings.h | 20 ++++++++++++++++++ .../QueryPlan/Optimizations/optimizeTree.cpp | 21 +++++++++++++++---- src/Processors/QueryPlan/QueryPlan.cpp | 12 +++++------ src/Processors/QueryPlan/QueryPlan.h | 9 ++++---- .../MergeTree/StorageFromMergeTreeDataPart.h | 2 +- src/Storages/StorageBuffer.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageMaterializedView.cpp | 2 +- src/Storages/StorageMergeTree.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- src/Storages/StorageView.cpp | 2 +- .../01655_plan_optimizations.reference | 2 ++ .../0_stateless/01655_plan_optimizations.sh | 5 ++++- 24 files changed, 92 insertions(+), 35 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp create mode 100644 src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index d0d83448b68..d14c9a7e45e 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -535,12 +535,13 @@ M(566, CANNOT_RMDIR) \ M(567, DUPLICATED_PART_UUIDS) \ M(568, RAFT_ERROR) \ + M(569, TOO_MANY_QUERY_PLAN_OPTIMIZATIONS) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ M(1001, STD_EXCEPTION) \ M(1002, UNKNOWN_EXCEPTION) \ - M(1003, INVALID_SHARD_ID) + M(1003, INVALID_SHARD_ID) \ /* See END */ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9bb9ad30f15..549a1716a44 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -431,6 +431,7 @@ class IColumn; M(UnionMode, union_default_mode, UnionMode::Unspecified, "Set default Union Mode in SelectWithUnion query. Possible values: empty string, 'ALL', 'DISTINCT'. If empty, query without Union Mode will throw exception.", 0) \ M(Bool, optimize_aggregators_of_group_by_keys, true, "Eliminates min/max/any/anyLast aggregators of GROUP BY keys in SELECT section", 0) \ M(Bool, optimize_group_by_function_keys, true, "Eliminates functions of other keys in GROUP BY section", 0) \ + M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ // End of COMMON_SETTINGS // Please add settings related to formats into the FORMAT_FACTORY_SETTINGS below. diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 1406eecc5c0..e994a6a0ef6 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -1215,7 +1215,7 @@ ActionsDAG::SplitResult ActionsDAG::splitActionsForFilter(const std::string & co namespace { -struct ConjinctionNodes +struct ConjunctionNodes { std::vector allowed; std::vector rejected; @@ -1225,9 +1225,9 @@ struct ConjinctionNodes /// Assuming predicate is a conjunction (probably, trivial). /// Find separate conjunctions nodes. Split nodes into allowed and rejected sets. /// Allowed predicate is a predicate which can be calculated using only nodes from allowed_nodes set. -ConjinctionNodes getConjinctionNodes(ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) +ConjunctionNodes getConjunctionNodes(ActionsDAG::Node * predicate, std::unordered_set allowed_nodes) { - ConjinctionNodes conjunction; + ConjunctionNodes conjunction; std::unordered_set allowed; std::unordered_set rejected; @@ -1299,6 +1299,7 @@ ConjinctionNodes getConjinctionNodes(ActionsDAG::Node * predicate, std::unordere if (conjunction.allowed.empty()) { + /// If nothing was added to conjunction, check if it is trivial. if (allowed_nodes.count(predicate)) conjunction.allowed.push_back(predicate); } @@ -1450,7 +1451,7 @@ ActionsDAGPtr ActionsDAG::splitActionsForFilter(const std::string & filter_name, } } - auto conjunction = getConjinctionNodes(predicate, allowed_nodes); + auto conjunction = getConjunctionNodes(predicate, allowed_nodes); auto actions = cloneActionsForConjunction(conjunction.allowed); if (!actions) return nullptr; diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 2e3baa181fd..14ed5df949b 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -152,6 +152,9 @@ public: } }; + /// NOTE: std::list is an implementation detail. + /// It allows to add and remove new nodes inplace without reallocation. + /// Raw pointers to nodes remain valid. using Nodes = std::list; using Inputs = std::vector; diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 47726e49d50..e0fcc4738ba 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -284,7 +284,7 @@ void SelectStreamFactory::createForShard( if (try_results.empty() || local_delay < max_remote_delay) { auto plan = createLocalPlan(modified_query_ast, header, context, stage); - return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline())); + return QueryPipeline::getPipe(std::move(*plan->buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef())))); } else { diff --git a/src/Interpreters/InterpreterExplainQuery.cpp b/src/Interpreters/InterpreterExplainQuery.cpp index b13350d7ba2..5135e40e4dd 100644 --- a/src/Interpreters/InterpreterExplainQuery.cpp +++ b/src/Interpreters/InterpreterExplainQuery.cpp @@ -117,7 +117,7 @@ struct QueryPlanSettings { QueryPlan::ExplainPlanOptions query_plan_options; - /// Apply query plan optimisations. + /// Apply query plan optimizations. bool optimize = true; constexpr static char name[] = "PLAN"; @@ -251,7 +251,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() interpreter.buildQueryPlan(plan); if (settings.optimize) - plan.optimize(); + plan.optimize(QueryPlanOptimizationSettings(context.getSettingsRef())); plan.explainPlan(buf, settings.query_plan_options); } @@ -265,7 +265,7 @@ BlockInputStreamPtr InterpreterExplainQuery::executeImpl() InterpreterSelectWithUnionQuery interpreter(ast.getExplainedQuery(), context, SelectQueryOptions()); interpreter.buildQueryPlan(plan); - auto pipeline = plan.buildQueryPipeline(); + auto pipeline = plan.buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef())); if (settings.graph) { diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 9f97160f77f..f13f8fb4106 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -548,7 +548,7 @@ BlockIO InterpreterSelectQuery::execute() buildQueryPlan(query_plan); - res.pipeline = std::move(*query_plan.buildQueryPipeline()); + res.pipeline = std::move(*query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(context->getSettingsRef()))); return res; } diff --git a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 59fcff61936..b894db79c7b 100644 --- a/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -413,7 +413,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() QueryPlan query_plan; buildQueryPlan(query_plan); - auto pipeline = query_plan.buildQueryPipeline(); + auto pipeline = query_plan.buildQueryPipeline(QueryPlanOptimizationSettings(context->getSettingsRef())); res.pipeline = std::move(*pipeline); res.pipeline.addInterpreterContext(context); diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 528b5ec6d8e..43ab42b42c7 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -756,7 +756,7 @@ QueryPipelinePtr MutationsInterpreter::addStreamsForLaterStages(const std::vecto } } - auto pipeline = plan.buildQueryPipeline(); + auto pipeline = plan.buildQueryPipeline(QueryPlanOptimizationSettings(context.getSettingsRef())); pipeline->addSimpleTransform([&](const Block & header) { return std::make_shared(header); diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index a5c3af488a9..f96237fc71a 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -9,7 +9,7 @@ namespace QueryPlanOptimizations { /// This is the main function which optimizes the whole QueryPlan tree. -void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes); +void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); /// Optimization is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp new file mode 100644 index 00000000000..cbd38d46ebf --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp @@ -0,0 +1,12 @@ +#include +#include + +namespace DB +{ + +QueryPlanOptimizationSettings::QueryPlanOptimizationSettings(const Settings & settings) +{ + max_optimizations_to_apply = settings.query_plan_max_optimizations_to_apply; +} + +} diff --git a/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h new file mode 100644 index 00000000000..074298e24a1 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h @@ -0,0 +1,20 @@ +#pragma once + +#include + +namespace DB +{ + +struct Settings; + +struct QueryPlanOptimizationSettings +{ + QueryPlanOptimizationSettings() = delete; + explicit QueryPlanOptimizationSettings(const Settings & settings); + + /// If not zero, throw if too many optimizations were applied to query plan. + /// It helps to avoid infinite optimization loop. + size_t max_optimizations_to_apply = 0; +}; + +} diff --git a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp index cc81a7f39fc..858bde9c660 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeTree.cpp @@ -1,10 +1,20 @@ #include +#include +#include #include -namespace DB::QueryPlanOptimizations +namespace DB { -void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes) +namespace ErrorCodes +{ + extern const int TOO_MANY_QUERY_PLAN_OPTIMIZATIONS; +} + +namespace QueryPlanOptimizations +{ + +void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes) { const auto & optimizations = getOptimizations(); @@ -23,7 +33,7 @@ void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes) std::stack stack; stack.push(Frame{.node = &root}); - size_t max_optimizations_to_apply = 0; + size_t max_optimizations_to_apply = settings.max_optimizations_to_apply; size_t total_applied_optimizations = 0; while (!stack.empty()) @@ -58,7 +68,9 @@ void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes) continue; if (max_optimizations_to_apply && max_optimizations_to_apply < total_applied_optimizations) - continue; + throw Exception(ErrorCodes::TOO_MANY_QUERY_PLAN_OPTIMIZATIONS, + "Too many optimizations applied to query plan. Current limit {}", + max_optimizations_to_apply); /// Try to apply optimization. auto update_depth = optimization.apply(frame.node, nodes); @@ -81,3 +93,4 @@ void optimizeTree(QueryPlan::Node & root, QueryPlan::Nodes & nodes) } } +} diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 755944fdf9f..f5d5e0d99b7 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -130,10 +130,10 @@ void QueryPlan::addStep(QueryPlanStepPtr step) " input expected", ErrorCodes::LOGICAL_ERROR); } -QueryPipelinePtr QueryPlan::buildQueryPipeline() +QueryPipelinePtr QueryPlan::buildQueryPipeline(const QueryPlanOptimizationSettings & optimization_settings) { checkInitialized(); - optimize(); + optimize(optimization_settings); struct Frame { @@ -177,7 +177,7 @@ QueryPipelinePtr QueryPlan::buildQueryPipeline() return last_pipeline; } -Pipe QueryPlan::convertToPipe() +Pipe QueryPlan::convertToPipe(const QueryPlanOptimizationSettings & optimization_settings) { if (!isInitialized()) return {}; @@ -185,7 +185,7 @@ Pipe QueryPlan::convertToPipe() if (isCompleted()) throw Exception("Cannot convert completed QueryPlan to Pipe", ErrorCodes::LOGICAL_ERROR); - return QueryPipeline::getPipe(std::move(*buildQueryPipeline())); + return QueryPipeline::getPipe(std::move(*buildQueryPipeline(optimization_settings))); } void QueryPlan::addInterpreterContext(std::shared_ptr context) @@ -333,9 +333,9 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio } } -void QueryPlan::optimize() +void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { - QueryPlanOptimizations::optimizeTree(*root, nodes); + QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes); } } diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index 9d2d7d93a36..7973f9af45a 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -5,6 +5,7 @@ #include #include +#include namespace DB { @@ -27,7 +28,7 @@ class Pipe; /// A tree of query steps. /// The goal of QueryPlan is to build QueryPipeline. -/// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimisations. +/// QueryPlan let delay pipeline creation which is helpful for pipeline-level optimizations. class QueryPlan { public: @@ -43,12 +44,12 @@ public: bool isCompleted() const; /// Tree is not empty and root hasOutputStream() const DataStream & getCurrentDataStream() const; /// Checks that (isInitialized() && !isCompleted()) - void optimize(); + void optimize(const QueryPlanOptimizationSettings & optimization_settings); - QueryPipelinePtr buildQueryPipeline(); + QueryPipelinePtr buildQueryPipeline(const QueryPlanOptimizationSettings & optimization_settings); /// If initialized, build pipeline and convert to pipe. Otherwise, return empty pipe. - Pipe convertToPipe(); + Pipe convertToPipe(const QueryPlanOptimizationSettings & optimization_settings); struct ExplainPlanOptions { diff --git a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h index 98e1880de54..1d011effc69 100644 --- a/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h +++ b/src/Storages/MergeTree/StorageFromMergeTreeDataPart.h @@ -33,7 +33,7 @@ public: std::move(*MergeTreeDataSelectExecutor(part->storage) .readFromParts({part}, column_names, metadata_snapshot, query_info, context, max_block_size, num_streams)); - return query_plan.convertToPipe(); + return query_plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index e28d5f4d6d1..33aa2140a1f 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -166,7 +166,7 @@ Pipe StorageBuffer::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return plan.convertToPipe(); + return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } void StorageBuffer::read( diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index c08dc38fa2d..f66e2f782ca 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -501,7 +501,7 @@ Pipe StorageDistributed::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return plan.convertToPipe(); + return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } void StorageDistributed::read( diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index af00b37b1d5..02654b37d68 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -126,7 +126,7 @@ Pipe StorageMaterializedView::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return plan.convertToPipe(); + return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } void StorageMaterializedView::read( diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index 202e909af0f..c8f44c78e6e 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -198,7 +198,7 @@ Pipe StorageMergeTree::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return plan.convertToPipe(); + return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } std::optional StorageMergeTree::totalRows(const Settings &) const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 518577c473c..a0d96f43c17 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3809,7 +3809,7 @@ Pipe StorageReplicatedMergeTree::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return plan.convertToPipe(); + return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } diff --git a/src/Storages/StorageView.cpp b/src/Storages/StorageView.cpp index 1ee5ab3d0ca..632d3807f83 100644 --- a/src/Storages/StorageView.cpp +++ b/src/Storages/StorageView.cpp @@ -59,7 +59,7 @@ Pipe StorageView::read( { QueryPlan plan; read(plan, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); - return plan.convertToPipe(); + return plan.convertToPipe(QueryPlanOptimizationSettings(context.getSettingsRef())); } void StorageView::read( diff --git a/tests/queries/0_stateless/01655_plan_optimizations.reference b/tests/queries/0_stateless/01655_plan_optimizations.reference index f261e134494..99b32b74ca7 100644 --- a/tests/queries/0_stateless/01655_plan_optimizations.reference +++ b/tests/queries/0_stateless/01655_plan_optimizations.reference @@ -1,3 +1,5 @@ +Too many optimizations applied to query plan +Too many optimizations applied to query plan > sipHash should be calculated after filtration FUNCTION sipHash64 Filter column: equals diff --git a/tests/queries/0_stateless/01655_plan_optimizations.sh b/tests/queries/0_stateless/01655_plan_optimizations.sh index 84452fe651f..3148dc4a597 100755 --- a/tests/queries/0_stateless/01655_plan_optimizations.sh +++ b/tests/queries/0_stateless/01655_plan_optimizations.sh @@ -4,6 +4,9 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh +$CLICKHOUSE_CLIENT -q "select x + 1 from (select y + 2 as x from (select dummy + 3 as y)) settings query_plan_max_optimizations_to_apply = 1" 2>&1 | + grep -o "Too many optimizations applied to query plan" + echo "> sipHash should be calculated after filtration" $CLICKHOUSE_CLIENT -q "explain actions = 1 select sum(x), sum(y) from (select sipHash64(number) as x, bitAnd(number, 1024) as y from numbers_mt(1000000000) limit 1000000000) where y = 0" | grep -o "FUNCTION sipHash64\|Filter column: equals" echo "> sorting steps should know about limit" @@ -146,4 +149,4 @@ $CLICKHOUSE_CLIENT -q " $CLICKHOUSE_CLIENT -q " select * from ( select y, sum(x) from (select number as x, number % 4 as y from numbers(10)) group by y with totals - ) where y != 2" \ No newline at end of file + ) where y != 2" From b082b661f78e90632bf05d2929ae9428e030de94 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 26 Feb 2021 19:32:12 +0300 Subject: [PATCH 421/510] Fix ya.make --- src/Processors/ya.make | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/ya.make b/src/Processors/ya.make index a44272cf9c0..4b376cdbfb2 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -113,6 +113,7 @@ SRCS( QueryPlan/MergingFinal.cpp QueryPlan/MergingSortedStep.cpp QueryPlan/OffsetStep.cpp + QueryPlan/Optimizations/QueryPlanOptimizationSettings.cpp QueryPlan/Optimizations/filterPushDown.cpp QueryPlan/Optimizations/liftUpArrayJoin.cpp QueryPlan/Optimizations/limitPushDown.cpp From ac4d3b504eb97ddc491b31daee5eeeef04229bd6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 26 Feb 2021 20:15:11 +0300 Subject: [PATCH 422/510] Allow to start up with modified binary under gdb --- programs/server/Server.cpp | 31 +++++++++++++++++++++---------- 1 file changed, 21 insertions(+), 10 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index f501e182cb7..8fcdfc4beac 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -82,6 +82,7 @@ #if defined(OS_LINUX) # include +# include # include #endif @@ -480,16 +481,26 @@ int Server::main(const std::vector & /*args*/) } else { - throw Exception(ErrorCodes::CORRUPTED_DATA, - "Calculated checksum of the ClickHouse binary ({0}) does not correspond" - " to the reference checksum stored in the binary ({1})." - " It may indicate one of the following:" - " - the file {2} was changed just after startup;" - " - the file {2} is damaged on disk due to faulty hardware;" - " - the loaded executable is damaged in memory due to faulty hardware;" - " - the file {2} was intentionally modified;" - " - logical error in code." - , calculated_binary_hash, stored_binary_hash, executable_path); + /// If program is run under debugger, ptrace will fail. + if (ptrace(PTRACE_TRACEME, 0, nullptr, nullptr) == -1) + { + /// Program is run under debugger. Modification of it's binary image is ok for breakpoints. + LOG_WARNING(log, "Server is run under debugger and its binary image is modified (most likely with breakpoints).", + calculated_binary_hash); + } + else + { + throw Exception(ErrorCodes::CORRUPTED_DATA, + "Calculated checksum of the ClickHouse binary ({0}) does not correspond" + " to the reference checksum stored in the binary ({1})." + " It may indicate one of the following:" + " - the file {2} was changed just after startup;" + " - the file {2} is damaged on disk due to faulty hardware;" + " - the loaded executable is damaged in memory due to faulty hardware;" + " - the file {2} was intentionally modified;" + " - logical error in code." + , calculated_binary_hash, stored_binary_hash, executable_path); + } } } else From cb638e0f876ab0162cb97e5fb45fe08c363d74bf Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 26 Feb 2021 21:06:17 +0300 Subject: [PATCH 423/510] Fix style --- src/Client/PacketReceiver.h | 6 ------ 1 file changed, 6 deletions(-) diff --git a/src/Client/PacketReceiver.h b/src/Client/PacketReceiver.h index 39384cb4041..2252e63a2f6 100644 --- a/src/Client/PacketReceiver.h +++ b/src/Client/PacketReceiver.h @@ -13,12 +13,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_READ_FROM_SOCKET; -} - /// Class for nonblocking packet receiving. It runs connection->receivePacket /// in fiber and sets special read callback which is called when /// reading from socket blocks. When read callback is called, From e79c308194127ce7c8653cdb88e29acd1a0101e7 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Fri, 26 Feb 2021 21:49:33 +0300 Subject: [PATCH 424/510] Fix the link MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Пытаюсь поправить битую ссылку. --- docs/en/sql-reference/functions/type-conversion-functions.md | 2 +- docs/ru/sql-reference/functions/type-conversion-functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/type-conversion-functions.md b/docs/en/sql-reference/functions/type-conversion-functions.md index 37e97ca0b6a..7d607e79246 100644 --- a/docs/en/sql-reference/functions/type-conversion-functions.md +++ b/docs/en/sql-reference/functions/type-conversion-functions.md @@ -361,7 +361,7 @@ Result: These functions accept a string and interpret the bytes placed at the beginning of the string as a number in host order (little endian). If the string isn’t long enough, the functions work as if the string is padded with the necessary number of null bytes. If the string is longer than needed, the extra bytes are ignored. A date is interpreted as the number of days since the beginning of the Unix Epoch, and a date with time is interpreted as the number of seconds since the beginning of the Unix Epoch. -## reinterpretAsString {#type_conversion_functions-reinterpretasstring} +## reinterpretAsString {#type_conversion_functions-reinterpretAsString} This function accepts a number or date or date with time, and returns a string containing bytes representing the corresponding value in host order (little endian). Null bytes are dropped from the end. For example, a UInt32 type value of 255 is a string that is one byte long. diff --git a/docs/ru/sql-reference/functions/type-conversion-functions.md b/docs/ru/sql-reference/functions/type-conversion-functions.md index 4e0d1f56729..319d51d607d 100644 --- a/docs/ru/sql-reference/functions/type-conversion-functions.md +++ b/docs/ru/sql-reference/functions/type-conversion-functions.md @@ -361,7 +361,7 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut; Функции принимают строку и интерпретируют байты, расположенные в начале строки, как число в host order (little endian). Если строка имеет недостаточную длину, то функции работают так, как будто строка дополнена необходимым количеством нулевых байт. Если строка длиннее, чем нужно, то лишние байты игнорируются. Дата интерпретируется, как число дней с начала unix-эпохи, а дата-с-временем - как число секунд с начала unix-эпохи. -## reinterpretAsString {#type_conversion_functions-reinterpretasstring} +## reinterpretAsString {#type_conversion_functions-reinterpretAsString} Функция принимает число или дату или дату-с-временем и возвращает строку, содержащую байты, представляющие соответствующее значение в host order (little endian). При этом, отбрасываются нулевые байты с конца. Например, значение 255 типа UInt32 будет строкой длины 1 байт. From a48bb7e3a01b42fd1e5b4fda7452fe0d73593b96 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 26 Feb 2021 22:05:25 +0300 Subject: [PATCH 425/510] Fix redundant zookeeper reconnect and possibility of multiple zookeeper session for a single ClickHouse server --- docker/test/integration/base/Dockerfile | 3 +- src/Common/ZooKeeper/IKeeper.h | 3 ++ src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 50 ++++++++++--------- src/Common/ZooKeeper/ZooKeeper.h | 15 ++++-- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 + src/Interpreters/Context.cpp | 5 ++ .../integration/test_reload_zookeeper/test.py | 8 +++ 8 files changed, 60 insertions(+), 28 deletions(-) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 4963ff0094d..938d8d45ffd 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -18,7 +18,8 @@ RUN apt-get update \ curl \ tar \ krb5-user \ - iproute2 + iproute2 \ + lsof RUN rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ diff --git a/src/Common/ZooKeeper/IKeeper.h b/src/Common/ZooKeeper/IKeeper.h index c53ea60ec7c..9ff37a7045d 100644 --- a/src/Common/ZooKeeper/IKeeper.h +++ b/src/Common/ZooKeeper/IKeeper.h @@ -391,6 +391,9 @@ public: virtual void multi( const Requests & requests, MultiCallback callback) = 0; + + /// Expire session and finish all pending requests + virtual void finalize() = 0; }; } diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index ca9f584304f..a64b76336d2 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -83,6 +83,7 @@ public: const Requests & requests, MultiCallback callback) override; + void finalize() override; struct Node { @@ -130,7 +131,6 @@ private: void pushRequest(RequestInfo && request); - void finalize(); ThreadFromGlobalPool processing_thread; diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index a1c6eb9b481..599980a867a 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -44,7 +44,7 @@ static void check(Coordination::Error code, const std::string & path) } -void ZooKeeper::init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_, +void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_) { log = &Poco::Logger::get("ZooKeeper"); @@ -60,13 +60,16 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho if (hosts.empty()) throw KeeperException("No hosts passed to ZooKeeper constructor.", Coordination::Error::ZBADARGUMENTS); - std::vector hosts_strings; - splitInto<','>(hosts_strings, hosts); Coordination::ZooKeeper::Nodes nodes; - nodes.reserve(hosts_strings.size()); + nodes.reserve(hosts.size()); + + Strings shuffled_hosts = hosts; + /// Shuffle the hosts to distribute the load among ZooKeeper nodes. + pcg64 generator(randomSeed()); + std::shuffle(shuffled_hosts.begin(), shuffled_hosts.end(), generator); bool dns_error = false; - for (auto & host_string : hosts_strings) + for (auto & host_string : shuffled_hosts) { try { @@ -109,9 +112,9 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho Poco::Timespan(0, operation_timeout_ms_ * 1000)); if (chroot.empty()) - LOG_TRACE(log, "Initialized, hosts: {}", hosts); + LOG_TRACE(log, "Initialized, hosts: {}", fmt::join(hosts, ",")); else - LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", hosts, chroot); + LOG_TRACE(log, "Initialized, hosts: {}, chroot: {}", fmt::join(hosts, ","), chroot); } else if (implementation == "testkeeper") { @@ -128,7 +131,16 @@ void ZooKeeper::init(const std::string & implementation_, const std::string & ho throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", Coordination::Error::ZNONODE); } -ZooKeeper::ZooKeeper(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, +ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_, + int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_) +{ + Strings hosts_; + splitInto<','>(hosts_, hosts_string); + + init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); +} + +ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_) { init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); @@ -141,8 +153,6 @@ struct ZooKeeperArgs Poco::Util::AbstractConfiguration::Keys keys; config.keys(config_name, keys); - std::vector hosts_strings; - session_timeout_ms = Coordination::DEFAULT_SESSION_TIMEOUT_MS; operation_timeout_ms = Coordination::DEFAULT_OPERATION_TIMEOUT_MS; implementation = "zookeeper"; @@ -150,7 +160,7 @@ struct ZooKeeperArgs { if (startsWith(key, "node")) { - hosts_strings.push_back( + hosts.push_back( (config.getBool(config_name + "." + key + ".secure", false) ? "secure://" : "") + config.getString(config_name + "." + key + ".host") + ":" + config.getString(config_name + "." + key + ".port", "2181") @@ -180,17 +190,6 @@ struct ZooKeeperArgs throw KeeperException(std::string("Unknown key ") + key + " in config file", Coordination::Error::ZBADARGUMENTS); } - /// Shuffle the hosts to distribute the load among ZooKeeper nodes. - pcg64 generator(randomSeed()); - std::shuffle(hosts_strings.begin(), hosts_strings.end(), generator); - - for (auto & host : hosts_strings) - { - if (!hosts.empty()) - hosts += ','; - hosts += host; - } - if (!chroot.empty()) { if (chroot.front() != '/') @@ -200,7 +199,7 @@ struct ZooKeeperArgs } } - std::string hosts; + Strings hosts; std::string identity; int session_timeout_ms; int operation_timeout_ms; @@ -922,6 +921,10 @@ Coordination::Error ZooKeeper::tryMultiNoThrow(const Coordination::Requests & re } } +void ZooKeeper::finalize() +{ + impl->finalize(); +} size_t KeeperMultiException::getFailedOpIndex(Coordination::Error exception_code, const Coordination::Responses & responses) { @@ -1000,4 +1003,5 @@ Coordination::RequestPtr makeCheckRequest(const std::string & path, int version) request->version = version; return request; } + } diff --git a/src/Common/ZooKeeper/ZooKeeper.h b/src/Common/ZooKeeper/ZooKeeper.h index 5b37e4d6024..4a65ff070f7 100644 --- a/src/Common/ZooKeeper/ZooKeeper.h +++ b/src/Common/ZooKeeper/ZooKeeper.h @@ -50,7 +50,14 @@ class ZooKeeper public: using Ptr = std::shared_ptr; - ZooKeeper(const std::string & hosts_, const std::string & identity_ = "", + /// hosts_string -- comma separated [secure://]host:port list + ZooKeeper(const std::string & hosts_string, const std::string & identity_ = "", + int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, + int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, + const std::string & chroot_ = "", + const std::string & implementation_ = "zookeeper"); + + ZooKeeper(const Strings & hosts_, const std::string & identity_ = "", int32_t session_timeout_ms_ = Coordination::DEFAULT_SESSION_TIMEOUT_MS, int32_t operation_timeout_ms_ = Coordination::DEFAULT_OPERATION_TIMEOUT_MS, const std::string & chroot_ = "", @@ -247,10 +254,12 @@ public: /// Like the previous one but don't throw any exceptions on future.get() FutureMulti tryAsyncMulti(const Coordination::Requests & ops); + void finalize(); + private: friend class EphemeralNodeHolder; - void init(const std::string & implementation_, const std::string & hosts_, const std::string & identity_, + void init(const std::string & implementation_, const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_); /// The following methods don't throw exceptions but return error codes. @@ -266,7 +275,7 @@ private: std::unique_ptr impl; - std::string hosts; + Strings hosts; std::string identity; int32_t session_timeout_ms; int32_t operation_timeout_ms; diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 71b7cd56149..d8b9e23ced2 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -167,6 +167,8 @@ public: const Requests & requests, MultiCallback callback) override; + void finalize() override { finalize(false, false); } + private: String root_path; ACLs default_acls; diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 98e4a87fba3..6f27a6bdb7d 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -1661,7 +1661,12 @@ void Context::resetZooKeeper() const static void reloadZooKeeperIfChangedImpl(const ConfigurationPtr & config, const std::string & config_name, zkutil::ZooKeeperPtr & zk) { if (!zk || zk->configChanged(*config, config_name)) + { + if (zk) + zk->finalize(); + zk = std::make_shared(*config, config_name); + } } void Context::reloadZooKeeperIfChanged(const ConfigurationPtr & config) const diff --git a/tests/integration/test_reload_zookeeper/test.py b/tests/integration/test_reload_zookeeper/test.py index 82c47f4ec9e..1fe0ab13a7f 100644 --- a/tests/integration/test_reload_zookeeper/test.py +++ b/tests/integration/test_reload_zookeeper/test.py @@ -74,6 +74,9 @@ def test_reload_zookeeper(start_cluster): with pytest.raises(QueryRuntimeException): node.query("SELECT COUNT() FROM test_table", settings={"select_sequential_consistency" : 1}) + def get_active_zk_connections(): + return str(node.exec_in_container(['bash', '-c', 'lsof -a -i4 -i6 -itcp -w | grep 2181 | grep ESTABLISHED | wc -l'], privileged=True, user='root')).strip() + ## set config to zoo2, server will be normal new_config = """ @@ -89,5 +92,10 @@ def test_reload_zookeeper(start_cluster): node.replace_config("/etc/clickhouse-server/conf.d/zookeeper.xml", new_config) node.query("SYSTEM RELOAD CONFIG") + active_zk_connections = get_active_zk_connections() + assert active_zk_connections == '1', "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) + assert_eq_with_retry(node, "SELECT COUNT() FROM test_table", '1000', retry_count=120, sleep_time=0.5) + active_zk_connections = get_active_zk_connections() + assert active_zk_connections == '1', "Total connections to ZooKeeper not equal to 1, {}".format(active_zk_connections) From bc1a316b28c53b348e91b807fd0dda7abdcc3cdd Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 25 Feb 2021 09:43:29 +0300 Subject: [PATCH 426/510] Bump zookeeper version to 3.6.2 in tests 3.4.12 was released 1.05.2018 --- .../runner/compose/docker_compose_zookeeper.yml | 12 ++++++------ tests/integration/test_zookeeper_config/test.py | 2 +- .../docker-compose/zookeeper-service.yml | 2 +- .../example/docker-compose/zookeeper-service.yml | 2 +- .../docker-compose/zookeeper-service.yml | 2 +- .../docker-compose/zookeeper-service.yml | 2 +- .../docker-compose/zookeeper-service.yml | 2 +- .../rbac/docker-compose/zookeeper-service.yml | 2 +- 8 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_zookeeper.yml b/docker/test/integration/runner/compose/docker_compose_zookeeper.yml index 49e285b5515..1601d217a25 100644 --- a/docker/test/integration/runner/compose/docker_compose_zookeeper.yml +++ b/docker/test/integration/runner/compose/docker_compose_zookeeper.yml @@ -1,11 +1,11 @@ version: '2.3' services: zoo1: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 restart: always environment: ZOO_TICK_TIME: 500 - ZOO_SERVERS: server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 ZOO_MY_ID: 1 JVMFLAGS: -Dzookeeper.forceSync=no volumes: @@ -16,11 +16,11 @@ services: source: ${ZK_DATA_LOG1:-} target: /datalog zoo2: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 restart: always environment: ZOO_TICK_TIME: 500 - ZOO_SERVERS: server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888 ZOO_MY_ID: 2 JVMFLAGS: -Dzookeeper.forceSync=no volumes: @@ -31,11 +31,11 @@ services: source: ${ZK_DATA_LOG2:-} target: /datalog zoo3: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 restart: always environment: ZOO_TICK_TIME: 500 - ZOO_SERVERS: server.1=zoo1:2888:3888 server.2=zoo2:2888:3888 server.3=zoo3:2888:3888 + ZOO_SERVERS: server.1=zoo1:2888:3888;2181 server.2=zoo2:2888:3888;2181 server.3=zoo3:2888:3888;2181 ZOO_MY_ID: 3 JVMFLAGS: -Dzookeeper.forceSync=no volumes: diff --git a/tests/integration/test_zookeeper_config/test.py b/tests/integration/test_zookeeper_config/test.py index eb5ab2da98f..80875da45e0 100644 --- a/tests/integration/test_zookeeper_config/test.py +++ b/tests/integration/test_zookeeper_config/test.py @@ -129,7 +129,7 @@ def test_secure_connection(): # We need absolute path in zookeeper volumes. Generate it dynamically. TEMPLATE = ''' zoo{zoo_id}: - image: zookeeper:3.5.6 + image: zookeeper:3.6.2 restart: always environment: ZOO_TICK_TIME: 500 diff --git a/tests/testflows/aes_encryption/docker-compose/zookeeper-service.yml b/tests/testflows/aes_encryption/docker-compose/zookeeper-service.yml index f3df33358be..f27405b97a2 100644 --- a/tests/testflows/aes_encryption/docker-compose/zookeeper-service.yml +++ b/tests/testflows/aes_encryption/docker-compose/zookeeper-service.yml @@ -2,7 +2,7 @@ version: '2.3' services: zookeeper: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 expose: - "2181" environment: diff --git a/tests/testflows/example/docker-compose/zookeeper-service.yml b/tests/testflows/example/docker-compose/zookeeper-service.yml index 6691a2df31c..ca732a48dbd 100644 --- a/tests/testflows/example/docker-compose/zookeeper-service.yml +++ b/tests/testflows/example/docker-compose/zookeeper-service.yml @@ -2,7 +2,7 @@ version: '2.3' services: zookeeper: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 expose: - "2181" environment: diff --git a/tests/testflows/ldap/authentication/docker-compose/zookeeper-service.yml b/tests/testflows/ldap/authentication/docker-compose/zookeeper-service.yml index 6691a2df31c..ca732a48dbd 100644 --- a/tests/testflows/ldap/authentication/docker-compose/zookeeper-service.yml +++ b/tests/testflows/ldap/authentication/docker-compose/zookeeper-service.yml @@ -2,7 +2,7 @@ version: '2.3' services: zookeeper: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 expose: - "2181" environment: diff --git a/tests/testflows/ldap/external_user_directory/docker-compose/zookeeper-service.yml b/tests/testflows/ldap/external_user_directory/docker-compose/zookeeper-service.yml index 6691a2df31c..ca732a48dbd 100644 --- a/tests/testflows/ldap/external_user_directory/docker-compose/zookeeper-service.yml +++ b/tests/testflows/ldap/external_user_directory/docker-compose/zookeeper-service.yml @@ -2,7 +2,7 @@ version: '2.3' services: zookeeper: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 expose: - "2181" environment: diff --git a/tests/testflows/ldap/role_mapping/docker-compose/zookeeper-service.yml b/tests/testflows/ldap/role_mapping/docker-compose/zookeeper-service.yml index 6691a2df31c..ca732a48dbd 100644 --- a/tests/testflows/ldap/role_mapping/docker-compose/zookeeper-service.yml +++ b/tests/testflows/ldap/role_mapping/docker-compose/zookeeper-service.yml @@ -2,7 +2,7 @@ version: '2.3' services: zookeeper: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 expose: - "2181" environment: diff --git a/tests/testflows/rbac/docker-compose/zookeeper-service.yml b/tests/testflows/rbac/docker-compose/zookeeper-service.yml index f3df33358be..f27405b97a2 100755 --- a/tests/testflows/rbac/docker-compose/zookeeper-service.yml +++ b/tests/testflows/rbac/docker-compose/zookeeper-service.yml @@ -2,7 +2,7 @@ version: '2.3' services: zookeeper: - image: zookeeper:3.4.12 + image: zookeeper:3.6.2 expose: - "2181" environment: From df95411d515500f3a15ba70f237febfe4aa2eb2f Mon Sep 17 00:00:00 2001 From: Sina <30809018+Sin4wd@users.noreply.github.com> Date: Sat, 27 Feb 2021 01:04:35 +0330 Subject: [PATCH 427/510] Fix broken links in "See Also" The link was transformed into "https://clickhouse.tech/docs/en/sql-reference/sql-reference/statements/create/table#ttl-expression" which did not exist and got 404. --- docs/en/sql-reference/statements/alter/ttl.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/ttl.md b/docs/en/sql-reference/statements/alter/ttl.md index 5331afdb2f8..e8bfb78ec68 100644 --- a/docs/en/sql-reference/statements/alter/ttl.md +++ b/docs/en/sql-reference/statements/alter/ttl.md @@ -81,5 +81,5 @@ The `TTL` is no longer there, so the second row is not deleted: ### See Also -- More about the [TTL-expression](../../../sql-reference/statements/create/table#ttl-expression). -- Modify column [with TTL](../../../sql-reference/statements/alter/column#alter_modify-column). +- More about the [TTL-expression](../../../../sql-reference/statements/create/table#ttl-expression). +- Modify column [with TTL](../../../../sql-reference/statements/alter/column#alter_modify-column). From 9e93d7f507a61d747758e349fccc2b6b1057e528 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 27 Feb 2021 11:07:14 +0300 Subject: [PATCH 428/510] Fix tidy and add comments --- src/Common/ZooKeeper/TestKeeper.h | 2 +- src/Common/ZooKeeper/ZooKeeper.cpp | 6 +++--- src/Common/ZooKeeper/ZooKeeperImpl.h | 14 +++++++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 6 ++++++ 4 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Common/ZooKeeper/TestKeeper.h b/src/Common/ZooKeeper/TestKeeper.h index a64b76336d2..b46f98c0074 100644 --- a/src/Common/ZooKeeper/TestKeeper.h +++ b/src/Common/ZooKeeper/TestKeeper.h @@ -30,7 +30,7 @@ using TestKeeperRequestPtr = std::shared_ptr; * * NOTE: You can add various failure modes for better testing. */ -class TestKeeper : public IKeeper +class TestKeeper final : public IKeeper { public: TestKeeper(const String & root_path_, Poco::Timespan operation_timeout_); diff --git a/src/Common/ZooKeeper/ZooKeeper.cpp b/src/Common/ZooKeeper/ZooKeeper.cpp index 599980a867a..330985e1599 100644 --- a/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/src/Common/ZooKeeper/ZooKeeper.cpp @@ -134,10 +134,10 @@ void ZooKeeper::init(const std::string & implementation_, const Strings & hosts_ ZooKeeper::ZooKeeper(const std::string & hosts_string, const std::string & identity_, int32_t session_timeout_ms_, int32_t operation_timeout_ms_, const std::string & chroot_, const std::string & implementation_) { - Strings hosts_; - splitInto<','>(hosts_, hosts_string); + Strings hosts_strings; + splitInto<','>(hosts_strings, hosts_string); - init(implementation_, hosts_, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); + init(implementation_, hosts_strings, identity_, session_timeout_ms_, operation_timeout_ms_, chroot_); } ZooKeeper::ZooKeeper(const Strings & hosts_, const std::string & identity_, int32_t session_timeout_ms_, diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index d8b9e23ced2..26d944814d8 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -88,7 +88,7 @@ using namespace DB; /** Usage scenario: look at the documentation for IKeeper class. */ -class ZooKeeper : public IKeeper +class ZooKeeper final : public IKeeper { public: struct Node @@ -167,6 +167,18 @@ public: const Requests & requests, MultiCallback callback) override; + /// Without forcefully invalidating (finalizing) ZooKeeper session before + /// establishing a new one, there was a possibility that server is using + /// two ZooKeeper sessions simultaneously in different parts of code. + /// This is stong antipattern and we always prevented it. + + /// ZooKeeper is linearizeable for writes, but not linearizeable for + /// reads, it only maintains "sequential consistency": in every session + /// you observe all events in order but possibly with some delay. If you + /// perform write in one session, then notify different part of code and + /// it will do read in another session, that read may not see the + /// already performed write. + void finalize() override { finalize(false, false); } private: diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index f2c88cdedd9..94bf2dfcf6d 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -144,6 +144,12 @@ static const auto MUTATIONS_FINALIZING_IDLE_SLEEP_MS = 5 * 1000; void StorageReplicatedMergeTree::setZooKeeper() { + /// Every ReplicatedMergeTree table is using only one ZooKeeper session. + /// But if several ReplicatedMergeTree tables are using different + /// ZooKeeper sessions, some queries like ATTACH PARTITION FROM may have + /// strange effects. So we always use only one session for all tables. + /// (excluding auxiliary zookeepers) + std::lock_guard lock(current_zookeeper_mutex); if (zookeeper_name == default_zookeeper_name) { From 63b95c7451c70f35d3692b88340d87c4a0e4649d Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Sat, 27 Feb 2021 11:18:28 +0300 Subject: [PATCH 429/510] MySQL dictionary source: A mechanism for retrying unexpected connection loss during communication with mysql server (#21237) * Added a new type of Exception for better recognition of connection failures * Added more logging for mysql communication * Retries on connection loss during query. Make MySQL dictionary source resilient to unexpected loss of connection during querying MySQL. This behavior is controlled with ".fail_on_connection_loss" config parameter, which defaults to false. * Stripped some debugging leftover garbage * Minor followup corrections * Corrections after PR comments * Yet more fixes --- base/mysqlxx/Exception.h | 9 ++++ base/mysqlxx/Pool.cpp | 51 +++++++++++++--------- base/mysqlxx/Pool.h | 10 +++-- base/mysqlxx/Query.cpp | 25 ++++++++++- src/Dictionaries/MySQLDictionarySource.cpp | 44 ++++++++++++++++--- src/Dictionaries/MySQLDictionarySource.h | 4 ++ 6 files changed, 113 insertions(+), 30 deletions(-) diff --git a/base/mysqlxx/Exception.h b/base/mysqlxx/Exception.h index eaeb3565af1..48cd0997b94 100644 --- a/base/mysqlxx/Exception.h +++ b/base/mysqlxx/Exception.h @@ -26,6 +26,15 @@ struct ConnectionFailed : public Exception }; +/// Connection to MySQL server was lost +struct ConnectionLost : public Exception +{ + ConnectionLost(const std::string & msg, int code = 0) : Exception(msg, code) {} + const char * name() const throw() override { return "mysqlxx::ConnectionLost"; } + const char * className() const throw() override { return "mysqlxx::ConnectionLost"; } +}; + + /// Erroneous query. struct BadQuery : public Exception { diff --git a/base/mysqlxx/Pool.cpp b/base/mysqlxx/Pool.cpp index b524087836d..cf8b3cf9267 100644 --- a/base/mysqlxx/Pool.cpp +++ b/base/mysqlxx/Pool.cpp @@ -10,7 +10,6 @@ #include -#include #include @@ -41,7 +40,9 @@ void Pool::Entry::decrementRefCount() Pool::Pool(const Poco::Util::AbstractConfiguration & cfg, const std::string & config_name, unsigned default_connections_, unsigned max_connections_, const char * parent_config_name_) - : default_connections(default_connections_), max_connections(max_connections_) + : logger(Poco::Logger::get("mysqlxx::Pool")) + , default_connections(default_connections_) + , max_connections(max_connections_) { server = cfg.getString(config_name + ".host"); @@ -130,20 +131,30 @@ Pool::Entry Pool::get() initialize(); for (;;) { + logger.trace("(%s): Iterating through existing MySQL connections", getDescription()); + for (auto & connection : connections) { if (connection->ref_count == 0) return Entry(connection, this); } + logger.trace("(%s): Trying to allocate a new connection.", getDescription()); if (connections.size() < static_cast(max_connections)) { Connection * conn = allocConnection(); if (conn) return Entry(conn, this); + + logger.trace("(%s): Unable to create a new connection: Allocation failed.", getDescription()); + } + else + { + logger.trace("(%s): Unable to create a new connection: Max number of connections has been reached.", getDescription()); } lock.unlock(); + logger.trace("(%s): Sleeping for %d seconds.", getDescription(), MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); lock.lock(); } @@ -167,8 +178,7 @@ Pool::Entry Pool::tryGet() if (res.tryForceConnected()) /// Tries to reestablish connection as well return res; - auto & logger = Poco::Util::Application::instance().logger(); - logger.information("Idle connection to mysql server cannot be recovered, dropping it."); + logger.debug("(%s): Idle connection to MySQL server cannot be recovered, dropping it.", getDescription()); /// This one is disconnected, cannot be reestablished and so needs to be disposed of. connection_it = connections.erase(connection_it); @@ -191,6 +201,8 @@ Pool::Entry Pool::tryGet() void Pool::removeConnection(Connection* connection) { + logger.trace("(%s): Removing connection.", getDescription()); + std::lock_guard lock(mutex); if (connection) { @@ -215,8 +227,6 @@ void Pool::Entry::forceConnected() const if (data == nullptr) throw Poco::RuntimeException("Tried to access NULL database connection."); - Poco::Util::Application & app = Poco::Util::Application::instance(); - bool first = true; while (!tryForceConnected()) { @@ -225,7 +235,7 @@ void Pool::Entry::forceConnected() const else sleepForSeconds(MYSQLXX_POOL_SLEEP_ON_CONNECT_FAIL); - app.logger().information("MYSQL: Reconnecting to " + pool->description); + pool->logger.debug("Entry: Reconnecting to MySQL server %s", pool->description); data->conn.connect( pool->db.c_str(), pool->server.c_str(), @@ -248,18 +258,22 @@ bool Pool::Entry::tryForceConnected() const { auto * const mysql_driver = data->conn.getDriver(); const auto prev_connection_id = mysql_thread_id(mysql_driver); + + pool->logger.trace("Entry(connection %lu): sending PING to check if it is alive.", prev_connection_id); if (data->conn.ping()) /// Attempts to reestablish lost connection { const auto current_connection_id = mysql_thread_id(mysql_driver); if (prev_connection_id != current_connection_id) { - auto & logger = Poco::Util::Application::instance().logger(); - logger.information("Reconnected to mysql server. Connection id changed: %lu -> %lu", - prev_connection_id, current_connection_id); + pool->logger.debug("Entry(connection %lu): Reconnected to MySQL server. Connection id changed: %lu -> %lu", + current_connection_id, prev_connection_id, current_connection_id); } + + pool->logger.trace("Entry(connection %lu): PING ok.", current_connection_id); return true; } + pool->logger.trace("Entry(connection %lu): PING failed.", prev_connection_id); return false; } @@ -280,15 +294,13 @@ void Pool::initialize() Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) { - Poco::Util::Application & app = Poco::Util::Application::instance(); - - std::unique_ptr conn(new Connection); + std::unique_ptr conn_ptr{new Connection}; try { - app.logger().information("MYSQL: Connecting to " + description); + logger.debug("Connecting to %s", description); - conn->conn.connect( + conn_ptr->conn.connect( db.c_str(), server.c_str(), user.c_str(), @@ -305,25 +317,24 @@ Pool::Connection * Pool::allocConnection(bool dont_throw_if_failed_first_time) } catch (mysqlxx::ConnectionFailed & e) { + logger.error(e.what()); + if ((!was_successful && !dont_throw_if_failed_first_time) || e.errnum() == ER_ACCESS_DENIED_ERROR || e.errnum() == ER_DBACCESS_DENIED_ERROR || e.errnum() == ER_BAD_DB_ERROR) { - app.logger().error(e.what()); throw; } else { - app.logger().error(e.what()); return nullptr; } } + connections.push_back(conn_ptr.get()); was_successful = true; - auto * connection = conn.release(); - connections.push_back(connection); - return connection; + return conn_ptr.release(); } } diff --git a/base/mysqlxx/Pool.h b/base/mysqlxx/Pool.h index 0f64e58054c..b6189663f55 100644 --- a/base/mysqlxx/Pool.h +++ b/base/mysqlxx/Pool.h @@ -6,6 +6,8 @@ #include #include +#include + #include @@ -167,13 +169,13 @@ public: unsigned max_connections_ = MYSQLXX_POOL_DEFAULT_MAX_CONNECTIONS, unsigned enable_local_infile_ = MYSQLXX_DEFAULT_ENABLE_LOCAL_INFILE, bool opt_reconnect_ = MYSQLXX_DEFAULT_MYSQL_OPT_RECONNECT) - : default_connections(default_connections_), max_connections(max_connections_), - db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_), + : logger(Poco::Logger::get("mysqlxx::Pool")), default_connections(default_connections_), + max_connections(max_connections_), db(db_), server(server_), user(user_), password(password_), port(port_), socket(socket_), connect_timeout(connect_timeout_), rw_timeout(rw_timeout_), enable_local_infile(enable_local_infile_), opt_reconnect(opt_reconnect_) {} Pool(const Pool & other) - : default_connections{other.default_connections}, + : logger(other.logger), default_connections{other.default_connections}, max_connections{other.max_connections}, db{other.db}, server{other.server}, user{other.user}, password{other.password}, @@ -203,6 +205,8 @@ public: void removeConnection(Connection * connection); protected: + Poco::Logger & logger; + /// Number of MySQL connections which are created at launch. unsigned default_connections; /// Maximum possible number of connections diff --git a/base/mysqlxx/Query.cpp b/base/mysqlxx/Query.cpp index f3485c54edc..d4514c3e988 100644 --- a/base/mysqlxx/Query.cpp +++ b/base/mysqlxx/Query.cpp @@ -1,11 +1,16 @@ #if __has_include() +#include #include #else +#include #include #endif +#include + #include #include +#include namespace mysqlxx @@ -57,8 +62,24 @@ void Query::reset() void Query::executeImpl() { std::string query_string = query_buf.str(); - if (mysql_real_query(conn->getDriver(), query_string.data(), query_string.size())) - throw BadQuery(errorMessage(conn->getDriver()), mysql_errno(conn->getDriver())); + + MYSQL* mysql_driver = conn->getDriver(); + + auto & logger = Poco::Logger::get("mysqlxx::Query"); + logger.trace("Running MySQL query using connection %lu", mysql_thread_id(mysql_driver)); + if (mysql_real_query(mysql_driver, query_string.data(), query_string.size())) + { + const auto err_no = mysql_errno(mysql_driver); + switch (err_no) + { + case CR_SERVER_GONE_ERROR: + [[fallthrough]]; + case CR_SERVER_LOST: + throw ConnectionLost(errorMessage(mysql_driver), err_no); + default: + throw BadQuery(errorMessage(mysql_driver), err_no); + } + } } UseQueryResult Query::use() diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index a21b1bd50fc..572080aee1e 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -47,11 +47,13 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) # include # include # include "readInvalidateQuery.h" +# include # include namespace DB { static const UInt64 max_block_size = 8192; +static const size_t default_num_tries_on_connection_loss = 3; MySQLDictionarySource::MySQLDictionarySource( @@ -72,7 +74,10 @@ MySQLDictionarySource::MySQLDictionarySource( , query_builder{dict_struct, db, "", table, where, IdentifierQuotingStyle::Backticks} , load_all_query{query_builder.composeLoadAllQuery()} , invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} - , close_connection{config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false)} + , close_connection( + config.getBool(config_prefix + ".close_connection", false) || config.getBool(config_prefix + ".share_connection", false)) + , max_tries_for_mysql_block_input_stream( + config.getBool(config_prefix + ".fail_on_connection_loss", false) ? 1 : default_num_tries_on_connection_loss) { } @@ -94,6 +99,7 @@ MySQLDictionarySource::MySQLDictionarySource(const MySQLDictionarySource & other , invalidate_query{other.invalidate_query} , invalidate_query_response{other.invalidate_query_response} , close_connection{other.close_connection} + , max_tries_for_mysql_block_input_stream{other.max_tries_for_mysql_block_input_stream} { } @@ -114,13 +120,41 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate() } } +BlockInputStreamPtr MySQLDictionarySource::retriedCreateMySqlBIStream(const std::string & data_fetch_query_str, const size_t max_tries) +{ + size_t count_connection_lost = 0; + + while (true) + { + auto connection = pool.get(); + + try + { + return std::make_shared( + connection, data_fetch_query_str, sample_block, max_block_size, close_connection); + } + catch (const mysqlxx::ConnectionLost & ecl) /// There are two retriable failures: CR_SERVER_GONE_ERROR, CR_SERVER_LOST + { + if (++count_connection_lost < max_tries) + { + LOG_WARNING(log, ecl.displayText()); + LOG_WARNING(log, "Lost connection ({}/{}). Trying to reconnect...", count_connection_lost, max_tries); + continue; + } + + LOG_ERROR(log, "Failed ({}/{}) to create BlockInputStream for MySQL dictionary source.", count_connection_lost, max_tries); + throw; + } + } +} + BlockInputStreamPtr MySQLDictionarySource::loadAll() { auto connection = pool.get(); last_modification = getLastModification(connection, false); LOG_TRACE(log, load_all_query); - return std::make_shared(connection, load_all_query, sample_block, max_block_size, close_connection); + return retriedCreateMySqlBIStream(load_all_query, max_tries_for_mysql_block_input_stream); } BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() @@ -130,7 +164,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() std::string load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); - return std::make_shared(connection, load_update_query, sample_block, max_block_size, close_connection); + return retriedCreateMySqlBIStream(load_update_query, max_tries_for_mysql_block_input_stream); } BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector & ids) @@ -138,7 +172,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector & i /// We do not log in here and do not update the modification time, as the request can be large, and often called. const auto query = query_builder.composeLoadIdsQuery(ids); - return std::make_shared(pool.get(), query, sample_block, max_block_size, close_connection); + return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream); } BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) @@ -146,7 +180,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, /// We do not log in here and do not update the modification time, as the request can be large, and often called. const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); - return std::make_shared(pool.get(), query, sample_block, max_block_size, close_connection); + return retriedCreateMySqlBIStream(query, max_tries_for_mysql_block_input_stream); } bool MySQLDictionarySource::isModified() const diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index 34f784cdfeb..54bc2da6a56 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -69,6 +69,9 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; + /// A helper method for recovering from "Lost connection to MySQL server during query" errors + BlockInputStreamPtr retriedCreateMySqlBIStream(const std::string & query_str, const size_t max_tries); + Poco::Logger * log; std::chrono::time_point update_time; @@ -86,6 +89,7 @@ private: std::string invalidate_query; mutable std::string invalidate_query_response; const bool close_connection; + const size_t max_tries_for_mysql_block_input_stream; }; } From 4d7ba0443fadf4086831d56e9f3ec3314d7d3874 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 27 Feb 2021 11:40:03 +0300 Subject: [PATCH 430/510] Add checking pending data --- src/Client/HedgedConnections.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 8048d5cf1d4..558fe844447 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -296,7 +296,7 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As { ReplicaLocation location = replica_with_last_received_packet.value(); replica_with_last_received_packet.reset(); - if (resumePacketReceiver(location)) + if (offset_states[location.offset].replicas[location.index].connection->hasReadPendingData() && resumePacketReceiver(location)) return location; } From 1b1f50e324f31af1c4a1d6a9111915cf9ede12ea Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Sat, 27 Feb 2021 11:45:32 +0300 Subject: [PATCH 431/510] Update comment --- src/Client/HedgedConnections.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 558fe844447..2c9c47ed767 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -291,7 +291,7 @@ Packet HedgedConnections::receivePacketUnlocked(AsyncCallback async_callback) HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(AsyncCallback async_callback) { - /// Firstly, resume replica with the last received packet if needed. + /// Firstly, resume replica with the last received packet if it has pending data. if (replica_with_last_received_packet) { ReplicaLocation location = replica_with_last_received_packet.value(); From 9dc3952ff35f17a5dddb9e50b02da615949a87c5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 27 Feb 2021 16:30:43 +0300 Subject: [PATCH 432/510] Fix typo --- src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ZooKeeper/ZooKeeperImpl.h b/src/Common/ZooKeeper/ZooKeeperImpl.h index 26d944814d8..afd2e89538f 100644 --- a/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -170,7 +170,7 @@ public: /// Without forcefully invalidating (finalizing) ZooKeeper session before /// establishing a new one, there was a possibility that server is using /// two ZooKeeper sessions simultaneously in different parts of code. - /// This is stong antipattern and we always prevented it. + /// This is strong antipattern and we always prevented it. /// ZooKeeper is linearizeable for writes, but not linearizeable for /// reads, it only maintains "sequential consistency": in every session From 4f61ef3099f42f17b496a0b0424773978d9a32dc Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 27 Feb 2021 08:49:45 +0000 Subject: [PATCH 433/510] Tests for zstd and zlib --- ...lib_http_compression_json_format.reference | 23 +++++++++++++++++++ ..._long_zlib_http_compression_json_format.sh | 7 ++++++ ...std_http_compression_json_format.reference | 23 +++++++++++++++++++ ..._long_zstd_http_compression_json_format.sh | 7 ++++++ 4 files changed, 60 insertions(+) create mode 100644 tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference create mode 100755 tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh create mode 100644 tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.reference create mode 100755 tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh diff --git a/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference new file mode 100644 index 00000000000..7c089a2fd05 --- /dev/null +++ b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.reference @@ -0,0 +1,23 @@ + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999998" + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999999" + } + ], + + "rows": 1000000, + + "rows_before_limit_at_least": 1048080, + diff --git a/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh new file mode 100755 index 00000000000..e663b329660 --- /dev/null +++ b/tests/queries/0_stateless/01746_long_zlib_http_compression_json_format.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: gzip' "${CLICKHOUSE_URL}&enable_http_compression=1&http_zlib_compression_level=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | gzip -d | tail -n30 | head -n23 diff --git a/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.reference b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.reference new file mode 100644 index 00000000000..7c089a2fd05 --- /dev/null +++ b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.reference @@ -0,0 +1,23 @@ + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999998" + }, + { + "datetime": "2020-12-12", + "pipeline": "test-pipeline", + "host": "clickhouse-test-host-001.clickhouse.com", + "home": "clickhouse", + "detail": "clickhouse", + "row_number": "999999" + } + ], + + "rows": 1000000, + + "rows_before_limit_at_least": 1048080, + diff --git a/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh new file mode 100755 index 00000000000..f520a21e9b3 --- /dev/null +++ b/tests/queries/0_stateless/01746_long_zstd_http_compression_json_format.sh @@ -0,0 +1,7 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +${CLICKHOUSE_CURL} -sS -H 'Accept-Encoding: zstd' "${CLICKHOUSE_URL}&enable_http_compression=1" -d "SELECT toDate('2020-12-12') as datetime, 'test-pipeline' as pipeline, 'clickhouse-test-host-001.clickhouse.com' as host, 'clickhouse' as home, 'clickhouse' as detail, number as row_number FROM numbers(1000000) FORMAT JSON" | zstd -d | tail -n30 | head -n23 From 2cafd54c17c75450bc3df9bc909e533621fe986e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Feb 2021 17:42:31 +0300 Subject: [PATCH 434/510] Add new dataset: Cell Towers --- .../example-datasets/cell-towers.md | 133 ++++++++++++++++++ .../getting-started/example-datasets/index.md | 1 + 2 files changed, 134 insertions(+) create mode 100644 docs/en/getting-started/example-datasets/cell-towers.md diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md new file mode 100644 index 00000000000..f032be76da0 --- /dev/null +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -0,0 +1,133 @@ +--- +toc_priority: 21 +toc_title: Cell Stations +--- + +# Cell Stations + +This dataset is from [OpenCellid](https://www.opencellid.org/) - The world's largest Open Database of Cell Towers. + +As of 2021 it contains more than 40 million records about cell towers (GSM, LTE, UMTS, etc.) around the world with their geographical coordinates and metadata (country code, network, etc). + +OpenCelliD Project is licensed under a Creative Commons Attribution-ShareAlike 4.0 International License, and we redistribute a snapshot of this dataset under the terms of the same license. The up to date version of the dataset is available to download after sign in. + + +## Get the Dataset + +Download the snapshot of the dataset from Feb 2021: [https://datasets.clickhouse.tech/cell_towers.csv.xz] (729 MB). + +Optionally validate the integrity: +``` +md5sum cell_towers.csv.xz +8cf986f4a0d9f12c6f384a0e9192c908 cell_towers.csv.xz +``` + +Decompress it with the following command: +``` +xz -d cell_towers.csv.xz +``` + +Create a table: + +``` +CREATE TABLE cell_towers +( + radio Enum8('' = 0, 'CDMA' = 1, 'GSM' = 2, 'LTE' = 3, 'NR' = 4, 'UMTS' = 5), + mcc UInt16, + net UInt16, + area UInt16, + cell UInt64, + unit Int16, + lon Float64, + lat Float64, + range UInt32, + samples UInt32, + changeable UInt8, + created DateTime, + updated DateTime, + averageSignal UInt8 +) +ENGINE = MergeTree ORDER BY (radio, mcc, net, created); +``` + +Insert the dataset: +``` +clickhouse-client --query "INSERT INTO cell_towers FORMAT CSVWithNames" < cell_towers.csv +``` + + +## Run some queries + +Number of cell towers by type: +``` +SELECT radio, count() AS c FROM cell_towers GROUP BY radio ORDER BY c DESC + +┌─radio─┬────────c─┐ +│ UMTS │ 20686487 │ +│ LTE │ 12101148 │ +│ GSM │ 9931312 │ +│ CDMA │ 556344 │ +│ NR │ 867 │ +└───────┴──────────┘ + +5 rows in set. Elapsed: 0.011 sec. Processed 43.28 million rows, 43.28 MB (3.83 billion rows/s., 3.83 GB/s.) +``` + +Cell towers by mobile country code (MCC): +``` +SELECT mcc, count() FROM cell_towers GROUP BY mcc ORDER BY count() DESC LIMIT 10 + +┌─mcc─┬─count()─┐ +│ 310 │ 5024650 │ +│ 262 │ 2622423 │ +│ 250 │ 1953176 │ +│ 208 │ 1891187 │ +│ 724 │ 1836150 │ +│ 404 │ 1729151 │ +│ 234 │ 1618924 │ +│ 510 │ 1353998 │ +│ 440 │ 1343355 │ +│ 311 │ 1332798 │ +└─────┴─────────┘ + +10 rows in set. Elapsed: 0.019 sec. Processed 43.28 million rows, 86.55 MB (2.33 billion rows/s., 4.65 GB/s.) +``` + +See the dictionary here: [https://en.wikipedia.org/wiki/Mobile_country_code](https://en.wikipedia.org/wiki/Mobile_country_code). + +So, the top countries are USA, Germany and Russia. + +You may want to create an [External Dictionary](../../sql-reference/dictionaries/external-dictionaries/external-dicts/) in ClickHouse to decode these values. + + +### Example of using `pointInPolygon` function + +Create a table where we will store polygons: + +``` +CREATE TEMPORARY TABLE moscow (polygon Array(Tuple(Float64, Float64))); +``` + +This is a rough shape of Moscow (without "new Moscow"): + +``` +INSERT INTO moscow VALUES ([(37.84172564285271, 55.78000432402266), (37.8381207618713, 55.775874525970494), (37.83979446823122, 55.775626746008065), (37.84243326983639, 55.77446586811748), (37.84262672750849, 55.771974101091104), (37.84153238623039, 55.77114545193181), (37.841124690460184, 55.76722010265554), (37.84239076983644, 55.76654891107098), (37.842283558197025, 55.76258709833121), (37.8421759312134, 55.758073999993734), (37.84198330422974, 55.75381499999371), (37.8416827275085, 55.749277102484484), (37.84157576190186, 55.74794544108413), (37.83897929098507, 55.74525257875241), (37.83739676451868, 55.74404373042019), (37.838732481460525, 55.74298009816793), (37.841183997352545, 55.743060321833575), (37.84097476190185, 55.73938799999373), (37.84048155819702, 55.73570799999372), (37.840095812164286, 55.73228210777237), (37.83983814285274, 55.73080491981639), (37.83846476321406, 55.729799917464675), (37.83835745269769, 55.72919751082619), (37.838636380279524, 55.72859509486539), (37.8395161005249, 55.727705075632784), (37.83897964285276, 55.722727886185154), (37.83862557539366, 55.72034817326636), (37.83559735744853, 55.71944437307499), (37.835370708803126, 55.71831419154461), (37.83738169402022, 55.71765218986692), (37.83823396494291, 55.71691750159089), (37.838056931213345, 55.71547311301385), (37.836812846557606, 55.71221445615604), (37.83522525396725, 55.709331054395555), (37.83269301586908, 55.70953687463627), (37.829667367706236, 55.70903403789297), (37.83311126588435, 55.70552351822608), (37.83058993121339, 55.70041317726053), (37.82983872750851, 55.69883771404813), (37.82934501586913, 55.69718947487017), (37.828926414016685, 55.69504441658371), (37.82876530422971, 55.69287499999378), (37.82894754100031, 55.690759754047335), (37.827697554878185, 55.68951421135665), (37.82447346292115, 55.68965045405069), (37.83136543914793, 55.68322046195302), (37.833554015869154, 55.67814012759211), (37.83544184655761, 55.67295011628339), (37.837480388885474, 55.6672498719639), (37.838960677246064, 55.66316274139358), (37.83926093121332, 55.66046999999383), (37.839025050262435, 55.65869897264431), (37.83670784390257, 55.65794084879904), (37.835656529083245, 55.65694309303843), (37.83704060449217, 55.65689306460552), (37.83696819873806, 55.65550363526252), (37.83760389616388, 55.65487847246661), (37.83687972750851, 55.65356745541324), (37.83515216004943, 55.65155951234079), (37.83312418518067, 55.64979413590619), (37.82801726983639, 55.64640836412121), (37.820614174591, 55.64164525405531), (37.818908190475426, 55.6421883258084), (37.81717543386075, 55.64112490388471), (37.81690987037274, 55.63916106913107), (37.815099354492155, 55.637925371757085), (37.808769150787356, 55.633798276884455), (37.80100123544311, 55.62873670012244), (37.79598013491824, 55.62554336109055), (37.78634567724606, 55.62033499605651), (37.78334147619623, 55.618768681480326), (37.77746201055901, 55.619855533402706), (37.77527329626457, 55.61909966711279), (37.77801986242668, 55.618770300976294), (37.778212973541216, 55.617257701952106), (37.77784818518065, 55.61574504433011), (37.77016867724609, 55.61148576294007), (37.760191219573976, 55.60599579539028), (37.75338926983641, 55.60227892751446), (37.746329965606634, 55.59920577639331), (37.73939925396728, 55.59631430313617), (37.73273665739439, 55.5935318803559), (37.7299954450912, 55.59350760316188), (37.7268679946899, 55.59469840523759), (37.72626726983634, 55.59229549697373), (37.7262673598022, 55.59081598950582), (37.71897193121335, 55.5877595845419), (37.70871550793456, 55.58393177431724), (37.700497489410374, 55.580917323756644), (37.69204305026244, 55.57778089778455), (37.68544477378839, 55.57815154690915), (37.68391050793454, 55.57472945079756), (37.678803592590306, 55.57328235936491), (37.6743402539673, 55.57255251445782), (37.66813862698363, 55.57216388774464), (37.617927457672096, 55.57505691895805), (37.60443099999999, 55.5757737568051), (37.599683515869145, 55.57749105910326), (37.59754177842709, 55.57796291823627), (37.59625834786988, 55.57906686095235), (37.59501783265684, 55.57746616444403), (37.593090671936025, 55.57671634534502), (37.587018007904, 55.577944600233785), (37.578692203704804, 55.57982895000019), (37.57327546607398, 55.58116294118248), (37.57385012109279, 55.581550362779), (37.57399562266922, 55.5820107079112), (37.5735356072979, 55.58226289171689), (37.57290393054962, 55.582393529795155), (37.57037722355653, 55.581919415056234), (37.5592298306885, 55.584471614867844), (37.54189249206543, 55.58867650795186), (37.5297256269836, 55.59158133551745), (37.517837865081766, 55.59443656218868), (37.51200186508174, 55.59635625174229), (37.506808949737554, 55.59907823904434), (37.49820432275389, 55.6062944994944), (37.494406071441674, 55.60967103463367), (37.494760001358024, 55.61066689753365), (37.49397137107085, 55.61220931698269), (37.49016528606031, 55.613417718449064), (37.48773249206542, 55.61530616333343), (37.47921386508177, 55.622640129112334), (37.470652153442394, 55.62993723476164), (37.46273446298218, 55.6368075123157), (37.46350692265317, 55.64068225239439), (37.46050283203121, 55.640794546982576), (37.457627470916734, 55.64118904154646), (37.450718034393326, 55.64690488145138), (37.44239252645875, 55.65397824729769), (37.434587576721185, 55.66053543155961), (37.43582144975277, 55.661693766520735), (37.43576786245721, 55.662755031737014), (37.430982915344174, 55.664610641628116), (37.428547447097685, 55.66778515273695), (37.42945134592044, 55.668633314343566), (37.42859571562949, 55.66948145750025), (37.4262836402282, 55.670813882451405), (37.418709037048295, 55.6811141674414), (37.41922139651101, 55.68235377885389), (37.419218771842885, 55.68359335082235), (37.417196501327446, 55.684375235224735), (37.41607020370478, 55.68540557585352), (37.415640857147146, 55.68686637150793), (37.414632153442334, 55.68903015131686), (37.413344899475064, 55.690896881757396), (37.41171432275391, 55.69264232162232), (37.40948282275393, 55.69455101638112), (37.40703674603271, 55.69638690385348), (37.39607169577025, 55.70451821283731), (37.38952706878662, 55.70942491932811), (37.387778313491815, 55.71149057784176), (37.39049275399779, 55.71419814298992), (37.385557272491454, 55.7155489617061), (37.38388335714726, 55.71849856042102), (37.378368238098155, 55.7292763261685), (37.37763597123337, 55.730845879211614), (37.37890062088197, 55.73167906388319), (37.37750451918789, 55.734703664681774), (37.375610832015965, 55.734851959522246), (37.3723813571472, 55.74105626086403), (37.37014935714723, 55.746115620904355), (37.36944173016362, 55.750883999993725), (37.36975304365541, 55.76335905525834), (37.37244070571134, 55.76432079697595), (37.3724259757175, 55.76636979670426), (37.369922155757884, 55.76735417953104), (37.369892695770275, 55.76823419316575), (37.370214730163575, 55.782312184391266), (37.370493611114505, 55.78436801120489), (37.37120164550783, 55.78596427165359), (37.37284851456452, 55.7874378183096), (37.37608325135799, 55.7886695054807), (37.3764587460632, 55.78947647305964), (37.37530000265506, 55.79146512926804), (37.38235915344241, 55.79899647809345), (37.384344043655396, 55.80113596939471), (37.38594269577028, 55.80322699999366), (37.38711208598329, 55.804919036911976), (37.3880239841309, 55.806610999993666), (37.38928977249147, 55.81001864976979), (37.39038389947512, 55.81348641242801), (37.39235781481933, 55.81983538336746), (37.393709457672124, 55.82417822811877), (37.394685720901464, 55.82792275755836), (37.39557615344238, 55.830447148154136), (37.39844478226658, 55.83167107969975), (37.40019761214057, 55.83151823557964), (37.400398790382326, 55.83264967594742), (37.39659544313046, 55.83322180909622), (37.39667059524539, 55.83402792148566), (37.39682089947515, 55.83638877400216), (37.39643489154053, 55.83861656112751), (37.3955338994751, 55.84072348043264), (37.392680272491454, 55.84502158126453), (37.39241188227847, 55.84659117913199), (37.392529730163616, 55.84816071336481), (37.39486835714723, 55.85288092980303), (37.39873052645878, 55.859893456073635), (37.40272161111449, 55.86441833633205), (37.40697072750854, 55.867579567544375), (37.410007082016016, 55.868369880337), (37.4120992989502, 55.86920843741314), (37.412668021163924, 55.87055369615854), (37.41482461111453, 55.87170587948249), (37.41862266137694, 55.873183961039565), (37.42413732540892, 55.874879126654704), (37.4312182698669, 55.875614937236705), (37.43111093783558, 55.8762723478417), (37.43332105622856, 55.87706546369396), (37.43385747619623, 55.87790681284802), (37.441303050262405, 55.88027084462084), (37.44747234260555, 55.87942070143253), (37.44716141796871, 55.88072960917233), (37.44769797085568, 55.88121221323979), (37.45204320500181, 55.882080694420715), (37.45673176190186, 55.882346110794586), (37.463383999999984, 55.88252729504517), (37.46682797486874, 55.88294937719063), (37.470014457672086, 55.88361266759345), (37.47751410450743, 55.88546991372396), (37.47860317658232, 55.88534929207307), (37.48165826025772, 55.882563306475106), (37.48316434442331, 55.8815803226785), (37.483831555817645, 55.882427612793315), (37.483182967125686, 55.88372791409729), (37.483092277908824, 55.88495581062434), (37.4855716508179, 55.8875561994203), (37.486440636245746, 55.887827444039566), (37.49014203439328, 55.88897899871799), (37.493210285705544, 55.890208937135604), (37.497512451065035, 55.891342397444696), (37.49780744510645, 55.89174030252967), (37.49940333499519, 55.89239745507079), (37.50018383334346, 55.89339220941865), (37.52421672750851, 55.903869074155224), (37.52977457672118, 55.90564076517974), (37.53503220370484, 55.90661661218259), (37.54042858064267, 55.90714113744566), (37.54320461007303, 55.905645048442985), (37.545686966066306, 55.906608607018505), (37.54743976120755, 55.90788552162358), (37.55796999999999, 55.90901557907218), (37.572711542327866, 55.91059395704873), (37.57942799999998, 55.91073854155573), (37.58502865872187, 55.91009969268444), (37.58739968913264, 55.90794809960554), (37.59131567193598, 55.908713267595054), (37.612687423278814, 55.902866854295375), (37.62348079629517, 55.90041967242986), (37.635797880950896, 55.898141151686396), (37.649487626983664, 55.89639275532968), (37.65619302513125, 55.89572360207488), (37.66294133862307, 55.895295577183965), (37.66874564418033, 55.89505457604897), (37.67375601586915, 55.89254677027454), (37.67744661901856, 55.8947775867987), (37.688347, 55.89450045676125), (37.69480554232789, 55.89422926332761), (37.70107096560668, 55.89322256101114), (37.705962965606716, 55.891763491662616), (37.711885134918205, 55.889110234998974), (37.71682005026245, 55.886577568759876), (37.7199315476074, 55.88458159806678), (37.72234560316464, 55.882281005794134), (37.72364385977171, 55.8809452036196), (37.725371142837474, 55.8809722706006), (37.727870902099546, 55.88037213862385), (37.73394330422971, 55.877941504088696), (37.745339592590376, 55.87208120378722), (37.75525267724611, 55.86703807949492), (37.76919976190188, 55.859821640197474), (37.827835219574, 55.82962968399116), (37.83341438888553, 55.82575289922351), (37.83652584655761, 55.82188784027888), (37.83809213491821, 55.81612575504693), (37.83605359521481, 55.81460347077685), (37.83632178569025, 55.81276696067908), (37.838623105812026, 55.811486181656385), (37.83912198147584, 55.807329380532785), (37.839079078033414, 55.80510270463816), (37.83965844708251, 55.79940712529036), (37.840581150787344, 55.79131399999368), (37.84172564285271, 55.78000432402266)]); +``` + +Check how many cell towers are in Moscow: + +``` +SELECT count() FROM cell_towers WHERE pointInPolygon((lon, lat), (SELECT * FROM moscow)) + +┌─count()─┐ +│ 310463 │ +└─────────┘ + +1 rows in set. Elapsed: 0.067 sec. Processed 43.28 million rows, 692.42 MB (645.83 million rows/s., 10.33 GB/s.) +``` + +The data is also available for interactive queries in the [Playground](https://gh-api.clickhouse.tech/play?user=play), [example](https://gh-api.clickhouse.tech/play?user=play#U0VMRUNUIG1jYywgY291bnQoKSBGUk9NIGNlbGxfdG93ZXJzIEdST1VQIEJZIG1jYyBPUkRFUiBCWSBjb3VudCgpIERFU0M=). + +Although you cannot create temporary tables there. + +[Original article](https://clickhouse.tech/docs/en/getting_started/example_datasets/cell-towers/) diff --git a/docs/en/getting-started/example-datasets/index.md b/docs/en/getting-started/example-datasets/index.md index 72f44d8caf1..53007c33306 100644 --- a/docs/en/getting-started/example-datasets/index.md +++ b/docs/en/getting-started/example-datasets/index.md @@ -20,5 +20,6 @@ The list of documented datasets: - [Terabyte of Click Logs from Criteo](../../getting-started/example-datasets/criteo.md) - [AMPLab Big Data Benchmark](../../getting-started/example-datasets/amplab-benchmark.md) - [Brown University Benchmark](../../getting-started/example-datasets/brown-benchmark.md) +- [Cell Towers](../../getting-started/example-datasets/cell-towers.md) [Original article](https://clickhouse.tech/docs/en/getting_started/example_datasets) From 5e3c9a5debedf927a789e75b8daed4d657d40e3d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Feb 2021 17:44:12 +0300 Subject: [PATCH 435/510] Rename --- docs/en/getting-started/example-datasets/cell-towers.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/cell-towers.md b/docs/en/getting-started/example-datasets/cell-towers.md index f032be76da0..76effdd4c62 100644 --- a/docs/en/getting-started/example-datasets/cell-towers.md +++ b/docs/en/getting-started/example-datasets/cell-towers.md @@ -1,9 +1,9 @@ --- toc_priority: 21 -toc_title: Cell Stations +toc_title: Cell Towers --- -# Cell Stations +# Cell Towers This dataset is from [OpenCellid](https://www.opencellid.org/) - The world's largest Open Database of Cell Towers. From 315824978de64164442ce8c5c6b5cb161041a72b Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Sat, 27 Feb 2021 19:19:21 +0300 Subject: [PATCH 436/510] CheckConstraintsBlockOutputStream optimize nullable column case --- src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 90d6560b9a8..3b0b4db72f9 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -65,10 +65,8 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) /// Check if constraint value is nullable const auto & null_map = column_nullable->getNullMapColumn(); - const auto & data = null_map.getData(); - const auto * it = std::find(data.begin(), data.end(), true); - - bool null_map_contains_null = it != data.end(); + const PaddedPODArray & data = null_map.getData(); + bool null_map_contains_null = !memoryIsZero(data.raw_data(), data.size() * sizeof(UInt8)); if (null_map_contains_null) throw Exception( From bb5a8241b94739a3828bf98e041a6a489d32e7f9 Mon Sep 17 00:00:00 2001 From: Nikolay <211292+kolya7k@users.noreply.github.com> Date: Sat, 27 Feb 2021 21:05:04 +0300 Subject: [PATCH 437/510] Add information about my ClickHouse PHP extension I made a native PHP extension for ClickHouse using the clickhouse-cpp library. My goal is to make a fast extension for ClickHouse with interface similar to mysqli. --- docs/ru/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/ru/interfaces/third-party/client-libraries.md b/docs/ru/interfaces/third-party/client-libraries.md index 26e05b02509..97fa382fdd9 100644 --- a/docs/ru/interfaces/third-party/client-libraries.md +++ b/docs/ru/interfaces/third-party/client-libraries.md @@ -22,6 +22,7 @@ toc_title: "\u041a\u043b\u0438\u0435\u043d\u0442\u0441\u043a\u0438\u0435\u0020\u - [seva-code/php-click-house-client](https://packagist.org/packages/seva-code/php-click-house-client) - [SeasClick C++ client](https://github.com/SeasX/SeasClick) - [glushkovds/phpclickhouse-laravel](https://packagist.org/packages/glushkovds/phpclickhouse-laravel) + - [kolya7k ClickHouse PHP extension](https://github.com//kolya7k/clickhouse-php) - Go - [clickhouse](https://github.com/kshvakov/clickhouse/) - [go-clickhouse](https://github.com/roistat/go-clickhouse) From fd1cf49e926e2c56dacb794f70a04c1901fb8e33 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Feb 2021 22:22:38 +0300 Subject: [PATCH 438/510] Rewrite extractTextFromHTML function --- docker/test/fasttest/run.sh | 1 - src/Functions/extractTextFromHTML.cpp | 306 +++++++++ src/Functions/htmlOrXmlCoarseParse.cpp | 582 ------------------ src/Functions/registerFunctionsString.cpp | 13 +- .../01674_htm_xml_coarse_parse.sql | 13 +- .../01746_extract_text_from_html.reference | 106 ++++ .../01746_extract_text_from_html.sql | 61 ++ 7 files changed, 485 insertions(+), 597 deletions(-) create mode 100644 src/Functions/extractTextFromHTML.cpp delete mode 100644 src/Functions/htmlOrXmlCoarseParse.cpp create mode 100644 tests/queries/0_stateless/01746_extract_text_from_html.reference create mode 100644 tests/queries/0_stateless/01746_extract_text_from_html.sql diff --git a/docker/test/fasttest/run.sh b/docker/test/fasttest/run.sh index 370311b13c5..1bfc91ecd92 100755 --- a/docker/test/fasttest/run.sh +++ b/docker/test/fasttest/run.sh @@ -356,7 +356,6 @@ function run_tests # JSON functions 01666_blns - 01674_htm_xml_coarse_parse ) (time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 ||:) | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt" diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp new file mode 100644 index 00000000000..5bee4dc541f --- /dev/null +++ b/src/Functions/extractTextFromHTML.cpp @@ -0,0 +1,306 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_COLUMN; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +namespace +{ + +ALWAYS_INLINE bool startsWith(const char * s, const char * end, const char * prefix) +{ + return s + strlen(prefix) < end && 0 == memcmp(s, prefix, strlen(prefix)); +} + +ALWAYS_INLINE bool checkAndSkip(const char * __restrict & s, const char * end, const char * prefix) +{ + if (startsWith(s, end, prefix)) + { + s += strlen(prefix); + return true; + } + return false; +} + +bool processComment(const char * __restrict & src, const char * end) +{ + if (!checkAndSkip(src, end, "world'); +Helloworld +SELECT extractTextFromHTML('Helloworld'); +Helloworld +SELECT extractTextFromHTML('Helloworld'); +Helloworld +SELECT extractTextFromHTML('Hello World'); +Hello World +SELECT extractTextFromHTML('Hello World'); +Hello World +SELECT extractTextFromHTML('HelloWorld'); +HelloWorld +SELECT extractTextFromHTML('Hello World'); +Hello World +SELECT extractTextFromHTML('Hello World'); +Hello World +SELECT extractTextFromHTML('Hello World'); +Hello World +SELECT extractTextFromHTML('HelloWorld'); +HelloWorld +SELECT extractTextFromHTML('Hello World'); +Hello World +SELECT extractTextFromHTML(''); + \t Hello,\rworld \n +SELECT extractTextFromHTML('Hello world!'); +Hello Hello\tworld world! +SELECT extractTextFromHTML('Helloworld!'); +HelloHello\tworldworld! +SELECT extractTextFromHTML('Hello world]]> world!'); +Hello Hello world world! +SELECT extractTextFromHTML('John Smith]]>'); +John Smith +SELECT extractTextFromHTML('John ]]>'); +John +SELECT extractTextFromHTML('John Smith]]>'); +John +SELECT extractTextFromHTML('John ]]>]]>'); +John Smith +SELECT extractTextFromHTML('John ]]> ]]>'); +John Smith +SELECT extractTextFromHTML('John]]> ]]>'); +John Smith +SELECT extractTextFromHTML('John ]]>]]>]]>'); +John ]]>Smith +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML('HelloWorld goodbye'); +HelloWorld goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML('HelloWorld goodbye'); +HelloWorld goodbye +SELECT extractTextFromHTML('HelloWorld goodbye'); +HelloWorld goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML('Hello]]> goodbye'); +Hello]]> goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye diff --git a/tests/queries/0_stateless/01746_extract_text_from_html.sql b/tests/queries/0_stateless/01746_extract_text_from_html.sql new file mode 100644 index 00000000000..9bdd153228f --- /dev/null +++ b/tests/queries/0_stateless/01746_extract_text_from_html.sql @@ -0,0 +1,61 @@ +-- { echo } + +SELECT extractTextFromHTML(''); +SELECT extractTextFromHTML(' '); +SELECT extractTextFromHTML(' '); +SELECT extractTextFromHTML('Hello'); +SELECT extractTextFromHTML('Hello, world'); +SELECT extractTextFromHTML('Hello, world'); +SELECT extractTextFromHTML(' Hello, world'); +SELECT extractTextFromHTML(' Hello, world '); +SELECT extractTextFromHTML(' \t Hello,\rworld \n '); + +SELECT extractTextFromHTML('Hello world'); +SELECT extractTextFromHTML('Hello'); +SELECT extractTextFromHTML('Hello<>world'); +SELECT extractTextFromHTML('Helloworld'); +SELECT extractTextFromHTML('Helloworld'); +SELECT extractTextFromHTML('Helloworld'); +SELECT extractTextFromHTML('Helloworld'); +SELECT extractTextFromHTML('Helloworld'); + +SELECT extractTextFromHTML('Hello World'); +SELECT extractTextFromHTML('Hello World'); +SELECT extractTextFromHTML('HelloWorld'); +SELECT extractTextFromHTML('Hello World'); +SELECT extractTextFromHTML('Hello World'); +SELECT extractTextFromHTML('Hello World'); +SELECT extractTextFromHTML('HelloWorld'); +SELECT extractTextFromHTML('Hello World'); + +SELECT extractTextFromHTML(''); +SELECT extractTextFromHTML('Hello world!'); +SELECT extractTextFromHTML('Helloworld!'); + +SELECT extractTextFromHTML('Hello world]]> world!'); +SELECT extractTextFromHTML('John Smith]]>'); +SELECT extractTextFromHTML('John ]]>'); +SELECT extractTextFromHTML('John Smith]]>'); +SELECT extractTextFromHTML('John ]]>]]>'); +SELECT extractTextFromHTML('John ]]> ]]>'); +SELECT extractTextFromHTML('John]]> ]]>'); +SELECT extractTextFromHTML('John ]]>]]>]]>'); + +SELECT extractTextFromHTML('Hello goodbye'); +SELECT extractTextFromHTML('Hello goodbye'); +SELECT extractTextFromHTML('HelloWorld goodbye'); +SELECT extractTextFromHTML('Hello goodbye'); +SELECT extractTextFromHTML('Hello goodbye'); +SELECT extractTextFromHTML('HelloWorld goodbye'); +SELECT extractTextFromHTML('HelloWorld goodbye'); + +SELECT extractTextFromHTML('Hello goodbye'); +SELECT extractTextFromHTML('Hello goodbye'); +SELECT extractTextFromHTML('Hello]]> goodbye'); +SELECT extractTextFromHTML('Hello goodbye'); From 2ac673b12a9f02a36136263abd873159e28e4de8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 27 Feb 2021 22:33:55 +0300 Subject: [PATCH 439/510] Update logic and tests --- src/Functions/extractTextFromHTML.cpp | 22 +++++-- .../01674_htm_xml_coarse_parse.reference | 2 +- .../01746_extract_text_from_html.reference | 57 +------------------ .../01746_extract_text_from_html.sql | 1 + 4 files changed, 22 insertions(+), 60 deletions(-) diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index 5bee4dc541f..c6a9b84b33e 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -61,7 +61,7 @@ bool processCDATA(const char * __restrict & src, const char * end, char * __rest if (!checkAndSkip(src, end, "Hello, world world goodbye'); Hello goodbye -SELECT extractTextFromHTML('HelloWorld goodbye'); HelloWorld goodbye -SELECT extractTextFromHTML('HelloWorld goodbye'); HelloWorld goodbye -SELECT extractTextFromHTML('Hello goodbye'); Hello goodbye -SELECT extractTextFromHTML('Hello goodbye'); Hello goodbye -SELECT extractTextFromHTML('Hello]]> goodbye'); -Hello]]> goodbye -SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +Hello Hello goodbye diff --git a/tests/queries/0_stateless/01746_extract_text_from_html.sql b/tests/queries/0_stateless/01746_extract_text_from_html.sql index 9bdd153228f..0004849df87 100644 --- a/tests/queries/0_stateless/01746_extract_text_from_html.sql +++ b/tests/queries/0_stateless/01746_extract_text_from_html.sql @@ -57,5 +57,6 @@ SELECT extractTextFromHTML('Hello]]> goodbye'); +SELECT extractTextFromHTML('Hello]]> goodbye'); SELECT extractTextFromHTML('Hello goodbye'); From 0ab4afeeed567b4626b45bd7c7b984c085a6916b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Feb 2021 00:55:56 +0300 Subject: [PATCH 440/510] Tests and documentation --- .../01674_htm_xml_coarse_parse.reference | 2 +- .../01746_extract_text_from_html.reference | 107 ++++++++++++++---- .../01746_extract_text_from_html.sql | 10 ++ 3 files changed, 98 insertions(+), 21 deletions(-) diff --git a/tests/queries/0_stateless/01674_htm_xml_coarse_parse.reference b/tests/queries/0_stateless/01674_htm_xml_coarse_parse.reference index 72af13aedd0..9cca4934551 100644 --- a/tests/queries/0_stateless/01674_htm_xml_coarse_parse.reference +++ b/tests/queries/0_stateless/01674_htm_xml_coarse_parse.reference @@ -2,7 +2,7 @@ Here is CDTATA. This is a white space test. -This is a complex test. world goodbye'); Hello goodbye -HelloWorld goodbye -HelloWorld goodbye +SELECT extractTextFromHTML('HelloWorld goodbye'); +Hello World goodbye +SELECT extractTextFromHTML('HelloWorld goodbye'); +Hello World goodbye +SELECT extractTextFromHTML('Hello goodbye'); Hello goodbye +SELECT extractTextFromHTML('Hello goodbye'); Hello goodbye +SELECT extractTextFromHTML('Hello]]> goodbye'); Hello +SELECT extractTextFromHTML('Hello]]> goodbye'); Hello goodbye -Hello +SELECT extractTextFromHTML('Hello]]> goodbye'); +Hello ]]> goodbye +SELECT extractTextFromHTML('Hello goodbye'); Hello goodbye +SELECT extractTextFromHTML('Hello goodbye'); +Hello goodbye +SELECT extractTextFromHTML(']]>'); +]]> +SELECT extractTextFromHTML(' + +
xkcd.com + +'); +xkcd.com diff --git a/tests/queries/0_stateless/01746_extract_text_from_html.sql b/tests/queries/0_stateless/01746_extract_text_from_html.sql index 0004849df87..b4ccc775bef 100644 --- a/tests/queries/0_stateless/01746_extract_text_from_html.sql +++ b/tests/queries/0_stateless/01746_extract_text_from_html.sql @@ -58,5 +58,15 @@ SELECT extractTextFromHTML('Hello]]> goodbye'); SELECT extractTextFromHTML('Hello]]> goodbye'); +SELECT extractTextFromHTML('Hello]]> goodbye'); +SELECT extractTextFromHTML('Hello goodbye'); SELECT extractTextFromHTML('Hello goodbye'); + +SELECT extractTextFromHTML(']]>'); + +SELECT extractTextFromHTML(' + +
xkcd.com + +'); From 4ab18cdcd8a5eff3f4e386a86361a60f61222e23 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 28 Feb 2021 00:59:27 +0300 Subject: [PATCH 441/510] Tests and documentation --- src/Functions/extractTextFromHTML.cpp | 92 +++++++++++++++++++-------- 1 file changed, 65 insertions(+), 27 deletions(-) diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp index c6a9b84b33e..4b35eacaef0 100644 --- a/src/Functions/extractTextFromHTML.cpp +++ b/src/Functions/extractTextFromHTML.cpp @@ -6,6 +6,58 @@ #include +/** A function to extract text from HTML or XHTML. + * It does not necessarily 100% conforms to any of the HTML, XML or XHTML standards, + * but the implementation is reasonably accurate and it is fast. + * + * The rules are the following: + * + * 1. Comments are skipped. Example: + * Comment must end with -->. Nested comments are not possible. + * Note: constructions like are not valid comments in HTML but will be skipped by other rules. + * + * 2. CDATA is pasted verbatim. + * Note: CDATA is XML/XHTML specific. But we still process it for "best-effort" approach. + * + * 3. 'script' and 'style' elements are removed with all their content. + * Note: it's assumed that closing tag cannot appear inside content. + * For example, in JS string literal is has to be escaped as "<\/script>". + * Note: comments and CDATA is possible inside script or style - then closing tags are not searched inside CDATA. + * Example: ]]> + * But still searched inside comments. Sometimes it becomes complicated: + * var y = "-->"; alert(x + y); + * Note: script and style can be the names of XML namespaces - then they are not treat like usual script or style. + * Example: Hello. + * Note: whitespaces are possible after closing tag name: but not before: < / script>. + * + * 4. Other tags or tag-like elements are skipped without inner content. + * Example: . + * Note: it's expected that this HTML is illegal: + * Note: it will also skip something like tags: <>, , etc. + * Note: tag without end will be skipped to the end of input: + * 5. HTML and XML entities are not decoded. + * It should be processed by separate function. + * + * 6. Whitespaces in text are collapsed or inserted by specific rules. + * Whitespaces at beginning and at the end are removed. + * Consecutive whitespaces are collapsed. + * But if text is separated by other elements and there is no whitespace, it is inserted. + * It may be unnatural, examples: Helloworld, Helloworld + * - in HTML there will be no whitespace, but the function will insert it. + * But also consider: Hello

world

, Hello
world. + * This behaviour is reasonable for data analysis, e.g. convert HTML to a bag of words. + * + * 7. Also note that correct handling of whitespaces would require + * support of
 and CSS display and white-space properties.
+  *
+  * Usage example:
+  *
+  * SELECT extractTextFromHTML(html) FROM url('https://yandex.ru/', RawBLOB, 'html String')
+  *
+  * - ClickHouse has embedded web browser.
+  */
+
 namespace DB
 {
 
@@ -56,18 +108,11 @@ bool processComment(const char * __restrict & src, const char * end)
     return true;
 }
 
-bool processCDATA(const char * __restrict & src, const char * end, char * __restrict & dst, bool & pending_whitespace)
+bool processCDATA(const char * __restrict & src, const char * end, char * __restrict & dst)
 {
     if (!checkAndSkip(src, end, "(src, end);
+
+    if (needs_whitespace && src < lt)
     {
-        pending_whitespace = false;
         *dst = ' ';
         ++dst;
     }
 
-    const char * lt = find_first_symbols<'<'>(src, end);
-
     while (true)
     {
         const char * ws = find_first_symbols<' ', '\t', '\n', '\r', '\f', '\v'>(src, lt);
@@ -204,10 +245,7 @@ void copyText(const char * __restrict & src, const char * end, char * __restrict
 
         src = ws;
         while (src < lt && isWhitespaceASCII(*src))
-        {
-            pending_whitespace = true;
             ++src;
-        }
 
         if (src < lt)
         {
@@ -232,16 +270,16 @@ size_t extract(const char * __restrict src, size_t size, char * __restrict dst)
       * - CDATA should be copied verbatim;
       */
 
-    char * dst_begin = dst;
     const char * end = src + size;
-    bool pending_whitespace = false;
+    char * dst_begin = dst;
 
     while (src < end)
     {
-        copyText(src, end, dst, pending_whitespace);
+        bool needs_whitespace = dst != dst_begin && dst[-1] != ' ';
+        copyText(src, end, dst, needs_whitespace);
 
         processComment(src, end)
-            || processCDATA(src, end, dst, pending_whitespace)
+            || processCDATA(src, end, dst)
             || processElementAndSkipContent(src, end, "script")
             || processElementAndSkipContent(src, end, "style")
             || skipTag(src, end);

From 01ef06a42c4c213a6ec0d0c9188ab10e5a2f07b9 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Sun, 28 Feb 2021 02:31:47 +0300
Subject: [PATCH 442/510] Fix broken links

---
 docs/en/sql-reference/functions/hash-functions.md | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index 6bf1bebabaa..465ad01527f 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -9,7 +9,7 @@ Hash functions can be used for the deterministic pseudo-random shuffling of elem
 
 ## halfMD5 {#hash-functions-halfmd5}
 
-[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
+[Interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
 
 ``` sql
 halfMD5(par1, ...)
@@ -54,7 +54,7 @@ sipHash64(par1,...)
 
 This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) function.
 
-Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
+Function [interprets](../../sql-reference/functions/type-conversion-functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
 
 1.  After hashing all the input parameters, the function gets the array of hashes.
 2.  Function takes the first and the second elements and calculates a hash for the array of them.

From e5ae9cbb6365dcf2122672e6587a95f19ebbd187 Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Sun, 28 Feb 2021 04:03:22 +0300
Subject: [PATCH 443/510] Fix Arcadia

---
 src/Functions/ya.make | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/Functions/ya.make b/src/Functions/ya.make
index 20ba5f846a3..f8beaa8540c 100644
--- a/src/Functions/ya.make
+++ b/src/Functions/ya.make
@@ -246,6 +246,7 @@ SRCS(
     extractAllGroupsHorizontal.cpp
     extractAllGroupsVertical.cpp
     extractGroups.cpp
+    extractTextFromHTML.cpp
     extractTimeZoneFromFunctionArguments.cpp
     filesystem.cpp
     finalizeAggregation.cpp
@@ -291,7 +292,6 @@ SRCS(
     hasToken.cpp
     hasTokenCaseInsensitive.cpp
     hostName.cpp
-    htmlOrXmlCoarseParse.cpp
     hypot.cpp
     identity.cpp
     if.cpp

From ae9fea1d0af118a8f87b224d194d61da1567188b Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Sun, 28 Feb 2021 04:05:04 +0300
Subject: [PATCH 444/510] Fix gcc and clang-tidy

---
 src/Functions/extractTextFromHTML.cpp | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/src/Functions/extractTextFromHTML.cpp b/src/Functions/extractTextFromHTML.cpp
index 4b35eacaef0..528bd0c311f 100644
--- a/src/Functions/extractTextFromHTML.cpp
+++ b/src/Functions/extractTextFromHTML.cpp
@@ -70,12 +70,12 @@ namespace ErrorCodes
 namespace
 {
 
-ALWAYS_INLINE bool startsWith(const char * s, const char * end, const char * prefix)
+inline bool startsWith(const char * s, const char * end, const char * prefix)
 {
     return s + strlen(prefix) < end && 0 == memcmp(s, prefix, strlen(prefix));
 }
 
-ALWAYS_INLINE bool checkAndSkip(const char * __restrict & s, const char * end, const char * prefix)
+inline bool checkAndSkip(const char * __restrict & s, const char * end, const char * prefix)
 {
     if (startsWith(s, end, prefix))
     {
@@ -140,7 +140,7 @@ bool processCDATA(const char * __restrict & src, const char * end, char * __rest
 
 bool processElementAndSkipContent(const char * __restrict & src, const char * end, const char * tag_name)
 {
-    auto old_src = src;
+    const auto * old_src = src;
 
     if (!(src < end && *src == '<'))
         return false;

From 220a494fa479069c22b048c2aee2f4c3d4186d3a Mon Sep 17 00:00:00 2001
From: Alexey Milovidov 
Date: Sun, 28 Feb 2021 04:42:35 +0300
Subject: [PATCH 445/510] Fix idiotic syntax highlight in docs #18432

---
 website/css/highlight.css | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/website/css/highlight.css b/website/css/highlight.css
index 55a0054b07f..7cc8a4865dd 100644
--- a/website/css/highlight.css
+++ b/website/css/highlight.css
@@ -11,7 +11,7 @@
 .syntax .hll { background-color: #b9b6b0 }
 .syntax  { background: #f8f9fa; color: #2f1e2e }
 .syntax .c { color: #8d8687 } /* Comment */
-.syntax .err { color: #ef6155 } /* Error */
+.syntax .err {} /* Error */
 .syntax .k { color: #000000; font-weight: bold } /* Keyword */
 .syntax .l { color: #0088ff } /* Literal */
 .syntax .n { color: #2f1e2e } /* Name */

From 2d9b524bddb335a6376547443e6df69b5ed1a6fb Mon Sep 17 00:00:00 2001
From: Anmol Arora 
Date: Sun, 21 Feb 2021 14:04:55 +0000
Subject: [PATCH 446/510] Fill only requested columns when querying
 system.parts & system.parts_columns

---
 src/Storages/System/StorageSystemParts.cpp    | 178 ++++++++++++------
 src/Storages/System/StorageSystemParts.h      |   3 +-
 .../System/StorageSystemPartsBase.cpp         |  19 +-
 src/Storages/System/StorageSystemPartsBase.h  |   4 +-
 .../System/StorageSystemPartsColumns.cpp      | 121 ++++++++----
 .../System/StorageSystemPartsColumns.h        |   3 +-
 6 files changed, 221 insertions(+), 107 deletions(-)

diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp
index 45e1663cb93..eece092206d 100644
--- a/src/Storages/System/StorageSystemParts.cpp
+++ b/src/Storages/System/StorageSystemParts.cpp
@@ -81,7 +81,8 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_)
 {
 }
 
-void StorageSystemParts::processNextStorage(MutableColumns & columns_, const StoragesInfo & info, bool has_state_column)
+void StorageSystemParts::processNextStorage(
+    MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column)
 {
     using State = IMergeTreeDataPart::State;
     MergeTreeData::DataPartStateVector all_parts_state;
@@ -96,97 +97,154 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto
 
         ColumnSize columns_size = part->getTotalColumnsSize();
 
-        size_t i = 0;
+        size_t src_index = 0, res_index = 0;
+        if (columns_mask[src_index++])
         {
             WriteBufferFromOwnString out;
             part->partition.serializeText(*info.data, out, format_settings);
-            columns_[i++]->insert(out.str());
+            columns[res_index++]->insert(out.str());
         }
-        columns_[i++]->insert(part->name);
-        columns_[i++]->insert(part->uuid);
-        columns_[i++]->insert(part->getTypeName());
-        columns_[i++]->insert(part_state == State::Committed);
-        columns_[i++]->insert(part->getMarksCount());
-        columns_[i++]->insert(part->rows_count);
-        columns_[i++]->insert(part->getBytesOnDisk());
-        columns_[i++]->insert(columns_size.data_compressed);
-        columns_[i++]->insert(columns_size.data_uncompressed);
-        columns_[i++]->insert(columns_size.marks);
-        columns_[i++]->insert(static_cast(part->modification_time));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->name);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->uuid);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getTypeName());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part_state == State::Committed);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getMarksCount());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->rows_count);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getBytesOnDisk());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(columns_size.data_compressed);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(columns_size.data_uncompressed);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(columns_size.marks);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part->modification_time));
 
-        time_t remove_time = part->remove_time.load(std::memory_order_relaxed);
-        columns_[i++]->insert(static_cast(remove_time == std::numeric_limits::max() ? 0 : remove_time));
+        if (columns_mask[src_index++])
+        {
+            time_t remove_time = part->remove_time.load(std::memory_order_relaxed);
+            columns[res_index++]->insert(static_cast(remove_time == std::numeric_limits::max() ? 0 : remove_time));
+        }
 
         /// For convenience, in returned refcount, don't add references that was due to local variables in this method: all_parts, active_parts.
-        columns_[i++]->insert(static_cast(part.use_count() - 1));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part.use_count() - 1));
 
-        columns_[i++]->insert(part->getMinDate());
-        columns_[i++]->insert(part->getMaxDate());
-        columns_[i++]->insert(static_cast(part->getMinTime()));
-        columns_[i++]->insert(static_cast(part->getMaxTime()));
-        columns_[i++]->insert(part->info.partition_id);
-        columns_[i++]->insert(part->info.min_block);
-        columns_[i++]->insert(part->info.max_block);
-        columns_[i++]->insert(part->info.level);
-        columns_[i++]->insert(static_cast(part->info.getDataVersion()));
-        columns_[i++]->insert(part->getIndexSizeInBytes());
-        columns_[i++]->insert(part->getIndexSizeInAllocatedBytes());
-        columns_[i++]->insert(part->is_frozen.load(std::memory_order_relaxed));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getMinDate());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getMaxDate());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part->getMinTime()));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part->getMaxTime()));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->info.partition_id);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->info.min_block);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->info.max_block);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->info.level);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part->info.getDataVersion()));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getIndexSizeInBytes());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->getIndexSizeInAllocatedBytes());
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(part->is_frozen.load(std::memory_order_relaxed));
+
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(info.database);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(info.table);
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(info.engine);
 
-        columns_[i++]->insert(info.database);
-        columns_[i++]->insert(info.table);
-        columns_[i++]->insert(info.engine);
         if (part->isStoredOnDisk())
         {
-            columns_[i++]->insert(part->volume->getDisk()->getName());
-            columns_[i++]->insert(part->getFullPath());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->volume->getDisk()->getName());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->getFullPath());
         }
         else
         {
-            columns_[i++]->insertDefault();
-            columns_[i++]->insertDefault();
+            if (columns_mask[src_index++])
+                columns[res_index++]->insertDefault();
+            if (columns_mask[src_index++])
+                columns[res_index++]->insertDefault();
         }
 
-        MinimalisticDataPartChecksums helper;
-        helper.computeTotalChecksums(part->checksums);
 
-        auto checksum = helper.hash_of_all_files;
-        columns_[i++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second));
+        {
+            MinimalisticDataPartChecksums helper;
+            if (columns_mask[src_index] || columns_mask[src_index + 1] || columns_mask[src_index + 2])
+                helper.computeTotalChecksums(part->checksums);
 
-        checksum = helper.hash_of_uncompressed_files;
-        columns_[i++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second));
-
-        checksum = helper.uncompressed_hash_of_compressed_files;
-        columns_[i++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second));
+            if (columns_mask[src_index++])
+            {
+                auto checksum = helper.hash_of_all_files;
+                columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second));
+            }
+            if (columns_mask[src_index++])
+            {
+                auto checksum = helper.hash_of_uncompressed_files;
+                columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second));
+            }
+            if (columns_mask[src_index++])
+            {
+                auto checksum = helper.uncompressed_hash_of_compressed_files;
+                columns[res_index++]->insert(getHexUIntLowercase(checksum.first) + getHexUIntLowercase(checksum.second));
+            }
+        }
 
         /// delete_ttl_info
-        {
-            columns_[i++]->insert(static_cast(part->ttl_infos.table_ttl.min));
-            columns_[i++]->insert(static_cast(part->ttl_infos.table_ttl.max));
-        }
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part->ttl_infos.table_ttl.min));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(static_cast(part->ttl_infos.table_ttl.max));
 
         auto add_ttl_info_map = [&](const TTLInfoMap & ttl_info_map)
         {
             Array expression_array;
             Array min_array;
             Array max_array;
-            expression_array.reserve(ttl_info_map.size());
-            min_array.reserve(ttl_info_map.size());
-            max_array.reserve(ttl_info_map.size());
+            if (columns_mask[src_index])
+                expression_array.reserve(ttl_info_map.size());
+            if (columns_mask[src_index + 1])
+                min_array.reserve(ttl_info_map.size());
+            if (columns_mask[src_index + 2])
+                max_array.reserve(ttl_info_map.size());
             for (const auto & [expression, ttl_info] : ttl_info_map)
             {
-                expression_array.emplace_back(expression);
-                min_array.push_back(static_cast(ttl_info.min));
-                max_array.push_back(static_cast(ttl_info.max));
+                if (columns_mask[src_index])
+                    expression_array.emplace_back(expression);
+                if (columns_mask[src_index + 1])
+                    min_array.push_back(static_cast(ttl_info.min));
+                if (columns_mask[src_index + 2])
+                    max_array.push_back(static_cast(ttl_info.max));
             }
-            columns_[i++]->insert(expression_array);
-            columns_[i++]->insert(min_array);
-            columns_[i++]->insert(max_array);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(expression_array);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(min_array);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(max_array);
         };
 
         add_ttl_info_map(part->ttl_infos.moves_ttl);
 
-        columns_[i++]->insert(queryToString(part->default_codec->getCodecDesc()));
+        if (columns_mask[src_index++])
+            columns[res_index++]->insert(queryToString(part->default_codec->getCodecDesc()));
 
         add_ttl_info_map(part->ttl_infos.recompression_ttl);
         add_ttl_info_map(part->ttl_infos.group_by_ttl);
@@ -195,7 +253,7 @@ void StorageSystemParts::processNextStorage(MutableColumns & columns_, const Sto
         /// _state column should be the latest.
         /// Do not use part->getState*, it can be changed from different thread
         if (has_state_column)
-            columns_[i++]->insert(IMergeTreeDataPart::stateToString(part_state));
+            columns[res_index++]->insert(IMergeTreeDataPart::stateToString(part_state));
     }
 }
 
diff --git a/src/Storages/System/StorageSystemParts.h b/src/Storages/System/StorageSystemParts.h
index 2de4cd3f3a4..d67e62049cd 100644
--- a/src/Storages/System/StorageSystemParts.h
+++ b/src/Storages/System/StorageSystemParts.h
@@ -20,7 +20,8 @@ public:
 
 protected:
     explicit StorageSystemParts(const StorageID & table_id_);
-    void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override;
+    void processNextStorage(
+        MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override;
 };
 
 }
diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp
index 9b5bf5a4b48..39cc651e147 100644
--- a/src/Storages/System/StorageSystemPartsBase.cpp
+++ b/src/Storages/System/StorageSystemPartsBase.cpp
@@ -245,16 +245,29 @@ Pipe StorageSystemPartsBase::read(
 
     /// Create the result.
 
-    MutableColumns res_columns = metadata_snapshot->getSampleBlock().cloneEmptyColumns();
+    NameSet names_set(column_names.begin(), column_names.end());
+
+    Block sample = metadata_snapshot->getSampleBlock();
+    Block header;
+
+    std::vector columns_mask(sample.columns());
+    for (size_t i = 0; i < sample.columns(); ++i)
+    {
+        if (names_set.count(sample.getByPosition(i).name))
+        {
+            columns_mask[i] = 1;
+            header.insert(sample.getByPosition(i));
+        }
+    }
+    MutableColumns res_columns = header.cloneEmptyColumns();
     if (has_state_column)
         res_columns.push_back(ColumnString::create());
 
     while (StoragesInfo info = stream.next())
     {
-        processNextStorage(res_columns, info, has_state_column);
+        processNextStorage(res_columns, columns_mask, info, has_state_column);
     }
 
-    Block header = metadata_snapshot->getSampleBlock();
     if (has_state_column)
         header.insert(ColumnWithTypeAndName(std::make_shared(), "_state"));
 
diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h
index eec6d5ab331..4c3ca78b5bb 100644
--- a/src/Storages/System/StorageSystemPartsBase.h
+++ b/src/Storages/System/StorageSystemPartsBase.h
@@ -74,7 +74,9 @@ protected:
 
     StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_);
 
-    virtual void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) = 0;
+    virtual void
+    processNextStorage(MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column)
+        = 0;
 };
 
 }
diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp
index c570cc85e8b..8754e424281 100644
--- a/src/Storages/System/StorageSystemPartsColumns.cpp
+++ b/src/Storages/System/StorageSystemPartsColumns.cpp
@@ -60,7 +60,8 @@ StorageSystemPartsColumns::StorageSystemPartsColumns(const StorageID & table_id_
 {
 }
 
-void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, const StoragesInfo & info, bool has_state_column)
+void StorageSystemPartsColumns::processNextStorage(
+    MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column)
 {
     /// Prepare information about columns in storage.
     struct ColumnInfo
@@ -105,67 +106,105 @@ void StorageSystemPartsColumns::processNextStorage(MutableColumns & columns_, co
         for (const auto & column : part->getColumns())
         {
             ++column_position;
-            size_t j = 0;
+            size_t src_index = 0, res_index = 0;
+            if (columns_mask[src_index++])
             {
                 WriteBufferFromOwnString out;
                 part->partition.serializeText(*info.data, out, format_settings);
-                columns_[j++]->insert(out.str());
+                columns[res_index++]->insert(out.str());
             }
-            columns_[j++]->insert(part->name);
-            columns_[j++]->insert(part->getTypeName());
-            columns_[j++]->insert(part_state == State::Committed);
-            columns_[j++]->insert(part->getMarksCount());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->name);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->getTypeName());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part_state == State::Committed);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->getMarksCount());
 
-            columns_[j++]->insert(part->rows_count);
-            columns_[j++]->insert(part->getBytesOnDisk());
-            columns_[j++]->insert(columns_size.data_compressed);
-            columns_[j++]->insert(columns_size.data_uncompressed);
-            columns_[j++]->insert(columns_size.marks);
-            columns_[j++]->insert(UInt64(part->modification_time));
-            columns_[j++]->insert(UInt64(part->remove_time.load(std::memory_order_relaxed)));
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->rows_count);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->getBytesOnDisk());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(columns_size.data_compressed);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(columns_size.data_uncompressed);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(columns_size.marks);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(UInt64(part->modification_time));
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(UInt64(part->remove_time.load(std::memory_order_relaxed)));
 
-            columns_[j++]->insert(UInt64(use_count));
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(UInt64(use_count));
 
-            columns_[j++]->insert(min_date);
-            columns_[j++]->insert(max_date);
-            columns_[j++]->insert(part->info.partition_id);
-            columns_[j++]->insert(part->info.min_block);
-            columns_[j++]->insert(part->info.max_block);
-            columns_[j++]->insert(part->info.level);
-            columns_[j++]->insert(UInt64(part->info.getDataVersion()));
-            columns_[j++]->insert(index_size_in_bytes);
-            columns_[j++]->insert(index_size_in_allocated_bytes);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(min_date);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(max_date);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->info.partition_id);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->info.min_block);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->info.max_block);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->info.level);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(UInt64(part->info.getDataVersion()));
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(index_size_in_bytes);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(index_size_in_allocated_bytes);
 
-            columns_[j++]->insert(info.database);
-            columns_[j++]->insert(info.table);
-            columns_[j++]->insert(info.engine);
-            columns_[j++]->insert(part->volume->getDisk()->getName());
-            columns_[j++]->insert(part->getFullPath());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(info.database);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(info.table);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(info.engine);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->volume->getDisk()->getName());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(part->getFullPath());
 
-            columns_[j++]->insert(column.name);
-            columns_[j++]->insert(column.type->getName());
-            columns_[j++]->insert(column_position);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column.name);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column.type->getName());
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column_position);
 
             auto column_info_it = columns_info.find(column.name);
             if (column_info_it != columns_info.end())
             {
-                columns_[j++]->insert(column_info_it->second.default_kind);
-                columns_[j++]->insert(column_info_it->second.default_expression);
+                if (columns_mask[src_index++])
+                    columns[res_index++]->insert(column_info_it->second.default_kind);
+                if (columns_mask[src_index++])
+                    columns[res_index++]->insert(column_info_it->second.default_expression);
             }
             else
             {
-                columns_[j++]->insertDefault();
-                columns_[j++]->insertDefault();
+                if (columns_mask[src_index++])
+                    columns[res_index++]->insertDefault();
+                if (columns_mask[src_index++])
+                    columns[res_index++]->insertDefault();
             }
 
             ColumnSize column_size = part->getColumnSize(column.name, *column.type);
-            columns_[j++]->insert(column_size.data_compressed + column_size.marks);
-            columns_[j++]->insert(column_size.data_compressed);
-            columns_[j++]->insert(column_size.data_uncompressed);
-            columns_[j++]->insert(column_size.marks);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column_size.data_compressed + column_size.marks);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column_size.data_compressed);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column_size.data_uncompressed);
+            if (columns_mask[src_index++])
+                columns[res_index++]->insert(column_size.marks);
 
             if (has_state_column)
-                columns_[j++]->insert(part->stateString());
+                columns[res_index++]->insert(part->stateString());
         }
     }
 }
diff --git a/src/Storages/System/StorageSystemPartsColumns.h b/src/Storages/System/StorageSystemPartsColumns.h
index 6347a418875..ec12a608cd1 100644
--- a/src/Storages/System/StorageSystemPartsColumns.h
+++ b/src/Storages/System/StorageSystemPartsColumns.h
@@ -22,7 +22,8 @@ public:
 
 protected:
     StorageSystemPartsColumns(const StorageID & table_id_);
-    void processNextStorage(MutableColumns & columns, const StoragesInfo & info, bool has_state_column) override;
+    void processNextStorage(
+        MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) override;
 };
 
 }

From f7dbcfc12a2ccaa4c57a149fb8ea624d46c511b6 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 18:40:56 +0300
Subject: [PATCH 447/510] DDLTask: Use std::uncaught_exceptions() over
 deprecated std::uncaught_exception()

---
 src/Interpreters/DDLTask.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h
index 18c1f4c80cd..45702599fcf 100644
--- a/src/Interpreters/DDLTask.h
+++ b/src/Interpreters/DDLTask.h
@@ -189,7 +189,7 @@ public:
 
     void commit();
 
-    ~ZooKeeperMetadataTransaction() { assert(isExecuted() || std::uncaught_exception()); }
+    ~ZooKeeperMetadataTransaction() { assert(isExecuted() || std::uncaught_exceptions()); }
 };
 
 }

From 78c6e0527d77683e83f483bc4abc2f698ba7b8a6 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 18:39:10 +0300
Subject: [PATCH 448/510] DDLWorker: Fix reference to zookeeper with
 distributed_ddl.pool_size > 1 (thread pool)

---
 src/Interpreters/DDLWorker.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 67f716c235c..8757b71c61a 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -365,7 +365,7 @@ void DDLWorker::scheduleTasks()
 
         if (worker_pool)
         {
-            worker_pool->scheduleOrThrowOnError([this, &saved_task, &zookeeper]()
+            worker_pool->scheduleOrThrowOnError([this, &saved_task, zookeeper]()
             {
                 setThreadName("DDLWorkerExec");
                 processTask(saved_task, zookeeper);

From b899d8d9195d5f9f7e89e61eab83112f341ea107 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 18:43:32 +0300
Subject: [PATCH 449/510] tests: regression for test for invalid reference to
 zookeeper for distributed_ddl.pool_size > 1

---
 tests/integration/test_distributed_ddl_parallel/test.py | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py
index 96530b111cb..73a16bed471 100644
--- a/tests/integration/test_distributed_ddl_parallel/test.py
+++ b/tests/integration/test_distributed_ddl_parallel/test.py
@@ -87,3 +87,7 @@ def test_two_in_parallel_two_queued():
         for thread in threads:
             thread.join()
     inner_test()
+
+def test_smoke():
+    for _ in range(100):
+        initiator.query('DROP DATABASE IF EXISTS foo ON CLUSTER cluster')

From 06e8bb955d82d1180bf719292b6a6985a03ec0c7 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 19:13:09 +0300
Subject: [PATCH 450/510] tests: catch exceptions in threads in
 test_distributed_ddl_parallel

Otherwise the test will not report an error.
---
 .../test_distributed_ddl_parallel/test.py     | 27 ++++++++++++++++---
 1 file changed, 23 insertions(+), 4 deletions(-)

diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py
index 73a16bed471..4474928815a 100644
--- a/tests/integration/test_distributed_ddl_parallel/test.py
+++ b/tests/integration/test_distributed_ddl_parallel/test.py
@@ -10,6 +10,25 @@ from helpers.cluster import ClickHouseCluster
 
 cluster = ClickHouseCluster(__file__)
 
+# By default the exceptions that was throwed in threads will be ignored
+# (they will not mark the test as failed, only printed to stderr).
+#
+# Wrap thrading.Thread and re-throw exception on join()
+class SafeThread(threading.Thread):
+    def __init__(self, target):
+        super().__init__()
+        self.target = target
+        self.exception = None
+    def run(self):
+        try:
+            self.target()
+        except Exception as e: # pylint: disable=broad-except
+            self.exception = e
+    def join(self, timeout=None):
+        super().join(timeout)
+        if self.exception:
+            raise self.exception
+
 def add_instance(name):
     main_configs=[
         'configs/ddl.xml',
@@ -68,11 +87,11 @@ def test_all_in_parallel():
     def inner_test():
         threads = []
         for _ in range(2):
-            threads.append(threading.Thread(target=thread_reload_dictionary))
+            threads.append(SafeThread(target=thread_reload_dictionary))
         for thread in threads:
             thread.start()
         for thread in threads:
-            thread.join()
+            thread.join(60)
     inner_test()
 
 def test_two_in_parallel_two_queued():
@@ -81,11 +100,11 @@ def test_two_in_parallel_two_queued():
     def inner_test():
         threads = []
         for _ in range(4):
-            threads.append(threading.Thread(target=thread_reload_dictionary))
+            threads.append(SafeThread(target=thread_reload_dictionary))
         for thread in threads:
             thread.start()
         for thread in threads:
-            thread.join()
+            thread.join(60)
     inner_test()
 
 def test_smoke():

From a795db57f1ed990739c340ae4ff124493acea24f Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 19:58:08 +0300
Subject: [PATCH 451/510] tests: decrease distributed_ddl_task_timeout to 60 in
 test_distributed_ddl_parallel

---
 .../test_distributed_ddl_parallel/test.py            | 12 ++++++++----
 1 file changed, 8 insertions(+), 4 deletions(-)

diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py
index 4474928815a..a9f2f069e27 100644
--- a/tests/integration/test_distributed_ddl_parallel/test.py
+++ b/tests/integration/test_distributed_ddl_parallel/test.py
@@ -70,7 +70,9 @@ def longer_then(sec):
 
 # It takes 7 seconds to load slow_dict.
 def thread_reload_dictionary():
-    initiator.query('SYSTEM RELOAD DICTIONARY ON CLUSTER cluster slow_dict')
+    initiator.query('SYSTEM RELOAD DICTIONARY ON CLUSTER cluster slow_dict', settings={
+        'distributed_ddl_task_timeout': 60,
+    })
 
 # NOTE: uses inner function to exclude slow start_cluster() from timeout.
 
@@ -91,7 +93,7 @@ def test_all_in_parallel():
         for thread in threads:
             thread.start()
         for thread in threads:
-            thread.join(60)
+            thread.join(70)
     inner_test()
 
 def test_two_in_parallel_two_queued():
@@ -104,9 +106,11 @@ def test_two_in_parallel_two_queued():
         for thread in threads:
             thread.start()
         for thread in threads:
-            thread.join(60)
+            thread.join(70)
     inner_test()
 
 def test_smoke():
     for _ in range(100):
-        initiator.query('DROP DATABASE IF EXISTS foo ON CLUSTER cluster')
+        initiator.query('DROP DATABASE IF EXISTS foo ON CLUSTER cluster', settings={
+            'distributed_ddl_task_timeout': 60,
+        })

From 7481b05d27214f2d361fd8ac176946ed1aa2175a Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 20:33:42 +0300
Subject: [PATCH 452/510] DDLWorker: avoid comparing pointers for getting min
 element

The current_tasks always updated with emplace_back, so front() is ok.
---
 src/Interpreters/DDLWorker.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 8757b71c61a..7c3db09128d 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -338,7 +338,7 @@ void DDLWorker::scheduleTasks()
     if (!server_startup)
     {
         /// We will recheck status of last executed tasks. It's useful if main thread was just restarted.
-        auto & min_task = *std::min_element(current_tasks.begin(), current_tasks.end());
+        auto & min_task = current_tasks.front();
         String min_entry_name = last_skipped_entry_name ? std::min(min_task->entry_name, *last_skipped_entry_name) : min_task->entry_name;
         begin_node = std::upper_bound(queue_nodes.begin(), queue_nodes.end(), min_entry_name);
         current_tasks.clear();

From 0cd67ed051174302591a9ddbb790f3ec7433b4a8 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Mon, 22 Feb 2021 20:35:52 +0300
Subject: [PATCH 453/510] DDLWorker: remove only completed tasks (significant
 for distributed_ddl.pool_size > 1)

Otherwise it will SIGSEGV due to invalid-read (UAF).
---
 src/Interpreters/DDLWorker.cpp | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 7c3db09128d..09aae87beb2 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -341,7 +341,7 @@ void DDLWorker::scheduleTasks()
         auto & min_task = current_tasks.front();
         String min_entry_name = last_skipped_entry_name ? std::min(min_task->entry_name, *last_skipped_entry_name) : min_task->entry_name;
         begin_node = std::upper_bound(queue_nodes.begin(), queue_nodes.end(), min_entry_name);
-        current_tasks.clear();
+        current_tasks.remove_if([](const DDLTaskPtr & t) { return t->completely_processed.load(); });
     }
 
     assert(current_tasks.empty());

From eeda1fe7562cf0573e2308ee0d0bcef36ef2e9f1 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Tue, 23 Feb 2021 08:22:08 +0300
Subject: [PATCH 454/510] DDLWorker: wait for pending async tasks

---
 src/Interpreters/DDLWorker.cpp | 3 +++
 1 file changed, 3 insertions(+)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 09aae87beb2..347fedadb62 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -1007,6 +1007,9 @@ void DDLWorker::runMainThread()
             if (Coordination::isHardwareError(e.code))
             {
                 initialized = false;
+                /// Wait for pending async tasks
+                if (1 < pool_size)
+                    worker_pool = std::make_unique(pool_size);
                 LOG_INFO(log, "Lost ZooKeeper connection, will try to connect again: {}", getCurrentExceptionMessage(true));
             }
             else

From 54b1496408ce8df75156028f867a3017e2d673b3 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Tue, 23 Feb 2021 08:23:24 +0300
Subject: [PATCH 455/510] DDLWorker: optimize processing of unfinished tasks

Do not look at zookeeper if was_executed is false
---
 src/Interpreters/DDLWorker.cpp | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 347fedadb62..e5bed2c752c 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -314,11 +314,14 @@ void DDLWorker::scheduleTasks()
     {
         /// Main thread of DDLWorker was restarted, probably due to lost connection with ZooKeeper.
         /// We have some unfinished tasks. To avoid duplication of some queries, try to write execution status.
-        bool task_still_exists = zookeeper->exists(task->entry_path);
-        bool status_written = zookeeper->exists(task->getFinishedNodePath());
-        if (task->was_executed && !status_written && task_still_exists)
+        if (task->was_executed)
         {
-            processTask(*task, zookeeper);
+            bool task_still_exists = zookeeper->exists(task->entry_path);
+            bool status_written = zookeeper->exists(task->getFinishedNodePath());
+            if (!status_written && task_still_exists)
+            {
+                processTask(*task, zookeeper);
+            }
         }
     }
 

From 3c7e765b270e619aca88263cd2a032c7b8727dc2 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Tue, 23 Feb 2021 08:26:46 +0300
Subject: [PATCH 456/510] DDLWorker: process unfinished tasks only after
 reinitialization

---
 src/Interpreters/DDLWorker.cpp | 25 +++++++++++++++----------
 src/Interpreters/DDLWorker.h   |  2 +-
 2 files changed, 16 insertions(+), 11 deletions(-)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index e5bed2c752c..45ef0d949f5 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -305,22 +305,25 @@ static void filterAndSortQueueNodes(Strings & all_nodes)
     std::sort(all_nodes.begin(), all_nodes.end());
 }
 
-void DDLWorker::scheduleTasks()
+void DDLWorker::scheduleTasks(bool reinitialized)
 {
     LOG_DEBUG(log, "Scheduling tasks");
     auto zookeeper = tryGetZooKeeper();
 
-    for (auto & task : current_tasks)
+    /// Main thread of DDLWorker was restarted, probably due to lost connection with ZooKeeper.
+    /// We have some unfinished tasks. To avoid duplication of some queries, try to write execution status.
+    if (reinitialized)
     {
-        /// Main thread of DDLWorker was restarted, probably due to lost connection with ZooKeeper.
-        /// We have some unfinished tasks. To avoid duplication of some queries, try to write execution status.
-        if (task->was_executed)
+        for (auto & task : current_tasks)
         {
-            bool task_still_exists = zookeeper->exists(task->entry_path);
-            bool status_written = zookeeper->exists(task->getFinishedNodePath());
-            if (!status_written && task_still_exists)
+            if (task->was_executed)
             {
-                processTask(*task, zookeeper);
+                bool task_still_exists = zookeeper->exists(task->entry_path);
+                bool status_written = zookeeper->exists(task->getFinishedNodePath());
+                if (!status_written && task_still_exists)
+                {
+                    processTask(*task, zookeeper);
+                }
             }
         }
     }
@@ -992,6 +995,8 @@ void DDLWorker::runMainThread()
     {
         try
         {
+            bool reinitialized = !initialized;
+
             /// Reinitialize DDLWorker state (including ZooKeeper connection) if required
             if (!initialized)
             {
@@ -1000,7 +1005,7 @@ void DDLWorker::runMainThread()
             }
 
             cleanup_event->set();
-            scheduleTasks();
+            scheduleTasks(reinitialized);
 
             LOG_DEBUG(log, "Waiting for queue updates");
             queue_updated_event->wait();
diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h
index 8b0a8f038a0..e0d8077265f 100644
--- a/src/Interpreters/DDLWorker.h
+++ b/src/Interpreters/DDLWorker.h
@@ -69,7 +69,7 @@ protected:
     ZooKeeperPtr getAndSetZooKeeper();
 
     /// Iterates through queue tasks in ZooKeeper, runs execution of new tasks
-    void scheduleTasks();
+    void scheduleTasks(bool reinitialized);
 
     DDLTaskBase & saveTask(DDLTaskPtr && task);
 

From dfd1c73b31d4f89fb5507407213c7798b40199e2 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Tue, 23 Feb 2021 23:20:31 +0300
Subject: [PATCH 457/510] Load dictionaries before starting accepting
 connections and DDLWorker

---
 programs/server/Server.cpp | 62 +++++++++++++++++++-------------------
 1 file changed, 31 insertions(+), 31 deletions(-)

diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp
index 8fcdfc4beac..72db8f59494 100644
--- a/programs/server/Server.cpp
+++ b/programs/server/Server.cpp
@@ -1017,17 +1017,6 @@ int Server::main(const std::vector & /*args*/)
         LOG_INFO(log, "Query Profiler and TraceCollector are disabled because they require PHDR cache to be created"
             " (otherwise the function 'dl_iterate_phdr' is not lock free and not async-signal safe).");
 
-    if (has_zookeeper && config().has("distributed_ddl"))
-    {
-        /// DDL worker should be started after all tables were loaded
-        String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
-        int pool_size = config().getInt("distributed_ddl.pool_size", 1);
-        if (pool_size < 1)
-            throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
-        global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, *global_context, &config(),
-                                                                 "distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID));
-    }
-
     std::unique_ptr dns_cache_updater;
     if (config().has("disable_internal_dns_cache") && config().getInt("disable_internal_dns_cache"))
     {
@@ -1309,6 +1298,37 @@ int Server::main(const std::vector & /*args*/)
                 std::thread::hardware_concurrency());
         }
 
+        /// try to load dictionaries immediately, throw on error and die
+        ext::scope_guard dictionaries_xmls, models_xmls;
+        try
+        {
+            if (!config().getBool("dictionaries_lazy_load", true))
+            {
+                global_context->tryCreateEmbeddedDictionaries();
+                global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true);
+            }
+            dictionaries_xmls = global_context->getExternalDictionariesLoader().addConfigRepository(
+                std::make_unique(config(), "dictionaries_config"));
+            models_xmls = global_context->getExternalModelsLoader().addConfigRepository(
+                std::make_unique(config(), "models_config"));
+        }
+        catch (...)
+        {
+            LOG_ERROR(log, "Caught exception while loading dictionaries.");
+            throw;
+        }
+
+        if (has_zookeeper && config().has("distributed_ddl"))
+        {
+            /// DDL worker should be started after all tables were loaded
+            String ddl_zookeeper_path = config().getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
+            int pool_size = config().getInt("distributed_ddl.pool_size", 1);
+            if (pool_size < 1)
+                throw Exception("distributed_ddl.pool_size should be greater then 0", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
+            global_context->setDDLWorker(std::make_unique(pool_size, ddl_zookeeper_path, *global_context, &config(),
+                                                                     "distributed_ddl", "DDLWorker", &CurrentMetrics::MaxDDLEntryID));
+        }
+
         LOG_INFO(log, "Ready for connections.");
 
         SCOPE_EXIT({
@@ -1358,26 +1378,6 @@ int Server::main(const std::vector & /*args*/)
             }
         });
 
-        /// try to load dictionaries immediately, throw on error and die
-        ext::scope_guard dictionaries_xmls, models_xmls;
-        try
-        {
-            if (!config().getBool("dictionaries_lazy_load", true))
-            {
-                global_context->tryCreateEmbeddedDictionaries();
-                global_context->getExternalDictionariesLoader().enableAlwaysLoadEverything(true);
-            }
-            dictionaries_xmls = global_context->getExternalDictionariesLoader().addConfigRepository(
-                std::make_unique(config(), "dictionaries_config"));
-            models_xmls = global_context->getExternalModelsLoader().addConfigRepository(
-                std::make_unique(config(), "models_config"));
-        }
-        catch (...)
-        {
-            LOG_ERROR(log, "Caught exception while loading dictionaries.");
-            throw;
-        }
-
         std::vector> metrics_transmitters;
         for (const auto & graphite_key : DB::getMultipleKeysFromConfig(config(), "", "graphite"))
         {

From 955974a8b5ba5461894d6ac7692c6593cb62cb9a Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Wed, 24 Feb 2021 08:07:31 +0300
Subject: [PATCH 458/510] DDLWorker: avoid NULL dereference on termination and
 failed zookeeper initialization

Log snipped shows the problem:

    2021.02.24 04:40:29.349181 [ 39 ] {}  DDLWorker: DDLWorker is configured to use multiple threads. It's not recommended because queries can be reordered. Also it may cause some unknown issues to appear.
    2021.02.24 04:40:29.349516 [ 39 ] {}  Application: Ready for connections.
    2021.02.24 04:40:29.349602 [ 74 ] {}  DDLWorker: Started DDLWorker cleanup thread
    2021.02.24 04:40:29.349639 [ 73 ] {}  DDLWorker: Starting DDLWorker thread
    2021.02.24 04:40:29.349698 [ 73 ] {}  DDLWorker: Started DDLWorker thread
    2021.02.24 04:40:29.352548 [ 73 ] {}  virtual void DB::DDLWorker::initializeMainThread(): Code: 999, e.displayText() = Coordination::Exception: All connection tries failed while connecting to ZooKeeper. nodes: 192.168.112.3:2181
    Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused (version 21.3.1.1), 192.168.112.3:2181
    Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused (version 21.3.1.1), 192.168.112.3:2181
    Poco::Exception. Code: 1000, e.code() = 111, e.displayText() = Connection refused (version 21.3.1.1), 192.168.112.3:2181
     (Connection loss), Stack trace (when copying this message, always include the lines below):

    0. Coordination::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, Coordination::Error, int) @ 0xfe93923 in /usr/bin/clickhouse
    1. Coordination::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, Coordination::Error) @ 0xfe93ba2 in /usr/bin/clickhouse
    2. Coordination::ZooKeeper::connect(std::__1::vector > const&, Poco::Timespan) @ 0xfed3a01 in /usr/bin/clickhouse
    3. Coordination::ZooKeeper::ZooKeeper(std::__1::vector > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, Poco::Timespan, Poco::Timespan, Poco::Timespan) @ 0xfed2222 in /usr/bin/clickhouse
    4. zkutil::ZooKeeper::init(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, int, int, std::__1::basic_string, std::__1::allocator > const&) @ 0xfe961cd in /usr/bin/clickhouse
    5. zkutil::ZooKeeper::ZooKeeper(Poco::Util::AbstractConfiguration const&, std::__1::basic_string, std::__1::allocator > const&) @ 0xfe97a96 in /usr/bin/clickhouse
    6. void std::__1::allocator_traits >::__construct(std::__1::integral_constant, std::__1::allocator&, zkutil::ZooKeeper*, Poco::Util::AbstractConfiguration const&, char const (&) [10]) @ 0xed98387 in /usr/bin/clickhouse
    7. DB::Context::getZooKeeper() const @ 0xed75190 in /usr/bin/clickhouse
    8. DB::DDLWorker::getAndSetZooKeeper() @ 0xedb81c9 in /usr/bin/clickhouse
    9. DB::DDLWorker::initializeMainThread() @ 0xedc9eb0 in /usr/bin/clickhouse
    10. DB::DDLWorker::runMainThread() @ 0xedb5d01 in /usr/bin/clickhouse
    11. ThreadFromGlobalPool::ThreadFromGlobalPool(void (DB::DDLWorker::*&&)(), DB::DDLWorker*&&)::'lambda'()::operator()() @ 0xedcafa1 in /usr/bin/clickhouse
    12. ThreadPoolImpl::worker(std::__1::__list_iterator) @ 0x892651f in /usr/bin/clickhouse
    13. ? @ 0x8929fb3 in /usr/bin/clickhouse
    14. start_thread @ 0x8ea7 in /lib/x86_64-linux-gnu/libpthread-2.31.so
    15. __clone @ 0xfddef in /lib/x86_64-linux-gnu/libc-2.31.so
     (version 21.3.1.1)
    ...
    2021.02.24 04:40:30.025278 [ 41 ] {}  BaseDaemon: Received signal 15
    2021.02.24 04:40:30.025336 [ 41 ] {}  Application: Received termination signal (Terminated)
    ...
    2021.02.24 04:40:30.582078 [ 39 ] {}  Application: Closed all listening sockets.
    2021.02.24 04:40:30.582124 [ 39 ] {}  Application: Closed connections.
    2021.02.24 04:40:30.583770 [ 39 ] {}  Application: Shutting down storages.
    2021.02.24 04:40:30.583932 [ 39 ] {}  Context: Shutdown disk data
    2021.02.24 04:40:30.583951 [ 39 ] {}  Context: Shutdown disk default
    2021.02.24 04:40:30.584163 [ 46 ] {}  SystemLog (system.query_log): Terminating
    2021.02.24 04:40:30.586025 [ 39 ] {}  BackgroundSchedulePool/BgSchPool: Waiting for threads to finish.
    2021.02.24 04:40:34.352701 [ 73 ] {}  DDLWorker: Initialized DDLWorker thread
    2021.02.24 04:40:34.352758 [ 73 ] {}  DDLWorker: Scheduling tasks
---
 src/Databases/DatabaseReplicatedWorker.cpp |  6 ++++--
 src/Databases/DatabaseReplicatedWorker.h   |  2 +-
 src/Interpreters/DDLWorker.cpp             | 12 ++++++++----
 src/Interpreters/DDLWorker.h               |  3 ++-
 4 files changed, 15 insertions(+), 8 deletions(-)

diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp
index e0c5717711c..25a19c4dfb2 100644
--- a/src/Databases/DatabaseReplicatedWorker.cpp
+++ b/src/Databases/DatabaseReplicatedWorker.cpp
@@ -22,7 +22,7 @@ DatabaseReplicatedDDLWorker::DatabaseReplicatedDDLWorker(DatabaseReplicated * db
     /// We also need similar graph to load tables on server startup in order of topsort.
 }
 
-void DatabaseReplicatedDDLWorker::initializeMainThread()
+bool DatabaseReplicatedDDLWorker::initializeMainThread()
 {
     while (!stop_flag)
     {
@@ -33,7 +33,7 @@ void DatabaseReplicatedDDLWorker::initializeMainThread()
                 database->tryConnectToZooKeeperAndInitDatabase(false);
             initializeReplication();
             initialized = true;
-            return;
+            return true;
         }
         catch (...)
         {
@@ -41,6 +41,8 @@ void DatabaseReplicatedDDLWorker::initializeMainThread()
             sleepForSeconds(5);
         }
     }
+
+    return false;
 }
 
 void DatabaseReplicatedDDLWorker::shutdown()
diff --git a/src/Databases/DatabaseReplicatedWorker.h b/src/Databases/DatabaseReplicatedWorker.h
index 6ba46a98bca..3a45817c755 100644
--- a/src/Databases/DatabaseReplicatedWorker.h
+++ b/src/Databases/DatabaseReplicatedWorker.h
@@ -30,7 +30,7 @@ public:
     void shutdown() override;
 
 private:
-    void initializeMainThread() override;
+    bool initializeMainThread() override;
     void initializeReplication();
 
     DDLTaskPtr initAndCheckTask(const String & entry_name, String & out_reason, const ZooKeeperPtr & zookeeper) override;
diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 45ef0d949f5..19534c13c99 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -936,11 +936,11 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry)
 }
 
 
-void DDLWorker::initializeMainThread()
+bool DDLWorker::initializeMainThread()
 {
     assert(!initialized);
     setThreadName("DDLWorker");
-    LOG_DEBUG(log, "Started DDLWorker thread");
+    LOG_DEBUG(log, "Initializing DDLWorker thread");
 
     while (!stop_flag)
     {
@@ -949,7 +949,7 @@ void DDLWorker::initializeMainThread()
             auto zookeeper = getAndSetZooKeeper();
             zookeeper->createAncestors(fs::path(queue_dir) / "");
             initialized = true;
-            return;
+            return true;
         }
         catch (const Coordination::Exception & e)
         {
@@ -970,6 +970,8 @@ void DDLWorker::initializeMainThread()
         /// Avoid busy loop when ZooKeeper is not available.
         sleepForSeconds(5);
     }
+
+    return false;
 }
 
 void DDLWorker::runMainThread()
@@ -1000,7 +1002,9 @@ void DDLWorker::runMainThread()
             /// Reinitialize DDLWorker state (including ZooKeeper connection) if required
             if (!initialized)
             {
-                initializeMainThread();
+                /// Stopped
+                if (!initializeMainThread())
+                    break;
                 LOG_DEBUG(log, "Initialized DDLWorker thread");
             }
 
diff --git a/src/Interpreters/DDLWorker.h b/src/Interpreters/DDLWorker.h
index e0d8077265f..0ef7456430f 100644
--- a/src/Interpreters/DDLWorker.h
+++ b/src/Interpreters/DDLWorker.h
@@ -104,7 +104,8 @@ protected:
     /// Init task node
     void createStatusDirs(const std::string & node_path, const ZooKeeperPtr & zookeeper);
 
-    virtual void initializeMainThread();
+    /// Return false if the worker was stopped (stop_flag = true)
+    virtual bool initializeMainThread();
 
     void runMainThread();
     void runCleanupThread();

From 0e68fc67aaef9531440d7c5eb3686d47b5db1963 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Wed, 24 Feb 2021 21:22:36 +0300
Subject: [PATCH 459/510] Add other distributed_ddl settings into config.xml as
 an example

---
 programs/server/config.xml | 13 +++++++++++++
 1 file changed, 13 insertions(+)

diff --git a/programs/server/config.xml b/programs/server/config.xml
index ba9b8b04b05..b72cf53ca03 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -892,6 +892,19 @@
 
         
         
+
+        
+
+        
+        
+
+        
+        
+
+        
+        
     
 
     

From 7d51ae321239f67357acaedc881cf4a0f7a1c215 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Wed, 24 Feb 2021 21:22:36 +0300
Subject: [PATCH 460/510] DDLWorker: Fix processing of tasks in parallel
 (correct queue begin)

Otherwise it will:
- start from incorrect task and bail because finished node already
  exists
- and also process alreayd processed items
---
 src/Interpreters/DDLWorker.cpp | 24 ++++++++++++++----------
 1 file changed, 14 insertions(+), 10 deletions(-)

diff --git a/src/Interpreters/DDLWorker.cpp b/src/Interpreters/DDLWorker.cpp
index 19534c13c99..4da0d21791b 100644
--- a/src/Interpreters/DDLWorker.cpp
+++ b/src/Interpreters/DDLWorker.cpp
@@ -338,19 +338,23 @@ void DDLWorker::scheduleTasks(bool reinitialized)
     else if (max_tasks_in_queue < queue_nodes.size())
         cleanup_event->set();
 
-    bool server_startup = current_tasks.empty();
+    /// Detect queue start, using:
+    /// - skipped tasks
+    /// - in memory tasks (that are currently active)
     auto begin_node = queue_nodes.begin();
-
-    if (!server_startup)
+    UInt64 last_task_id = 0;
+    if (!current_tasks.empty())
     {
-        /// We will recheck status of last executed tasks. It's useful if main thread was just restarted.
-        auto & min_task = current_tasks.front();
-        String min_entry_name = last_skipped_entry_name ? std::min(min_task->entry_name, *last_skipped_entry_name) : min_task->entry_name;
-        begin_node = std::upper_bound(queue_nodes.begin(), queue_nodes.end(), min_entry_name);
-        current_tasks.remove_if([](const DDLTaskPtr & t) { return t->completely_processed.load(); });
+        auto & last_task = current_tasks.back();
+        last_task_id = DDLTaskBase::getLogEntryNumber(last_task->entry_name);
+        begin_node = std::upper_bound(queue_nodes.begin(), queue_nodes.end(), last_task->entry_name);
+    }
+    if (last_skipped_entry_name)
+    {
+        UInt64 last_skipped_entry_id = DDLTaskBase::getLogEntryNumber(*last_skipped_entry_name);
+        if (last_skipped_entry_id > last_task_id)
+            begin_node = std::upper_bound(queue_nodes.begin(), queue_nodes.end(), *last_skipped_entry_name);
     }
-
-    assert(current_tasks.empty());
 
     for (auto it = begin_node; it != queue_nodes.end() && !stop_flag; ++it)
     {

From d42d4cfd6b3ddce7ec92fb35b5b11e96f4da3e66 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Wed, 24 Feb 2021 21:22:36 +0300
Subject: [PATCH 461/510] Improve test_distributed_ddl_parallel to cover more
 cases

Refs: #21264
---
 .../configs/ddl_a.xml                         |  5 ++
 .../configs/ddl_b.xml                         |  5 ++
 .../configs/dict.xml                          | 70 +++++++++++++------
 .../configs/remote_servers.xml                | 18 ++++-
 .../test_distributed_ddl_parallel/test.py     | 67 ++++++++++++++----
 5 files changed, 126 insertions(+), 39 deletions(-)
 create mode 100644 tests/integration/test_distributed_ddl_parallel/configs/ddl_a.xml
 create mode 100644 tests/integration/test_distributed_ddl_parallel/configs/ddl_b.xml

diff --git a/tests/integration/test_distributed_ddl_parallel/configs/ddl_a.xml b/tests/integration/test_distributed_ddl_parallel/configs/ddl_a.xml
new file mode 100644
index 00000000000..b926f99c687
--- /dev/null
+++ b/tests/integration/test_distributed_ddl_parallel/configs/ddl_a.xml
@@ -0,0 +1,5 @@
+
+    
+        2
+    
+
diff --git a/tests/integration/test_distributed_ddl_parallel/configs/ddl_b.xml b/tests/integration/test_distributed_ddl_parallel/configs/ddl_b.xml
new file mode 100644
index 00000000000..2f038919032
--- /dev/null
+++ b/tests/integration/test_distributed_ddl_parallel/configs/ddl_b.xml
@@ -0,0 +1,5 @@
+
+    
+        20
+    
+
diff --git a/tests/integration/test_distributed_ddl_parallel/configs/dict.xml b/tests/integration/test_distributed_ddl_parallel/configs/dict.xml
index 610d55841a0..d94b3f61dd9 100644
--- a/tests/integration/test_distributed_ddl_parallel/configs/dict.xml
+++ b/tests/integration/test_distributed_ddl_parallel/configs/dict.xml
@@ -1,26 +1,50 @@
 
 
-   
-      slow_dict
-      
-         
-             sleep 7
-             TabSeparated
-         
-      
-      
-         
-      
-      
-         
-            id
-         
-         
-            value
-            String
-            
-         
-      
-      0
-  
+    
+        slow_dict_7
+        
+           
+               sleep 7
+               TabSeparated
+           
+        
+        
+           
+        
+        
+           
+              id
+           
+           
+              value
+              String
+              
+           
+        
+        0
+    
+
+    
+        slow_dict_3
+        
+           
+               sleep 3
+               TabSeparated
+           
+        
+        
+           
+        
+        
+           
+              id
+           
+           
+              value
+              String
+              
+           
+        
+        0
+    
 
diff --git a/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml b/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml
index 8ffa9f024d7..eb0ee60186b 100644
--- a/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml
+++ b/tests/integration/test_distributed_ddl_parallel/configs/remote_servers.xml
@@ -1,6 +1,6 @@
 
 
-    
+    
         
             
                 n1
@@ -13,6 +13,20 @@
                 9000
             
         
-    
+    
+    
+        
+            
+                n3
+                9000
+            
+        
+        
+            
+                n4
+                9000
+            
+        
+    
 
 
diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py
index a9f2f069e27..44971ca3d9e 100644
--- a/tests/integration/test_distributed_ddl_parallel/test.py
+++ b/tests/integration/test_distributed_ddl_parallel/test.py
@@ -29,11 +29,12 @@ class SafeThread(threading.Thread):
         if self.exception:
             raise self.exception
 
-def add_instance(name):
+def add_instance(name, ddl_config=None):
     main_configs=[
-        'configs/ddl.xml',
         'configs/remote_servers.xml',
     ]
+    if ddl_config:
+        main_configs.append(ddl_config)
     dictionaries=[
         'configs/dict.xml',
     ]
@@ -43,8 +44,12 @@ def add_instance(name):
         with_zookeeper=True)
 
 initiator = add_instance('initiator')
-n1 = add_instance('n1')
-n2 = add_instance('n2')
+# distributed_ddl.pool_size = 2
+n1 = add_instance('n1', 'configs/ddl_a.xml')
+n2 = add_instance('n2', 'configs/ddl_a.xml')
+# distributed_ddl.pool_size = 20
+n3 = add_instance('n3', 'configs/ddl_b.xml')
+n4 = add_instance('n4', 'configs/ddl_b.xml')
 
 @pytest.fixture(scope='module', autouse=True)
 def start_cluster():
@@ -68,19 +73,32 @@ def longer_then(sec):
         return inner
     return wrapper
 
-# It takes 7 seconds to load slow_dict.
-def thread_reload_dictionary():
-    initiator.query('SYSTEM RELOAD DICTIONARY ON CLUSTER cluster slow_dict', settings={
+# It takes 7 seconds to load slow_dict_7.
+def execute_reload_dictionary_slow_dict_7():
+    initiator.query('SYSTEM RELOAD DICTIONARY ON CLUSTER cluster_a slow_dict_7', settings={
+        'distributed_ddl_task_timeout': 60,
+    })
+def execute_reload_dictionary_slow_dict_3():
+    initiator.query('SYSTEM RELOAD DICTIONARY ON CLUSTER cluster_b slow_dict_3', settings={
+        'distributed_ddl_task_timeout': 60,
+    })
+def execute_smoke_query():
+    initiator.query('DROP DATABASE IF EXISTS foo ON CLUSTER cluster_b', settings={
         'distributed_ddl_task_timeout': 60,
     })
 
+def check_log():
+    # ensure that none of tasks processed multiple times
+    for _, instance in list(cluster.instances.items()):
+        assert not instance.contains_in_log('Coordination::Exception: Node exists')
+
 # NOTE: uses inner function to exclude slow start_cluster() from timeout.
 
-def test_dict_load():
+def test_slow_dict_load_7():
     @pytest.mark.timeout(10)
     @longer_then(7)
     def inner_test():
-        initiator.query('SYSTEM RELOAD DICTIONARY slow_dict')
+        initiator.query('SYSTEM RELOAD DICTIONARY slow_dict_7')
     inner_test()
 
 def test_all_in_parallel():
@@ -89,12 +107,13 @@ def test_all_in_parallel():
     def inner_test():
         threads = []
         for _ in range(2):
-            threads.append(SafeThread(target=thread_reload_dictionary))
+            threads.append(SafeThread(target=execute_reload_dictionary_slow_dict_7))
         for thread in threads:
             thread.start()
         for thread in threads:
             thread.join(70)
     inner_test()
+    check_log()
 
 def test_two_in_parallel_two_queued():
     @pytest.mark.timeout(19)
@@ -102,15 +121,35 @@ def test_two_in_parallel_two_queued():
     def inner_test():
         threads = []
         for _ in range(4):
-            threads.append(SafeThread(target=thread_reload_dictionary))
+            threads.append(SafeThread(target=execute_reload_dictionary_slow_dict_7))
         for thread in threads:
             thread.start()
         for thread in threads:
             thread.join(70)
     inner_test()
+    check_log()
 
 def test_smoke():
     for _ in range(100):
-        initiator.query('DROP DATABASE IF EXISTS foo ON CLUSTER cluster', settings={
-            'distributed_ddl_task_timeout': 60,
-        })
+        execute_smoke_query()
+    check_log()
+
+def test_smoke_parallel():
+    threads = []
+    for _ in range(100):
+        threads.append(SafeThread(target=execute_smoke_query))
+    for thread in threads:
+        thread.start()
+    for thread in threads:
+        thread.join(70)
+    check_log()
+
+def test_smoke_parallel_dict_reload():
+    threads = []
+    for _ in range(100):
+        threads.append(SafeThread(target=execute_reload_dictionary_slow_dict_3))
+    for thread in threads:
+        thread.start()
+    for thread in threads:
+        thread.join(70)
+    check_log()

From 51021c11647b49f067de737c78c1532b0a77db32 Mon Sep 17 00:00:00 2001
From: feng lv 
Date: Sun, 28 Feb 2021 05:24:39 +0000
Subject: [PATCH 462/510] forbid to drop a column if it's referenced by
 materialized view

---
 src/Interpreters/InterpreterAlterQuery.cpp    |   2 +-
 src/Interpreters/InterpreterSelectQuery.h     |   2 +
 src/Storages/IStorage.cpp                     |  17 +-
 src/Storages/IStorage.h                       |   8 +-
 src/Storages/MergeTree/MergeTreeData.cpp      |  26 ++-
 src/Storages/MergeTree/MergeTreeData.h        |   2 +-
 src/Storages/StorageBuffer.cpp                |  17 +-
 src/Storages/StorageBuffer.h                  |   2 +-
 src/Storages/StorageDistributed.cpp           |  17 +-
 src/Storages/StorageDistributed.h             |   2 +-
 src/Storages/StorageMaterializedView.cpp      |   3 +-
 src/Storages/StorageMaterializedView.h        |   2 +-
 src/Storages/StorageMerge.cpp                 |  15 +-
 src/Storages/StorageMerge.h                   |   2 +-
 src/Storages/StorageNull.cpp                  |  15 +-
 src/Storages/StorageNull.h                    |   2 +-
 src/Storages/StorageProxy.h                   |   4 +-
 ...bid_drop_column_referenced_by_mv.reference |   0
 ...46_forbid_drop_column_referenced_by_mv.sql | 172 ++++++++++++++++++
 19 files changed, 285 insertions(+), 25 deletions(-)
 create mode 100644 tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.reference
 create mode 100644 tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql

diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp
index 6294b31cc8c..37eaecf9a90 100644
--- a/src/Interpreters/InterpreterAlterQuery.cpp
+++ b/src/Interpreters/InterpreterAlterQuery.cpp
@@ -137,7 +137,7 @@ BlockIO InterpreterAlterQuery::execute()
         StorageInMemoryMetadata metadata = table->getInMemoryMetadata();
         alter_commands.validate(metadata, context);
         alter_commands.prepare(metadata);
-        table->checkAlterIsPossible(alter_commands, context.getSettingsRef());
+        table->checkAlterIsPossible(alter_commands, context);
         table->alter(alter_commands, context, alter_lock);
     }
 
diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h
index 1fff316e1d4..2c563c0f917 100644
--- a/src/Interpreters/InterpreterSelectQuery.h
+++ b/src/Interpreters/InterpreterSelectQuery.h
@@ -89,6 +89,8 @@ public:
 
     static void addEmptySourceToQueryPlan(QueryPlan & query_plan, const Block & source_header, const SelectQueryInfo & query_info);
 
+    Names getRequiredColumns() { return required_columns; }
+
 private:
     InterpreterSelectQuery(
         const ASTPtr & query_ptr_,
diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp
index 2400b0587ba..7b68e84e6a1 100644
--- a/src/Storages/IStorage.cpp
+++ b/src/Storages/IStorage.cpp
@@ -134,7 +134,7 @@ void IStorage::alter(const AlterCommands & params, const Context & context, Tabl
 }
 
 
-void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
+void IStorage::checkAlterIsPossible(const AlterCommands & commands, const Context & /* context */) const
 {
     for (const auto & command : commands)
     {
@@ -182,6 +182,21 @@ Names IStorage::getAllRegisteredNames() const
     return result;
 }
 
+NameDependencies IStorage::getColumnNamesAndReferencedMvMap(const Context & context) const
+{
+    NameDependencies name_deps;
+    auto dependencies = DatabaseCatalog::instance().getDependencies(storage_id);
+    for (const auto & depend_id : dependencies)
+    {
+        auto depend_table = DatabaseCatalog::instance().getTable(depend_id, context);
+        const auto & select_query = depend_table->getInMemoryMetadataPtr()->select.inner_query;
+        auto required_columns = InterpreterSelectQuery(select_query, context, SelectQueryOptions{}.noModify()).getRequiredColumns();
+        for (const auto & col_name : required_columns)
+            name_deps[col_name].push_back(depend_id.table_name);
+    }
+    return name_deps;
+}
+
 std::string PrewhereDAGInfo::dump() const
 {
     WriteBufferFromOwnString ss;
diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h
index 1a27dbd637f..72eb50e8a85 100644
--- a/src/Storages/IStorage.h
+++ b/src/Storages/IStorage.h
@@ -57,6 +57,8 @@ struct StreamLocalLimits;
 class EnabledQuota;
 struct SelectQueryInfo;
 
+using NameDependencies = std::unordered_map>;
+
 struct ColumnSize
 {
     size_t marks = 0;
@@ -173,8 +175,10 @@ public:
     virtual NamesAndTypesList getVirtuals() const;
 
     Names getAllRegisteredNames() const override;
-protected:
 
+    NameDependencies getColumnNamesAndReferencedMvMap(const Context & context) const;
+
+protected:
     /// Returns whether the column is virtual - by default all columns are real.
     /// Initially reserved virtual column name may be shadowed by real column.
     bool isVirtualColumn(const String & column_name, const StorageMetadataPtr & metadata_snapshot) const;
@@ -362,7 +366,7 @@ public:
     /** Checks that alter commands can be applied to storage. For example, columns can be modified,
       * or primary key can be changes, etc.
       */
-    virtual void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const;
+    virtual void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const;
 
     /**
       * Checks that mutation commands can be applied to storage.
diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp
index 2d841b98c59..ed6873f8370 100644
--- a/src/Storages/MergeTree/MergeTreeData.cpp
+++ b/src/Storages/MergeTree/MergeTreeData.cpp
@@ -4,12 +4,11 @@
 #include 
 #include 
 #include 
-#include 
-#include 
 #include 
+#include 
+#include 
 #include 
 #include 
-#include 
 #include 
 #include 
 #include 
@@ -17,10 +16,11 @@
 #include 
 #include 
 #include 
+#include 
 #include 
+#include 
 #include 
 #include 
-#include 
 #include 
 #include 
 #include 
@@ -30,10 +30,11 @@
 #include 
 #include 
 #include 
+#include 
 #include 
 #include 
-#include 
 #include 
+#include 
 #include 
 #include 
 #include 
@@ -1407,12 +1408,14 @@ void checkVersionColumnTypesConversion(const IDataType * old_type, const IDataTy
 
 }
 
-void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
+void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
 {
     /// Check that needed transformations can be applied to the list of columns without considering type conversions.
     StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
     StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
 
+    const auto & settings = context.getSettingsRef();
+
     if (!settings.allow_non_metadata_alters)
     {
 
@@ -1484,6 +1487,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
         old_types.emplace(column.name, column.type.get());
 
     NamesAndTypesList columns_to_check_conversion;
+    auto name_deps = getColumnNamesAndReferencedMvMap(context);
     for (const AlterCommand & command : commands)
     {
         /// Just validate partition expression
@@ -1563,6 +1567,16 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const S
                     "Trying to ALTER DROP key " + backQuoteIfNeed(command.column_name) + " column which is a part of key expression",
                     ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
             }
+
+            auto deps_mv = name_deps[command.column_name];
+            if (!deps_mv.empty())
+            {
+                throw Exception(
+                    "Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
+                        + toString(deps_mv),
+                    ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
+            }
+
             dropped_columns.emplace(command.column_name);
         }
         else if (command.isRequireMutationStage(getInMemoryMetadata()))
diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h
index f03f3f1dd8c..70c78af7bda 100644
--- a/src/Storages/MergeTree/MergeTreeData.h
+++ b/src/Storages/MergeTree/MergeTreeData.h
@@ -517,7 +517,7 @@ public:
     /// - all type conversions can be done.
     /// - columns corresponding to primary key, indices, sign, sampling expression and date are not affected.
     /// If something is wrong, throws an exception.
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
 
     /// Checks if the Mutation can be performed.
     /// (currently no additional checks: always ok)
diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp
index e28d5f4d6d1..46cdf237d91 100644
--- a/src/Storages/StorageBuffer.cpp
+++ b/src/Storages/StorageBuffer.cpp
@@ -58,6 +58,7 @@ namespace ErrorCodes
     extern const int LOGICAL_ERROR;
     extern const int INFINITE_LOOP;
     extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
+    extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
 }
 
 
@@ -910,8 +911,9 @@ void StorageBuffer::reschedule()
     flush_handle->scheduleAfter(std::min(min, max) * 1000);
 }
 
-void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
+void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
 {
+    auto name_deps = getColumnNamesAndReferencedMvMap(context);
     for (const auto & command : commands)
     {
         if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
@@ -919,6 +921,17 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const S
             throw Exception(
                 "Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
                 ErrorCodes::NOT_IMPLEMENTED);
+        if (command.type == AlterCommand::Type::DROP_COLUMN)
+        {
+            auto deps_mv = name_deps[command.column_name];
+            if (!deps_mv.empty())
+            {
+                throw Exception(
+                    "Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
+                        + toString(deps_mv),
+                    ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
+            }
+        }
     }
 }
 
@@ -955,7 +968,7 @@ std::optional StorageBuffer::totalBytes(const Settings & /*settings*/) c
 void StorageBuffer::alter(const AlterCommands & params, const Context & context, TableLockHolder &)
 {
     auto table_id = getStorageID();
-    checkAlterIsPossible(params, context.getSettingsRef());
+    checkAlterIsPossible(params, context);
     auto metadata_snapshot = getInMemoryMetadataPtr();
 
     /// Flush all buffers to storages, so that no non-empty blocks of the old
diff --git a/src/Storages/StorageBuffer.h b/src/Storages/StorageBuffer.h
index 46907ca196b..f6904ddb0e4 100644
--- a/src/Storages/StorageBuffer.h
+++ b/src/Storages/StorageBuffer.h
@@ -99,7 +99,7 @@ public:
 
     bool mayBenefitFromIndexForIn(const ASTPtr & left_in_operand, const Context & query_context, const StorageMetadataPtr & metadata_snapshot) const override;
 
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
 
     /// The structure of the subordinate table is not checked and does not change.
     void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp
index c08dc38fa2d..337b89af017 100644
--- a/src/Storages/StorageDistributed.cpp
+++ b/src/Storages/StorageDistributed.cpp
@@ -84,6 +84,7 @@ namespace ErrorCodes
     extern const int TOO_MANY_ROWS;
     extern const int UNABLE_TO_SKIP_UNUSED_SHARDS;
     extern const int INVALID_SHARD_ID;
+    extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
 }
 
 namespace ActionLocks
@@ -577,8 +578,9 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta
 }
 
 
-void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
+void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
 {
+    auto name_deps = getColumnNamesAndReferencedMvMap(context);
     for (const auto & command : commands)
     {
         if (command.type != AlterCommand::Type::ADD_COLUMN
@@ -589,6 +591,17 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, co
 
             throw Exception("Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
                 ErrorCodes::NOT_IMPLEMENTED);
+        if (command.type == AlterCommand::DROP_COLUMN)
+        {
+            auto deps_mv = name_deps[command.column_name];
+            if (!deps_mv.empty())
+            {
+                throw Exception(
+                    "Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
+                        + toString(deps_mv),
+                    ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
+            }
+        }
     }
 }
 
@@ -596,7 +609,7 @@ void StorageDistributed::alter(const AlterCommands & params, const Context & con
 {
     auto table_id = getStorageID();
 
-    checkAlterIsPossible(params, context.getSettingsRef());
+    checkAlterIsPossible(params, context);
     StorageInMemoryMetadata new_metadata = getInMemoryMetadata();
     params.apply(new_metadata, context);
     DatabaseCatalog::instance().getDatabase(table_id.database_name)->alterTable(context, table_id, new_metadata);
diff --git a/src/Storages/StorageDistributed.h b/src/Storages/StorageDistributed.h
index 4d3869f7c5c..3980ea8244d 100644
--- a/src/Storages/StorageDistributed.h
+++ b/src/Storages/StorageDistributed.h
@@ -85,7 +85,7 @@ public:
     void rename(const String & new_path_to_table_data, const StorageID & new_table_id) override;
     void renameOnDisk(const String & new_path_to_table_data);
 
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
 
     /// in the sub-tables, you need to manually add and delete columns
     /// the structure of the sub-table is not checked
diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp
index 2d211c8061b..4022529edae 100644
--- a/src/Storages/StorageMaterializedView.cpp
+++ b/src/Storages/StorageMaterializedView.cpp
@@ -296,8 +296,9 @@ void StorageMaterializedView::alter(
 }
 
 
-void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const
+void StorageMaterializedView::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
 {
+    const auto & settings = context.getSettingsRef();
     if (settings.allow_experimental_alter_materialized_view_structure)
     {
         for (const auto & command : commands)
diff --git a/src/Storages/StorageMaterializedView.h b/src/Storages/StorageMaterializedView.h
index 4b10cf7a9b5..ccb56ec9fe5 100644
--- a/src/Storages/StorageMaterializedView.h
+++ b/src/Storages/StorageMaterializedView.h
@@ -54,7 +54,7 @@ public:
 
     void checkMutationIsPossible(const MutationCommands & commands, const Settings & settings) const override;
 
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override;
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
 
     Pipe alterPartition(const StorageMetadataPtr & metadata_snapshot, const PartitionCommands & commands, const Context & context) override;
 
diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp
index 91ebfaa3a27..1048cbc6aa3 100644
--- a/src/Storages/StorageMerge.cpp
+++ b/src/Storages/StorageMerge.cpp
@@ -37,6 +37,7 @@ namespace ErrorCodes
     extern const int ILLEGAL_PREWHERE;
     extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
     extern const int SAMPLING_NOT_SUPPORTED;
+    extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
 }
 
 namespace
@@ -472,8 +473,9 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & cont
 }
 
 
-void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
+void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
 {
+    auto name_deps = getColumnNamesAndReferencedMvMap(context);
     for (const auto & command : commands)
     {
         if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
@@ -481,6 +483,17 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Se
             throw Exception(
                 "Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
                 ErrorCodes::NOT_IMPLEMENTED);
+        if (command.type == AlterCommand::Type::DROP_COLUMN)
+        {
+            auto deps_mv = name_deps[command.column_name];
+            if (!deps_mv.empty())
+            {
+                throw Exception(
+                    "Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
+                        + toString(deps_mv),
+                    ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
+            }
+        }
     }
 }
 
diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h
index 3ac251fbe52..eaffd34a379 100644
--- a/src/Storages/StorageMerge.h
+++ b/src/Storages/StorageMerge.h
@@ -38,7 +38,7 @@ public:
         size_t max_block_size,
         unsigned num_streams) override;
 
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
 
     /// you need to add and remove columns in the sub-tables manually
     /// the structure of sub-tables is not checked
diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp
index f324d502834..8b2c96ba436 100644
--- a/src/Storages/StorageNull.cpp
+++ b/src/Storages/StorageNull.cpp
@@ -16,6 +16,7 @@ namespace ErrorCodes
 {
     extern const int NOT_IMPLEMENTED;
     extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
+    extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
 }
 
 
@@ -35,8 +36,9 @@ void registerStorageNull(StorageFactory & factory)
     });
 }
 
-void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const
+void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const
 {
+    auto name_deps = getColumnNamesAndReferencedMvMap(context);
     for (const auto & command : commands)
     {
         if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN
@@ -44,6 +46,17 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Set
             throw Exception(
                 "Alter of type '" + alterTypeToString(command.type) + "' is not supported by storage " + getName(),
                 ErrorCodes::NOT_IMPLEMENTED);
+        if (command.type == AlterCommand::DROP_COLUMN)
+        {
+            auto deps_mv = name_deps[command.column_name];
+            if (!deps_mv.empty())
+            {
+                throw Exception(
+                    "Trying to ALTER DROP column " + backQuoteIfNeed(command.column_name) + " which is referenced by materialized view "
+                        + toString(deps_mv),
+                    ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN);
+            }
+        }
     }
 }
 
diff --git a/src/Storages/StorageNull.h b/src/Storages/StorageNull.h
index 7d3d15f1b0f..943c056a588 100644
--- a/src/Storages/StorageNull.h
+++ b/src/Storages/StorageNull.h
@@ -41,7 +41,7 @@ public:
         return std::make_shared(metadata_snapshot->getSampleBlock());
     }
 
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & /* settings */) const override;
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override;
 
     void alter(const AlterCommands & params, const Context & context, TableLockHolder & table_lock_holder) override;
 
diff --git a/src/Storages/StorageProxy.h b/src/Storages/StorageProxy.h
index fed9dd04e76..0349319d8fa 100644
--- a/src/Storages/StorageProxy.h
+++ b/src/Storages/StorageProxy.h
@@ -97,9 +97,9 @@ public:
         IStorage::setInMemoryMetadata(getNested()->getInMemoryMetadata());
     }
 
-    void checkAlterIsPossible(const AlterCommands & commands, const Settings & settings) const override
+    void checkAlterIsPossible(const AlterCommands & commands, const Context & context) const override
     {
-        getNested()->checkAlterIsPossible(commands, settings);
+        getNested()->checkAlterIsPossible(commands, context);
     }
 
     Pipe alterPartition(
diff --git a/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.reference b/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.reference
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql b/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql
new file mode 100644
index 00000000000..c05a7beff44
--- /dev/null
+++ b/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql
@@ -0,0 +1,172 @@
+-- MergeTree
+DROP TABLE IF EXISTS `01746_merge_tree`;
+CREATE TABLE `01746_merge_tree`
+(
+    `n1` Int8,
+    `n2` Int8,
+    `n3` Int8,
+    `n4` Int8
+)
+ENGINE = MergeTree
+ORDER BY n1;
+
+DROP TABLE IF EXISTS `01746_merge_tree_mv`;
+CREATE MATERIALIZED VIEW `01746_merge_tree_mv`
+ENGINE = Memory AS
+SELECT
+    n2,
+    n3
+FROM `01746_merge_tree`;
+
+ALTER TABLE `01746_merge_tree`
+    DROP COLUMN n3;  -- { serverError 524 }
+
+ALTER TABLE `01746_merge_tree`
+    DROP COLUMN n2;  -- { serverError 524 }
+
+-- ok
+ALTER TABLE `01746_merge_tree`
+    DROP COLUMN n4;
+
+DROP TABLE `01746_merge_tree`;
+DROP TABLE `01746_merge_tree_mv`;
+
+-- Null 
+DROP TABLE IF EXISTS `01746_null`;
+CREATE TABLE `01746_null`
+(
+    `n1` Int8,
+    `n2` Int8,
+    `n3` Int8
+)
+ENGINE = Null;
+
+DROP TABLE IF EXISTS `01746_null_mv`;
+CREATE MATERIALIZED VIEW `01746_null_mv`
+ENGINE = Memory AS
+SELECT
+    n1,
+    n2
+FROM `01746_null`;
+
+ALTER TABLE `01746_null`
+    DROP COLUMN n1; -- { serverError 524 }
+
+ALTER TABLE `01746_null`
+    DROP COLUMN n2; -- { serverError 524 }
+
+-- ok
+ALTER TABLE `01746_null`
+    DROP COLUMN n3;
+
+DROP TABLE `01746_null`;
+DROP TABLE `01746_null_mv`;
+
+-- Distributed
+
+DROP TABLE IF EXISTS `01746_local`;
+CREATE TABLE `01746_local`
+(
+    `n1` Int8,
+    `n2` Int8,
+    `n3` Int8
+)
+ENGINE = Memory;
+
+DROP TABLE IF EXISTS `01746_dist`;
+CREATE TABLE `01746_dist` AS `01746_local`
+ENGINE = Distributed('test_shard_localhost', default, `01746_local`, rand());
+
+DROP TABLE IF EXISTS `01746_dist_mv`;
+CREATE MATERIALIZED VIEW `01746_dist_mv`
+ENGINE = Memory AS
+SELECT
+    n1,
+    n2
+FROM `01746_dist`;
+
+ALTER TABLE `01746_dist`
+    DROP COLUMN n1; -- { serverError 524 }
+
+ALTER TABLE `01746_dist`
+    DROP COLUMN n2; -- { serverError 524 }
+
+-- ok
+ALTER TABLE `01746_dist`
+    DROP COLUMN n3;
+
+DROP TABLE `01746_local`;
+DROP TABLE `01746_dist`;
+DROP TABLE `01746_dist_mv`;
+
+-- Merge
+DROP TABLE IF EXISTS `01746_merge_t`;
+CREATE TABLE `01746_merge_t`
+(
+    `n1` Int8,
+    `n2` Int8,
+    `n3` Int8
+)
+ENGINE = Memory;
+
+DROP TABLE IF EXISTS `01746_merge`;
+CREATE TABLE `01746_merge` AS `01746_merge_t`
+ENGINE = Merge(default, '01746_merge_t');
+
+DROP TABLE IF EXISTS `01746_merge_mv`;
+CREATE MATERIALIZED VIEW `01746_merge_mv`
+ENGINE = Memory AS
+SELECT
+    n1,
+    n2
+FROM `01746_merge`;
+
+ALTER TABLE `01746_merge`
+    DROP COLUMN n1; -- { serverError 524 }
+
+ALTER TABLE `01746_merge`
+    DROP COLUMN n2; -- { serverError 524 }
+
+-- ok
+ALTER TABLE `01746_merge`
+    DROP COLUMN n3;
+
+DROP TABLE `01746_merge_t`;
+DROP TABLE `01746_merge`;
+DROP TABLE `01746_merge_mv`;
+
+-- Buffer
+DROP TABLE IF EXISTS `01746_buffer_t`;
+CREATE TABLE `01746_buffer_t`
+(
+    `n1` Int8,
+    `n2` Int8,
+    `n3` Int8
+)
+ENGINE = Memory;
+
+DROP TABLE IF EXISTS `01746_buffer`;
+CREATE TABLE `01746_buffer` AS `01746_buffer_t`
+ENGINE = Buffer(default, `01746_buffer_t`, 16, 10, 100, 10000, 1000000, 10000000, 100000000);
+
+DROP TABLE IF EXISTS `01746_buffer_mv`;
+CREATE MATERIALIZED VIEW `01746_buffer_mv`
+ENGINE = Memory AS
+SELECT
+    n1,
+    n2
+FROM `01746_buffer`;
+
+ALTER TABLE `01746_buffer`
+    DROP COLUMN n1; -- { serverError 524 }
+
+ALTER TABLE `01746_buffer`
+    DROP COLUMN n2; -- { serverError 524 }
+
+-- ok
+ALTER TABLE `01746_buffer`
+    DROP COLUMN n3;
+
+DROP TABLE `01746_buffer_t`;
+DROP TABLE `01746_buffer`;
+DROP TABLE `01746_buffer_mv`;

From 8b58dba09410f4539b997f3b79aff6c0ede7307c Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Sun, 28 Feb 2021 10:52:09 +0300
Subject: [PATCH 463/510] Drop unused HexWriteBuffer

---
 src/IO/HexWriteBuffer.cpp                | 30 ------------------------
 src/IO/HexWriteBuffer.h                  | 28 ----------------------
 src/IO/ya.make                           |  1 -
 src/Interpreters/Cluster.cpp             |  1 -
 src/Storages/MergeTree/MergeTreeData.cpp |  1 -
 5 files changed, 61 deletions(-)
 delete mode 100644 src/IO/HexWriteBuffer.cpp
 delete mode 100644 src/IO/HexWriteBuffer.h

diff --git a/src/IO/HexWriteBuffer.cpp b/src/IO/HexWriteBuffer.cpp
deleted file mode 100644
index 4e3403ba74b..00000000000
--- a/src/IO/HexWriteBuffer.cpp
+++ /dev/null
@@ -1,30 +0,0 @@
-#include 
-#include 
-#include 
-#include 
-
-
-namespace DB
-{
-
-void HexWriteBuffer::nextImpl()
-{
-    if (!offset())
-        return;
-
-    for (Position p = working_buffer.begin(); p != pos; ++p)
-    {
-        UInt8 byte = *p;
-        out.write(hexDigitUppercase(byte / 16));
-        out.write(hexDigitUppercase(byte % 16));
-    }
-}
-
-HexWriteBuffer::~HexWriteBuffer()
-{
-    /// FIXME move final flush into the caller
-    MemoryTracker::LockExceptionInThread lock;
-    nextImpl();
-}
-
-}
diff --git a/src/IO/HexWriteBuffer.h b/src/IO/HexWriteBuffer.h
deleted file mode 100644
index a68dd29065b..00000000000
--- a/src/IO/HexWriteBuffer.h
+++ /dev/null
@@ -1,28 +0,0 @@
-#pragma once
-
-#include 
-
-
-/// Since HexWriteBuffer is often created in the inner loop, we'll make its buffer size small.
-#define DBMS_HEX_WRITE_BUFFER_SIZE 32
-
-
-namespace DB
-{
-
-/** Everything that is written into it, translates to HEX (in capital letters) and writes to another WriteBuffer.
-  */
-class HexWriteBuffer final : public WriteBuffer
-{
-protected:
-    char buf[DBMS_HEX_WRITE_BUFFER_SIZE]; //-V730
-    WriteBuffer & out;
-
-    void nextImpl() override;
-
-public:
-    HexWriteBuffer(WriteBuffer & out_) : WriteBuffer(buf, sizeof(buf)), out(out_) {}
-    ~HexWriteBuffer() override;
-};
-
-}
diff --git a/src/IO/ya.make b/src/IO/ya.make
index 980719aa74f..6605cf64277 100644
--- a/src/IO/ya.make
+++ b/src/IO/ya.make
@@ -29,7 +29,6 @@ SRCS(
     HTTPChunkedReadBuffer.cpp
     HTTPCommon.cpp
     HashingWriteBuffer.cpp
-    HexWriteBuffer.cpp
     LZMADeflatingWriteBuffer.cpp
     LZMAInflatingReadBuffer.cpp
     LimitReadBuffer.cpp
diff --git a/src/Interpreters/Cluster.cpp b/src/Interpreters/Cluster.cpp
index c9c56c96cbe..fb9788e84c4 100644
--- a/src/Interpreters/Cluster.cpp
+++ b/src/Interpreters/Cluster.cpp
@@ -6,7 +6,6 @@
 #include 
 #include 
 #include 
-#include 
 #include 
 #include 
 #include 
diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp
index 2d841b98c59..d9e24581c0c 100644
--- a/src/Storages/MergeTree/MergeTreeData.cpp
+++ b/src/Storages/MergeTree/MergeTreeData.cpp
@@ -13,7 +13,6 @@
 #include 
 #include 
 #include 
-#include 
 #include 
 #include 
 #include 

From 568a49dad59acbc036eb6dacecfac297d618e853 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Sun, 28 Feb 2021 10:53:13 +0300
Subject: [PATCH 464/510] Remove unused AsynchronousWriteBuffer

---
 src/IO/AsynchronousWriteBuffer.h | 71 --------------------------------
 src/IO/tests/CMakeLists.txt      |  3 --
 src/IO/tests/async_write.cpp     | 26 ------------
 src/IO/tests/parse_int_perf.cpp  |  1 -
 4 files changed, 101 deletions(-)
 delete mode 100644 src/IO/AsynchronousWriteBuffer.h
 delete mode 100644 src/IO/tests/async_write.cpp

diff --git a/src/IO/AsynchronousWriteBuffer.h b/src/IO/AsynchronousWriteBuffer.h
deleted file mode 100644
index 8c44f8c7d4a..00000000000
--- a/src/IO/AsynchronousWriteBuffer.h
+++ /dev/null
@@ -1,71 +0,0 @@
-#pragma once
-
-#include 
-#include 
-#include 
-#include 
-
-
-namespace DB
-{
-
-
-/** Writes data asynchronously using double buffering.
-  */
-class AsynchronousWriteBuffer : public WriteBuffer
-{
-private:
-    WriteBuffer & out;               /// The main buffer, responsible for writing data.
-    std::vector  memory;       /// A piece of memory for duplicating the buffer.
-    ThreadPool pool;                 /// For asynchronous data writing.
-    bool started;                    /// Has an asynchronous data write started?
-
-    /// Swap the main and duplicate buffers.
-    void swapBuffers()
-    {
-        swap(out);
-    }
-
-    void nextImpl() override
-    {
-        if (!offset())
-            return;
-
-        if (started)
-            pool.wait();
-        else
-            started = true;
-
-        swapBuffers();
-
-        /// The data will be written in separate stream.
-        pool.scheduleOrThrowOnError([this] { thread(); });
-    }
-
-public:
-    AsynchronousWriteBuffer(WriteBuffer & out_) : WriteBuffer(nullptr, 0), out(out_), memory(out.buffer().size()), pool(1), started(false)
-    {
-        /// Data is written to the duplicate buffer.
-        set(memory.data(), memory.size());
-    }
-
-    ~AsynchronousWriteBuffer() override
-    {
-        /// FIXME move final flush into the caller
-        MemoryTracker::LockExceptionInThread lock;
-
-        if (started)
-            pool.wait();
-
-        swapBuffers();
-        out.next();
-    }
-
-    /// That is executed in a separate thread
-    void thread()
-    {
-        out.next();
-    }
-};
-
-}
diff --git a/src/IO/tests/CMakeLists.txt b/src/IO/tests/CMakeLists.txt
index fcd59d94cb0..79800d8339c 100644
--- a/src/IO/tests/CMakeLists.txt
+++ b/src/IO/tests/CMakeLists.txt
@@ -25,9 +25,6 @@ target_link_libraries (var_uint PRIVATE clickhouse_common_io)
 add_executable (read_escaped_string read_escaped_string.cpp)
 target_link_libraries (read_escaped_string PRIVATE clickhouse_common_io)
 
-add_executable (async_write async_write.cpp)
-target_link_libraries (async_write PRIVATE dbms)
-
 add_executable (parse_int_perf parse_int_perf.cpp)
 target_link_libraries (parse_int_perf PRIVATE clickhouse_common_io)
 
diff --git a/src/IO/tests/async_write.cpp b/src/IO/tests/async_write.cpp
deleted file mode 100644
index e3bff7cf341..00000000000
--- a/src/IO/tests/async_write.cpp
+++ /dev/null
@@ -1,26 +0,0 @@
-#include 
-
-#include 
-#include 
-#include 
-#include 
-#include 
-
-
-int main(int, char **)
-try
-{
-    DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO);
-    DB::WriteBufferFromFileDescriptor out1(STDOUT_FILENO);
-    DB::AsynchronousWriteBuffer out2(out1);
-    DB::CompressedWriteBuffer out3(out2);
-
-    DB::copyData(in1, out3);
-
-    return 0;
-}
-catch (const DB::Exception & e)
-{
-    std::cerr << e.what() << ", " << e.displayText() << std::endl;
-    return 1;
-}
diff --git a/src/IO/tests/parse_int_perf.cpp b/src/IO/tests/parse_int_perf.cpp
index 93f49d80258..e35a3d8a857 100644
--- a/src/IO/tests/parse_int_perf.cpp
+++ b/src/IO/tests/parse_int_perf.cpp
@@ -62,7 +62,6 @@ int main(int argc, char ** argv)
         {
             DB::WriteBufferFromVector wb(formatted);
         //    DB::CompressedWriteBuffer wb2(wb1);
-        //    DB::AsynchronousWriteBuffer wb(wb2);
             Stopwatch watch;
 
             UInt64 tsc = rdtsc();

From 303c389b368598a57ef85ccca1930533fd8a5c13 Mon Sep 17 00:00:00 2001
From: keenwolf 
Date: Sun, 28 Feb 2021 18:11:54 +0800
Subject: [PATCH 465/510] change the time_t to unsigned to handle properly the
 start date of 1970-01-01

---
 base/common/DateLUTImpl.h | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/base/common/DateLUTImpl.h b/base/common/DateLUTImpl.h
index 8991f69d3f3..471169ebcec 100644
--- a/base/common/DateLUTImpl.h
+++ b/base/common/DateLUTImpl.h
@@ -320,7 +320,7 @@ public:
         /// To consider the DST changing situation within this day.
         /// also make the special timezones with no whole hour offset such as 'Australia/Lord_Howe' been taken into account
         DayNum index = findIndex(t);
-        time_t res = t - lut[index].date;
+        UInt32 res = t - lut[index].date;
         if (lut[index].amount_of_offset_change != 0 && t >= lut[index].date + lut[index].time_at_offset_change)
             res += lut[index].amount_of_offset_change;
 

From eb0387c5a960d029c063bc3fb7992011f6877ca3 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin 
Date: Sun, 28 Feb 2021 10:18:49 +0300
Subject: [PATCH 466/510] Fix abnormal server termination for nested writers

Writers with nested writer can call next() from the dtor for nested
writer and this will cause exception again, so the buffer position
should be updated on exceptions.

Found by stress test (thread) here [1] and here [2]:

    2021.02.27 19:27:53.498977 [ 302 ] {}  BaseDaemon: (version 21.3.1.6130, build id: 2DAEC5DEBF03C5A1C3BF66B7779C886F16239345) (from thread 1614) Terminate called for uncaught exception:
    Code: 24, e.displayText() = DB::Exception: Cannot write to ostream at offset 2097498, Stack trace (when copying this message, always include the lines below):

    0. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/exception:0: Poco::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int) @ 0x15bef2ab in /usr/bin/clickhouse
    1. ./obj-x86_64-linux-gnu/../src/Common/Exception.cpp:56: DB::Exception::Exception(std::__1::basic_string, std::__1::allocator > const&, int, bool) @ 0x8aea92e in /usr/bin/clickhouse
    2. ./obj-x86_64-linux-gnu/../src/IO/WriteBufferFromOStream.cpp:0: DB::WriteBufferFromOStream::nextImpl() @ 0x8bbbc45 in /usr/bin/clickhouse
    3. ./obj-x86_64-linux-gnu/../src/IO/BufferBase.h:39: DB::WriteBufferFromOStream::~WriteBufferFromOStream() @ 0x8bbc077 in /usr/bin/clickhouse
    4. ./obj-x86_64-linux-gnu/../src/IO/WriteBufferFromOStream.cpp:44: DB::WriteBufferFromOStream::~WriteBufferFromOStream() @ 0x8bbc18a in /usr/bin/clickhouse
    5. ./obj-x86_64-linux-gnu/../src/IO/BufferWithOwnMemory.h:137: DB::ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() @ 0x118bdc29 in /usr/bin/clickhouse
    6. ./obj-x86_64-linux-gnu/../src/IO/ZstdDeflatingWriteBuffer.cpp:32: DB::ZstdDeflatingWriteBuffer::~ZstdDeflatingWriteBuffer() @ 0x118be3ea in /usr/bin/clickhouse
    7. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:0: DB::WriteBufferFromHTTPServerResponse::finalize() @ 0x12f1dceb in /usr/bin/clickhouse
    8. ./obj-x86_64-linux-gnu/../src/Server/HTTPHandler.cpp:703: DB::HTTPHandler::trySendExceptionToClient(std::__1::basic_string, std::__1::allocator > const&, int, DB::HTTPServerRequest&, DB::HTTPServerResponse&, DB::HTTPHandler::Output&) @ 0x12e9fecc in /usr/bin/clickhouse
    9. ./obj-x86_64-linux-gnu/../contrib/libcxx/include/string:1444: DB::HTTPHandler::handleRequest(DB::HTTPServerRequest&, DB::HTTPServerResponse&) @ 0x12ea0d60 in /usr/bin/clickhouse
    10. ./obj-x86_64-linux-gnu/../src/Server/HTTP/HTTPServerConnection.cpp:0: DB::HTTPServerConnection::run() @ 0x12f16db1 in /usr/bin/clickhouse
    11. ./obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerConnection.cpp:57: Poco::Net::TCPServerConnection::start() @ 0x15b184f3 in /usr/bin/clickhouse
    12. ./obj-x86_64-linux-gnu/../contrib/poco/Net/src/TCPServerDispatcher.cpp:0: Poco::Net::TCPServerDispatcher::run() @ 0x15b18c1f in /usr/bin/clickhouse
    13. ./obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/ScopedLock.h:36: Poco::PooledThread::run() @ 0x15c7fdb2 in /usr/bin/clickhouse
    14. ./obj-x86_64-linux-gnu/../contrib/poco/Foundation/src/Thread.cpp:56: Poco::(anonymous namespace)::RunnableHolder::run() @ 0x15c7e350 in /usr/bin/clickhouse
    15. ./obj-x86_64-linux-gnu/../contrib/poco/Foundation/include/Poco/SharedPtr.h:277: Poco::ThreadImpl::runnableEntry(void*) @ 0x15c7cb58 in /usr/bin/clickhouse
    16. __tsan_thread_start_func @ 0x8a04ced in /usr/bin/clickhouse
    17. start_thread @ 0x9609 in /usr/lib/x86_64-linux-gnu/libpthread-2.31.so
    18. __clone @ 0x122293 in /usr/lib/x86_64-linux-gnu/libc-2.31.so
     (version 21.3.1.6130)

  [1]: https://clickhouse-test-reports.s3.yandex.net/21279/4f61ef3099f42f17b496a0b0424773978d9a32dc/stress_test_(thread).html#fail1
  [2]: https://clickhouse-test-reports.s3.yandex.net/21292/ae9fea1d0af118a8f87b224d194d61da1567188b/stress_test_(thread).html#fail1

v2: https://clickhouse-test-reports.s3.yandex.net/21305/e969daa6e86c5e09cfef08cfde19712982b64e59/stress_test_(thread).html#fail1
---
 src/IO/BrotliWriteBuffer.cpp                  | 65 +++++++++++++------
 src/IO/BrotliWriteBuffer.h                    |  5 +-
 src/IO/LZMADeflatingWriteBuffer.cpp           | 59 ++++++++++++-----
 src/IO/LZMADeflatingWriteBuffer.h             |  5 +-
 src/IO/ZlibDeflatingWriteBuffer.cpp           | 45 ++++++++++---
 src/IO/ZlibDeflatingWriteBuffer.h             | 11 ++--
 src/IO/ZstdDeflatingWriteBuffer.cpp           | 50 ++++++++++----
 src/IO/ZstdDeflatingWriteBuffer.h             | 13 ++--
 src/IO/tests/lzma_buffers.cpp                 |  2 +-
 src/IO/tests/zlib_buffers.cpp                 |  2 +-
 src/IO/tests/zstd_buffers.cpp                 |  2 +-
 .../WriteBufferFromHTTPServerResponse.cpp     |  2 +
 12 files changed, 187 insertions(+), 74 deletions(-)

diff --git a/src/IO/BrotliWriteBuffer.cpp b/src/IO/BrotliWriteBuffer.cpp
index e562cc70e61..e87eeb1a2be 100644
--- a/src/IO/BrotliWriteBuffer.cpp
+++ b/src/IO/BrotliWriteBuffer.cpp
@@ -64,29 +64,38 @@ void BrotliWriteBuffer::nextImpl()
     in_data = reinterpret_cast(working_buffer.begin());
     in_available = offset();
 
-    do
+    try
     {
-        out->nextIfAtEnd();
-        out_data = reinterpret_cast(out->position());
-        out_capacity = out->buffer().end() - out->position();
-
-        int result = BrotliEncoderCompressStream(
-                brotli->state,
-                in_available ? BROTLI_OPERATION_PROCESS : BROTLI_OPERATION_FINISH,
-                &in_available,
-                &in_data,
-                &out_capacity,
-                &out_data,
-                nullptr);
-
-        out->position() = out->buffer().end() - out_capacity;
-
-        if (result == 0)
+        do
         {
-            throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
+            out->nextIfAtEnd();
+            out_data = reinterpret_cast(out->position());
+            out_capacity = out->buffer().end() - out->position();
+
+            int result = BrotliEncoderCompressStream(
+                    brotli->state,
+                    in_available ? BROTLI_OPERATION_PROCESS : BROTLI_OPERATION_FINISH,
+                    &in_available,
+                    &in_data,
+                    &out_capacity,
+                    &out_data,
+                    nullptr);
+
+            out->position() = out->buffer().end() - out_capacity;
+
+            if (result == 0)
+            {
+                throw Exception("brotli compress failed", ErrorCodes::BROTLI_WRITE_FAILED);
+            }
         }
+        while (in_available > 0);
+    }
+    catch (...)
+    {
+        /// Do not try to write next time after exception.
+        out->position() = out->buffer().begin();
+        throw;
     }
-    while (in_available > 0);
 }
 
 void BrotliWriteBuffer::finish()
@@ -94,6 +103,23 @@ void BrotliWriteBuffer::finish()
     if (finished)
         return;
 
+    try
+    {
+        finishImpl();
+        out->next();
+        finished = true;
+    }
+    catch (...)
+    {
+        /// Do not try to flush next time after exception.
+        out->position() = out->buffer().begin();
+        finished = true;
+        throw;
+    }
+}
+
+void BrotliWriteBuffer::finishImpl()
+{
     next();
 
     while (true)
@@ -115,7 +141,6 @@ void BrotliWriteBuffer::finish()
 
         if (BrotliEncoderIsFinished(brotli->state))
         {
-            finished = true;
             return;
         }
 
diff --git a/src/IO/BrotliWriteBuffer.h b/src/IO/BrotliWriteBuffer.h
index 5a294354f49..26788bc6795 100644
--- a/src/IO/BrotliWriteBuffer.h
+++ b/src/IO/BrotliWriteBuffer.h
@@ -18,11 +18,14 @@ public:
 
     ~BrotliWriteBuffer() override;
 
-    void finish();
+    void finalize() override { finish(); }
 
 private:
     void nextImpl() override;
 
+    void finish();
+    void finishImpl();
+
     class BrotliStateWrapper;
     std::unique_ptr brotli;
 
diff --git a/src/IO/LZMADeflatingWriteBuffer.cpp b/src/IO/LZMADeflatingWriteBuffer.cpp
index 5803bc1e9f1..96f1d34b01b 100644
--- a/src/IO/LZMADeflatingWriteBuffer.cpp
+++ b/src/IO/LZMADeflatingWriteBuffer.cpp
@@ -64,27 +64,36 @@ void LZMADeflatingWriteBuffer::nextImpl()
     lstr.next_in = reinterpret_cast(working_buffer.begin());
     lstr.avail_in = offset();
 
-    lzma_action action = LZMA_RUN;
-    do
+    try
     {
-        out->nextIfAtEnd();
-        lstr.next_out = reinterpret_cast(out->position());
-        lstr.avail_out = out->buffer().end() - out->position();
+        lzma_action action = LZMA_RUN;
+        do
+        {
+            out->nextIfAtEnd();
+            lstr.next_out = reinterpret_cast(out->position());
+            lstr.avail_out = out->buffer().end() - out->position();
 
-        lzma_ret ret = lzma_code(&lstr, action);
-        out->position() = out->buffer().end() - lstr.avail_out;
+            lzma_ret ret = lzma_code(&lstr, action);
+            out->position() = out->buffer().end() - lstr.avail_out;
 
-        if (ret == LZMA_STREAM_END)
-            return;
+            if (ret == LZMA_STREAM_END)
+                return;
 
-        if (ret != LZMA_OK)
-            throw Exception(
-                ErrorCodes::LZMA_STREAM_ENCODER_FAILED,
-                "lzma stream encoding failed: error code: {}; lzma_version: {}",
-                ret,
-                LZMA_VERSION_STRING);
+            if (ret != LZMA_OK)
+                throw Exception(
+                    ErrorCodes::LZMA_STREAM_ENCODER_FAILED,
+                    "lzma stream encoding failed: error code: {}; lzma_version: {}",
+                    ret,
+                    LZMA_VERSION_STRING);
 
-    } while (lstr.avail_in > 0 || lstr.avail_out == 0);
+        } while (lstr.avail_in > 0 || lstr.avail_out == 0);
+    }
+    catch (...)
+    {
+        /// Do not try to write next time after exception.
+        out->position() = out->buffer().begin();
+        throw;
+    }
 }
 
 
@@ -93,6 +102,23 @@ void LZMADeflatingWriteBuffer::finish()
     if (finished)
         return;
 
+    try
+    {
+        finishImpl();
+        out->next();
+        finished = true;
+    }
+    catch (...)
+    {
+        /// Do not try to flush next time after exception.
+        out->position() = out->buffer().begin();
+        finished = true;
+        throw;
+    }
+}
+
+void LZMADeflatingWriteBuffer::finishImpl()
+{
     next();
 
     do
@@ -106,7 +132,6 @@ void LZMADeflatingWriteBuffer::finish()
 
         if (ret == LZMA_STREAM_END)
         {
-            finished = true;
             return;
         }
 
diff --git a/src/IO/LZMADeflatingWriteBuffer.h b/src/IO/LZMADeflatingWriteBuffer.h
index efa4532d372..98eb1732e76 100644
--- a/src/IO/LZMADeflatingWriteBuffer.h
+++ b/src/IO/LZMADeflatingWriteBuffer.h
@@ -24,13 +24,16 @@ public:
         char * existing_memory = nullptr,
         size_t alignment = 0);
 
-    void finish();
+    void finalize() override { finish(); }
 
     ~LZMADeflatingWriteBuffer() override;
 
 private:
     void nextImpl() override;
 
+    void finish();
+    void finishImpl();
+
     std::unique_ptr out;
     lzma_stream lstr;
     bool finished = false;
diff --git a/src/IO/ZlibDeflatingWriteBuffer.cpp b/src/IO/ZlibDeflatingWriteBuffer.cpp
index 4b838ac6d0a..5da82b52279 100644
--- a/src/IO/ZlibDeflatingWriteBuffer.cpp
+++ b/src/IO/ZlibDeflatingWriteBuffer.cpp
@@ -75,19 +75,28 @@ void ZlibDeflatingWriteBuffer::nextImpl()
     zstr.next_in = reinterpret_cast(working_buffer.begin());
     zstr.avail_in = offset();
 
-    do
+    try
     {
-        out->nextIfAtEnd();
-        zstr.next_out = reinterpret_cast(out->position());
-        zstr.avail_out = out->buffer().end() - out->position();
+        do
+        {
+            out->nextIfAtEnd();
+            zstr.next_out = reinterpret_cast(out->position());
+            zstr.avail_out = out->buffer().end() - out->position();
 
-        int rc = deflate(&zstr, Z_NO_FLUSH);
-        out->position() = out->buffer().end() - zstr.avail_out;
+            int rc = deflate(&zstr, Z_NO_FLUSH);
+            out->position() = out->buffer().end() - zstr.avail_out;
 
-        if (rc != Z_OK)
-            throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
+            if (rc != Z_OK)
+                throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
+        }
+        while (zstr.avail_in > 0 || zstr.avail_out == 0);
+    }
+    catch (...)
+    {
+        /// Do not try to write next time after exception.
+        out->position() = out->buffer().begin();
+        throw;
     }
-    while (zstr.avail_in > 0 || zstr.avail_out == 0);
 }
 
 void ZlibDeflatingWriteBuffer::finish()
@@ -95,6 +104,23 @@ void ZlibDeflatingWriteBuffer::finish()
     if (finished)
         return;
 
+    try
+    {
+        finishImpl();
+        out->next();
+        finished = true;
+    }
+    catch (...)
+    {
+        /// Do not try to flush next time after exception.
+        out->position() = out->buffer().begin();
+        finished = true;
+        throw;
+    }
+}
+
+void ZlibDeflatingWriteBuffer::finishImpl()
+{
     next();
 
     /// https://github.com/zlib-ng/zlib-ng/issues/494
@@ -123,7 +149,6 @@ void ZlibDeflatingWriteBuffer::finish()
 
         if (rc == Z_STREAM_END)
         {
-            finished = true;
             return;
         }
 
diff --git a/src/IO/ZlibDeflatingWriteBuffer.h b/src/IO/ZlibDeflatingWriteBuffer.h
index f9df8f8157b..6f623f55f56 100644
--- a/src/IO/ZlibDeflatingWriteBuffer.h
+++ b/src/IO/ZlibDeflatingWriteBuffer.h
@@ -22,16 +22,19 @@ public:
             char * existing_memory = nullptr,
             size_t alignment = 0);
 
-    /// Flush all pending data and write zlib footer to the underlying buffer.
-    /// After the first call to this function, subsequent calls will have no effect and
-    /// an attempt to write to this buffer will result in exception.
-    void finish();
+    void finalize() override { finish(); }
 
     ~ZlibDeflatingWriteBuffer() override;
 
 private:
     void nextImpl() override;
 
+    void finishImpl();
+    /// Flush all pending data and write zlib footer to the underlying buffer.
+    /// After the first call to this function, subsequent calls will have no effect and
+    /// an attempt to write to this buffer will result in exception.
+    void finish();
+
     std::unique_ptr out;
     z_stream zstr;
     bool finished = false;
diff --git a/src/IO/ZstdDeflatingWriteBuffer.cpp b/src/IO/ZstdDeflatingWriteBuffer.cpp
index 9b79d5ae513..27694797db6 100644
--- a/src/IO/ZstdDeflatingWriteBuffer.cpp
+++ b/src/IO/ZstdDeflatingWriteBuffer.cpp
@@ -61,28 +61,53 @@ void ZstdDeflatingWriteBuffer::nextImpl()
     input.size = offset();
     input.pos = 0;
 
-    bool finished = false;
-    do
+    try
     {
-        out->nextIfAtEnd();
+        bool ended = false;
+        do
+        {
+            out->nextIfAtEnd();
 
-        output.dst = reinterpret_cast(out->buffer().begin());
-        output.size = out->buffer().size();
-        output.pos = out->offset();
+            output.dst = reinterpret_cast(out->buffer().begin());
+            output.size = out->buffer().size();
+            output.pos = out->offset();
 
 
-        ZSTD_compressStream2(cctx, &output, &input, mode);
-        out->position() = out->buffer().begin() + output.pos;
-        finished = (input.pos == input.size);
-    } while (!finished);
-
+            ZSTD_compressStream2(cctx, &output, &input, mode);
+            out->position() = out->buffer().begin() + output.pos;
+            ended = (input.pos == input.size);
+        } while (!ended);
+    }
+    catch (...)
+    {
+        /// Do not try to write next time after exception.
+        out->position() = out->buffer().begin();
+        throw;
+    }
 }
 
 void ZstdDeflatingWriteBuffer::finish()
 {
-    if (flushed)
+    if (finished)
         return;
 
+    try
+    {
+        finishImpl();
+        out->next();
+        finished = true;
+    }
+    catch (...)
+    {
+        /// Do not try to flush next time after exception.
+        out->position() = out->buffer().begin();
+        finished = true;
+        throw;
+    }
+}
+
+void ZstdDeflatingWriteBuffer::finishImpl()
+{
     next();
 
     out->nextIfAtEnd();
@@ -99,7 +124,6 @@ void ZstdDeflatingWriteBuffer::finish()
     if (ZSTD_isError(remaining))
         throw Exception(ErrorCodes::ZSTD_ENCODER_FAILED, "zstd stream encoder end failed: zstd version: {}", ZSTD_VERSION_STRING);
     out->position() = out->buffer().begin() + output.pos;
-    flushed = true;
 }
 
 }
diff --git a/src/IO/ZstdDeflatingWriteBuffer.h b/src/IO/ZstdDeflatingWriteBuffer.h
index 2c7dd38dbb0..b4ecc44d6f4 100644
--- a/src/IO/ZstdDeflatingWriteBuffer.h
+++ b/src/IO/ZstdDeflatingWriteBuffer.h
@@ -20,21 +20,24 @@ public:
         char * existing_memory = nullptr,
         size_t alignment = 0);
 
-    /// Flush all pending data and write zstd footer to the underlying buffer.
-    /// After the first call to this function, subsequent calls will have no effect and
-    /// an attempt to write to this buffer will result in exception.
-    void finish();
+    void finalize() override { finish(); }
 
     ~ZstdDeflatingWriteBuffer() override;
 
 private:
     void nextImpl() override;
 
+    /// Flush all pending data and write zstd footer to the underlying buffer.
+    /// After the first call to this function, subsequent calls will have no effect and
+    /// an attempt to write to this buffer will result in exception.
+    void finish();
+    void finishImpl();
+
     std::unique_ptr out;
     ZSTD_CCtx * cctx;
     ZSTD_inBuffer input;
     ZSTD_outBuffer output;
-    bool flushed = false;
+    bool finished = false;
 };
 
 }
diff --git a/src/IO/tests/lzma_buffers.cpp b/src/IO/tests/lzma_buffers.cpp
index 7eb6bf8b81c..ff3d518bfab 100644
--- a/src/IO/tests/lzma_buffers.cpp
+++ b/src/IO/tests/lzma_buffers.cpp
@@ -28,7 +28,7 @@ try
             DB::writeIntText(i, lzma_buf);
             DB::writeChar('\t', lzma_buf);
         }
-        lzma_buf.finish();
+        lzma_buf.finalize();
 
         stopwatch.stop();
 
diff --git a/src/IO/tests/zlib_buffers.cpp b/src/IO/tests/zlib_buffers.cpp
index 3428d5e995a..2068a3e6668 100644
--- a/src/IO/tests/zlib_buffers.cpp
+++ b/src/IO/tests/zlib_buffers.cpp
@@ -30,7 +30,7 @@ try
             DB::writeIntText(i, deflating_buf);
             DB::writeChar('\t', deflating_buf);
         }
-        deflating_buf.finish();
+        deflating_buf.finalize();
 
         stopwatch.stop();
         std::cout << "Writing done. Elapsed: " << stopwatch.elapsedSeconds() << " s."
diff --git a/src/IO/tests/zstd_buffers.cpp b/src/IO/tests/zstd_buffers.cpp
index f269c0b22fd..533229f4878 100644
--- a/src/IO/tests/zstd_buffers.cpp
+++ b/src/IO/tests/zstd_buffers.cpp
@@ -30,7 +30,7 @@ try
             DB::writeIntText(i, zstd_buf);
             DB::writeChar('\t', zstd_buf);
         }
-        zstd_buf.finish();
+        zstd_buf.finalize();
 
         stopwatch.stop();
 
diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
index 81f8cc30468..355af038da9 100644
--- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
+++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
@@ -171,6 +171,8 @@ void WriteBufferFromHTTPServerResponse::finalize()
     try
     {
         next();
+        if (out)
+            out->finalize();
         out.reset();
     }
     catch (...)

From 2df33be7c22307ecbe4b85259c3cca05a23840b8 Mon Sep 17 00:00:00 2001
From: fuwhu 
Date: Sun, 28 Feb 2021 23:17:31 +0800
Subject: [PATCH 467/510] Remove unused code in MergeTreeWriteAheadLog::restore

---
 src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp | 1 -
 1 file changed, 1 deletion(-)

diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp
index e726f0ffd51..7ddc8d93b03 100644
--- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp
+++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp
@@ -147,7 +147,6 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor
             }
             else if (action_type == ActionType::ADD_PART)
             {
-                auto part_disk = storage.reserveSpace(0)->getDisk();
                 auto single_disk_volume = std::make_shared("volume_" + part_name, disk, 0);
 
                 part = storage.createPart(

From 9b633ac5646a4e9e6bc31a8f1f43b5b0ee5429aa Mon Sep 17 00:00:00 2001
From: alexey-milovidov 
Date: Sun, 28 Feb 2021 23:27:01 +0300
Subject: [PATCH 468/510] Update ontime.md

---
 .../getting-started/example-datasets/ontime.md   | 16 +++++-----------
 1 file changed, 5 insertions(+), 11 deletions(-)

diff --git a/docs/en/getting-started/example-datasets/ontime.md b/docs/en/getting-started/example-datasets/ontime.md
index 6e46cddba52..83673cdceb6 100644
--- a/docs/en/getting-started/example-datasets/ontime.md
+++ b/docs/en/getting-started/example-datasets/ontime.md
@@ -15,17 +15,9 @@ This dataset can be obtained in two ways:
 Downloading data:
 
 ``` bash
-for s in `seq 1987 2018`
-do
-for m in `seq 1 12`
-do
-wget https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_${s}_${m}.zip
-done
-done
+echo https://transtats.bts.gov/PREZIP/On_Time_Reporting_Carrier_On_Time_Performance_1987_present_{1987..2021}_{1..12}.zip | xargs -P10 wget --no-check-certificate --continue
 ```
 
-(from https://github.com/Percona-Lab/ontime-airline-performance/blob/master/download.sh )
-
 Creating a table:
 
 ``` sql
@@ -145,12 +137,14 @@ ORDER BY (Carrier, FlightDate)
 SETTINGS index_granularity = 8192;
 ```
 
-Loading data:
+Loading data with multiple threads:
 
 ``` bash
-$ for i in *.zip; do echo $i; unzip -cq $i '*.csv' | sed 's/\.00//g' | clickhouse-client --input_format_with_names_use_header=0 --host=example-perftest01j --query="INSERT INTO ontime FORMAT CSVWithNames"; done
+ls -1 *.zip | xargs -I{} -P $(nproc) bash -c "echo {}; unzip -cq {} '*.csv' | sed 's/\.00//g' | clickhouse-client --input_format_with_names_use_header=0 --query='INSERT INTO ontime FORMAT CSVWithNames'"
 ```
 
+(if you will have memory shortage or other issues on your server, remove the `-P $(nproc)` part)
+
 ## Download of Prepared Partitions {#download-of-prepared-partitions}
 
 ``` bash

From 11f2a271a20664eab916af4dd08f13c7b765f04c Mon Sep 17 00:00:00 2001
From: alesapin 
Date: Mon, 1 Mar 2021 10:40:00 +0300
Subject: [PATCH 469/510] Remove useless unit test

---
 src/Coordination/tests/gtest_for_build.cpp | 118 ---------------------
 1 file changed, 118 deletions(-)

diff --git a/src/Coordination/tests/gtest_for_build.cpp b/src/Coordination/tests/gtest_for_build.cpp
index c064fcdbef4..37517808ef0 100644
--- a/src/Coordination/tests/gtest_for_build.cpp
+++ b/src/Coordination/tests/gtest_for_build.cpp
@@ -195,124 +195,6 @@ TEST(CoordinationTest, TestSummingRaft1)
     s1.launcher.shutdown(5);
 }
 
-TEST(CoordinationTest, TestSummingRaft3)
-{
-    ChangelogDirTest test1("./logs1");
-    SummingRaftServer s1(1, "localhost", 44444, "./logs1");
-    ChangelogDirTest test2("./logs2");
-    SummingRaftServer s2(2, "localhost", 44445, "./logs2");
-    ChangelogDirTest test3("./logs3");
-    SummingRaftServer s3(3, "localhost", 44446, "./logs3");
-
-    nuraft::srv_config first_config(1, 0, "localhost:44444", "", false, 0);
-    auto ret1 = s2.raft_instance->add_srv(first_config);
-    while (!ret1->get_accepted())
-    {
-
-        std::cout << "failed to add server: "
-                  << ret1->get_result_str() << std::endl;
-
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-        ret1 = s2.raft_instance->add_srv(first_config);
-    }
-
-    while (s1.raft_instance->get_leader() != 2)
-    {
-        std::cout << "Waiting s1 to join to s2 quorum\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    nuraft::srv_config third_config(3, 0, "localhost:44446", "", false, 0);
-    auto ret3 = s2.raft_instance->add_srv(third_config);
-    if (!ret3->get_accepted())
-    {
-        std::cout << "failed to add server: "
-                  << ret3->get_result_str() << std::endl;
-
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-        ret3 = s2.raft_instance->add_srv(third_config);
-    }
-
-    while (s3.raft_instance->get_leader() != 2)
-    {
-        std::cout << "Waiting s3 to join to s2 quorum\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    /// S2 is leader
-    EXPECT_EQ(s1.raft_instance->get_leader(), 2);
-    EXPECT_EQ(s2.raft_instance->get_leader(), 2);
-    EXPECT_EQ(s3.raft_instance->get_leader(), 2);
-
-    std::cerr << "Starting to add entries\n";
-    auto entry = getBuffer(1);
-    auto ret = s2.raft_instance->append_entries({entry});
-    while (!ret->get_accepted() || ret->get_result_code() != nuraft::cmd_result_code::OK)
-    {
-        std::cerr <<  ret->get_accepted() << "failed to replicate: entry 1" << ret->get_result_code() << std::endl;
-        ret = s2.raft_instance->append_entries({entry});
-    }
-
-    while (s1.state_machine->getValue() != 1)
-    {
-        std::cout << "Waiting s1 to apply entry\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    while (s2.state_machine->getValue() != 1)
-    {
-        std::cout << "Waiting s2 to apply entry\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    while (s3.state_machine->getValue() != 1)
-    {
-        std::cout << "Waiting s3 to apply entry\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    EXPECT_EQ(s1.state_machine->getValue(), 1);
-    EXPECT_EQ(s2.state_machine->getValue(), 1);
-    EXPECT_EQ(s3.state_machine->getValue(), 1);
-
-    auto non_leader_entry = getBuffer(3);
-    auto ret_non_leader1 = s1.raft_instance->append_entries({non_leader_entry});
-
-    EXPECT_FALSE(ret_non_leader1->get_accepted());
-
-    auto ret_non_leader3 = s3.raft_instance->append_entries({non_leader_entry});
-
-    EXPECT_FALSE(ret_non_leader3->get_accepted());
-
-    auto leader_entry = getBuffer(77);
-    auto ret_leader = s2.raft_instance->append_entries({leader_entry});
-    while (!ret_leader->get_accepted() || ret_leader->get_result_code() != nuraft::cmd_result_code::OK)
-    {
-        std::cerr << "failed to replicate: entry 78" << ret_leader->get_result_code() << std::endl;
-        ret_leader = s2.raft_instance->append_entries({leader_entry});
-    }
-
-    while (s1.state_machine->getValue() != 78)
-    {
-        std::cout << "Waiting s1 to apply entry\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    while (s3.state_machine->getValue() != 78)
-    {
-        std::cout << "Waiting s3 to apply entry\n";
-        std::this_thread::sleep_for(std::chrono::milliseconds(100));
-    }
-
-    EXPECT_EQ(s1.state_machine->getValue(), 78);
-    EXPECT_EQ(s2.state_machine->getValue(), 78);
-    EXPECT_EQ(s3.state_machine->getValue(), 78);
-
-    s1.launcher.shutdown(5);
-    s2.launcher.shutdown(5);
-    s3.launcher.shutdown(5);
-}
-
 nuraft::ptr getZooKeeperLogEntry(int64_t session_id, const Coordination::ZooKeeperRequestPtr & request)
 {
     DB::WriteBufferFromNuraftBuffer buf;

From 9c8afbeb5394f0b38aed5451ef1d0bba0f98cc5e Mon Sep 17 00:00:00 2001
From: alesapin 
Date: Mon, 1 Mar 2021 12:59:19 +0300
Subject: [PATCH 470/510] Fix alter modify query for partition key and other
 metadata fields

---
 src/Storages/MergeTree/MergeTreeData.cpp      | 13 ++--
 src/Storages/StorageReplicatedMergeTree.cpp   | 54 ++++++++++++----
 ...ter_partition_key_enum_zookeeper.reference |  6 ++
 ...747_alter_partition_key_enum_zookeeper.sql | 63 +++++++++++++++++++
 4 files changed, 119 insertions(+), 17 deletions(-)
 create mode 100644 tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference
 create mode 100644 tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql

diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp
index d9e24581c0c..0cedae9a2f4 100644
--- a/src/Storages/MergeTree/MergeTreeData.cpp
+++ b/src/Storages/MergeTree/MergeTreeData.cpp
@@ -165,7 +165,7 @@ MergeTreeData::MergeTreeData(
     {
         try
         {
-            checkPartitionKeyAndInitMinMax(metadata_.partition_key);
+            setProperties(metadata_, metadata_, attach);
             if (minmax_idx_date_column_pos == -1)
                 throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD);
         }
@@ -179,12 +179,10 @@ MergeTreeData::MergeTreeData(
     else
     {
         is_custom_partitioned = true;
-        checkPartitionKeyAndInitMinMax(metadata_.partition_key);
+        setProperties(metadata_, metadata_, attach);
         min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING;
     }
 
-    setProperties(metadata_, metadata_, attach);
-
     /// NOTE: using the same columns list as is read when performing actual merges.
     merging_params.check(metadata_);
 
@@ -398,6 +396,7 @@ void MergeTreeData::checkProperties(
 void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach)
 {
     checkProperties(new_metadata, old_metadata, attach);
+    checkPartitionKeyAndInitMinMax(new_metadata.partition_key);
     setInMemoryMetadata(new_metadata);
 }
 
@@ -440,6 +439,12 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa
 
     checkKeyExpression(*new_partition_key.expression, new_partition_key.sample_block, "Partition", allow_nullable_key);
 
+    /// Reset filled fields
+    minmax_idx_columns.clear();
+    minmax_idx_column_types.clear();
+    minmax_idx_date_column_pos = -1;
+    minmax_idx_time_column_pos = -1;
+
     /// Add all columns used in the partition key to the min-max index.
     const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes();
     minmax_idx_expr = std::make_shared(std::make_shared(minmax_idx_columns_with_types));
diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp
index 1a99f320365..c8b91f9533a 100644
--- a/src/Storages/StorageReplicatedMergeTree.cpp
+++ b/src/Storages/StorageReplicatedMergeTree.cpp
@@ -897,21 +897,8 @@ void StorageReplicatedMergeTree::setTableStructure(
     StorageInMemoryMetadata old_metadata = getInMemoryMetadata();
 
     if (new_columns != new_metadata.columns)
-    {
         new_metadata.columns = new_columns;
 
-        new_metadata.column_ttls_by_name.clear();
-        for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs())
-        {
-            auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, global_context, new_metadata.primary_key);
-            new_metadata.column_ttls_by_name[name] = new_ttl_entry;
-        }
-
-        /// The type of partition key expression may change
-        if (new_metadata.partition_key.definition_ast != nullptr)
-            new_metadata.partition_key.recalculateWithNewColumns(new_metadata.columns, global_context);
-    }
-
     if (!metadata_diff.empty())
     {
         auto parse_key_expr = [] (const String & key_expr)
@@ -977,6 +964,47 @@ void StorageReplicatedMergeTree::setTableStructure(
         }
     }
 
+    /// Changes in columns may affect following metadata fields
+    if (new_metadata.columns != old_metadata.columns)
+    {
+        new_metadata.column_ttls_by_name.clear();
+        for (const auto & [name, ast] : new_metadata.columns.getColumnTTLs())
+        {
+            auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, new_metadata.columns, global_context, new_metadata.primary_key);
+            new_metadata.column_ttls_by_name[name] = new_ttl_entry;
+        }
+
+        if (new_metadata.partition_key.definition_ast != nullptr)
+            new_metadata.partition_key.recalculateWithNewColumns(new_metadata.columns, global_context);
+
+        if (!metadata_diff.sorting_key_changed) /// otherwise already updated
+            new_metadata.sorting_key.recalculateWithNewColumns(new_metadata.columns, global_context);
+
+        /// Primary key is special, it exists even if not defined
+        if (new_metadata.primary_key.definition_ast != nullptr)
+        {
+            new_metadata.primary_key.recalculateWithNewColumns(new_metadata.columns, global_context);
+        }
+        else
+        {
+            new_metadata.primary_key = KeyDescription::getKeyFromAST(new_metadata.sorting_key.definition_ast, new_metadata.columns, global_context);
+            new_metadata.primary_key.definition_ast = nullptr;
+        }
+
+        if (!metadata_diff.sampling_expression_changed && new_metadata.sampling_key.definition_ast != nullptr)
+            new_metadata.sampling_key.recalculateWithNewColumns(new_metadata.columns, global_context);
+
+        if (!metadata_diff.skip_indices_changed) /// otherwise already updated
+        {
+            for (auto & index : new_metadata.secondary_indices)
+                index.recalculateWithNewColumns(new_metadata.columns, global_context);
+        }
+
+        if (!metadata_diff.ttl_table_changed && new_metadata.table_ttl.definition_ast != nullptr)
+            new_metadata.table_ttl = TTLTableDescription::getTTLForTableFromAST(
+                new_metadata.table_ttl.definition_ast, new_metadata.columns, global_context, new_metadata.primary_key);
+    }
+
     /// Even if the primary/sorting/partition keys didn't change we must reinitialize it
     /// because primary/partition key column types might have changed.
     checkTTLExpressions(new_metadata, old_metadata);
diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference
new file mode 100644
index 00000000000..02359f0f98b
--- /dev/null
+++ b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.reference
@@ -0,0 +1,6 @@
+IU	lada	2101	1970-04-19 15:00:00
+PS	jeep	Grand Cherokee	2005-10-03 15:00:00
+PS	jeep	Grand Cherokee	2005-10-03 15:00:00
+IU	lada	2101	1970-04-19 15:00:00
+PS	jeep	Grand Cherokee	2005-10-03 15:00:00
+PS	jeep	Grand Cherokee	2005-10-03 15:00:00
diff --git a/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql
new file mode 100644
index 00000000000..759c8ba3a0b
--- /dev/null
+++ b/tests/queries/0_stateless/01747_alter_partition_key_enum_zookeeper.sql
@@ -0,0 +1,63 @@
+DROP TABLE IF EXISTS report;
+
+CREATE TABLE report
+(
+    `product` Enum8('IU' = 1, 'WS' = 2),
+    `machine` String,
+    `branch` String,
+    `generated_time` DateTime
+)
+ENGINE = MergeTree
+PARTITION BY (product, toYYYYMM(generated_time))
+ORDER BY (product, machine, branch, generated_time);
+
+INSERT INTO report VALUES ('IU', 'lada', '2101', toDateTime('1970-04-19 15:00:00'));
+
+SELECT * FROM report  WHERE product = 'IU';
+
+ALTER TABLE report MODIFY COLUMN product Enum8('IU' = 1, 'WS' = 2, 'PS' = 3);
+
+SELECT * FROM report WHERE product = 'PS';
+
+INSERT INTO report VALUES ('PS', 'jeep', 'Grand Cherokee', toDateTime('2005-10-03 15:00:00'));
+
+SELECT * FROM report WHERE product = 'PS';
+
+DETACH TABLE report;
+ATTACH TABLE report;
+
+SELECT * FROM report WHERE product = 'PS';
+
+DROP TABLE IF EXISTS report;
+
+DROP TABLE IF EXISTS replicated_report;
+
+CREATE TABLE replicated_report
+(
+    `product` Enum8('IU' = 1, 'WS' = 2),
+    `machine` String,
+    `branch` String,
+    `generated_time` DateTime
+)
+ENGINE = ReplicatedMergeTree('/clickhouse/01747_alter_partition_key/t', '1')
+PARTITION BY (product, toYYYYMM(generated_time))
+ORDER BY (product, machine, branch, generated_time);
+
+INSERT INTO replicated_report VALUES ('IU', 'lada', '2101', toDateTime('1970-04-19 15:00:00'));
+
+SELECT * FROM replicated_report  WHERE product = 'IU';
+
+ALTER TABLE replicated_report MODIFY COLUMN product Enum8('IU' = 1, 'WS' = 2, 'PS' = 3) SETTINGS replication_alter_partitions_sync=2;
+
+SELECT * FROM replicated_report WHERE product = 'PS';
+
+INSERT INTO replicated_report VALUES ('PS', 'jeep', 'Grand Cherokee', toDateTime('2005-10-03 15:00:00'));
+
+SELECT * FROM replicated_report WHERE product = 'PS';
+
+DETACH TABLE replicated_report;
+ATTACH TABLE replicated_report;
+
+SELECT * FROM replicated_report WHERE product = 'PS';
+
+DROP TABLE IF EXISTS replicated_report;

From caaa0c5e8649dc13b65c7cfa9d32a740c6c701bf Mon Sep 17 00:00:00 2001
From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com>
Date: Mon, 1 Mar 2021 13:25:20 +0300
Subject: [PATCH 471/510] Restart tests.

---
 src/Storages/System/StorageSystemPartsBase.h | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)

diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h
index 4c3ca78b5bb..3f63d75e2b6 100644
--- a/src/Storages/System/StorageSystemPartsBase.h
+++ b/src/Storages/System/StorageSystemPartsBase.h
@@ -75,8 +75,7 @@ protected:
     StorageSystemPartsBase(const StorageID & table_id_, NamesAndTypesList && columns_);
 
     virtual void
-    processNextStorage(MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column)
-        = 0;
+    processNextStorage(MutableColumns & columns, std::vector & columns_mask, const StoragesInfo & info, bool has_state_column) = 0;
 };
 
 }

From 366fba4b0493904cf9a68f158b437cf6ac5a7114 Mon Sep 17 00:00:00 2001
From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com>
Date: Mon, 1 Mar 2021 16:06:44 +0300
Subject: [PATCH 472/510] Update 01592_long_window_functions1.sql

---
 tests/queries/0_stateless/01592_long_window_functions1.sql | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)

diff --git a/tests/queries/0_stateless/01592_long_window_functions1.sql b/tests/queries/0_stateless/01592_long_window_functions1.sql
index c7751ab9f33..bb0f77ff60a 100644
--- a/tests/queries/0_stateless/01592_long_window_functions1.sql
+++ b/tests/queries/0_stateless/01592_long_window_functions1.sql
@@ -1,6 +1,7 @@
 drop table if exists stack;
 
 set allow_experimental_window_functions = 1;
+set max_insert_threads = 4;
 
 create table stack(item_id Int64, brand_id Int64, rack_id Int64, dt DateTime, expiration_dt DateTime, quantity UInt64)
 Engine = MergeTree 
@@ -10,7 +11,7 @@ order by (brand_id, toStartOfHour(dt));
 insert into stack 
 select number%99991, number%11, number%1111, toDateTime('2020-01-01 00:00:00')+number/100, 
    toDateTime('2020-02-01 00:00:00')+number/10, intDiv(number,100)+1
-from numbers(10000000);
+from numbers_mt(10000000);
 
 select '---- arrays ----';
 

From a4f2ee0752278198833f495aa9643e0b56ac0685 Mon Sep 17 00:00:00 2001
From: Roman Bug 
Date: Mon, 1 Mar 2021 16:51:38 +0300
Subject: [PATCH 473/510] DOCSUP-6144: Edit and translate PR to Russian
 (#19780)

Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com>
Co-authored-by: romanzhukov 
Co-authored-by: Vladimir 
---
 docs/en/operations/system-tables/index.md     |  4 +-
 .../en/sql-reference/statements/select/all.md |  6 +--
 docs/ru/operations/system-tables/index.md     | 45 +++++++++++++++----
 .../external-dicts-dict-sources.md            |  2 +-
 .../ru/sql-reference/statements/select/all.md | 22 +++++++++
 5 files changed, 65 insertions(+), 14 deletions(-)
 create mode 100644 docs/ru/sql-reference/statements/select/all.md

diff --git a/docs/en/operations/system-tables/index.md b/docs/en/operations/system-tables/index.md
index 5dc23aee686..e66f082167e 100644
--- a/docs/en/operations/system-tables/index.md
+++ b/docs/en/operations/system-tables/index.md
@@ -20,7 +20,7 @@ System tables:
 
 Most of system tables store their data in RAM. A ClickHouse server creates such system tables at the start.
 
-Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), crash_log and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a storage filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one.
+Unlike other system tables, the system log tables [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), [crash_log](../../operations/system-tables/crash-log.md) and [text_log](../../operations/system-tables/text_log.md) are served by [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) table engine and store their data in a filesystem by default. If you remove a table from a filesystem, the ClickHouse server creates the empty one again at the time of the next data writing. If system table schema changed in a new release, then ClickHouse renames the current table and creates a new one.
 
 System log tables can be customized by creating a config file with the same name as the table under `/etc/clickhouse-server/config.d/`, or setting corresponding elements in `/etc/clickhouse-server/config.xml`. Elements can be customized are:
 
@@ -33,7 +33,7 @@ System log tables can be customized by creating a config file with the same name
 
 An example:
 
-```
+```xml
 
     
         system
diff --git a/docs/en/sql-reference/statements/select/all.md b/docs/en/sql-reference/statements/select/all.md
index 5e0de4c142b..891b82c4319 100644
--- a/docs/en/sql-reference/statements/select/all.md
+++ b/docs/en/sql-reference/statements/select/all.md
@@ -4,10 +4,8 @@ toc_title: ALL
 
 # ALL Clause {#select-all}
 
-`SELECT ALL` is identical to `SELECT` without `DISTINCT`.
+If there are multiple matching rows in the table, then `ALL` returns all of them. `SELECT ALL` is identical to `SELECT` without `DISTINCT`. If both `ALL` and `DISTINCT` specified, exception will be thrown.
 
-- If `ALL` specified, ignore it.
-- If both `ALL` and `DISTINCT` specified, exception will be thrown.
 
 `ALL` can also be specified inside aggregate function with the same effect(noop), for instance:
 
@@ -19,3 +17,5 @@ equals to
 ```sql
 SELECT sum(number) FROM numbers(10);
 ```
+
+[Original article](https://clickhouse.tech/docs/en/sql-reference/statements/select/all) 
diff --git a/docs/ru/operations/system-tables/index.md b/docs/ru/operations/system-tables/index.md
index 93ea1c92068..cdea6102a81 100644
--- a/docs/ru/operations/system-tables/index.md
+++ b/docs/ru/operations/system-tables/index.md
@@ -9,25 +9,54 @@ toc_title: "\u0421\u0438\u0441\u0442\u0435\u043c\u043d\u044b\u0435\u0020\u0442\u
 
 Системные таблицы содержат информацию о:
 
--   Состоянии сервера, процессов и окружении.
--   Внутренних процессах сервера.
+-   состоянии сервера, процессов и окружении.
+-   внутренних процессах сервера.
 
 Системные таблицы:
 
--   Находятся в базе данных `system`.
--   Доступны только для чтения данных.
--   Не могут быть удалены или изменены, но их можно отсоединить.
+-   находятся в базе данных `system`.
+-   доступны только для чтения данных.
+-   не могут быть удалены или изменены, но их можно отсоединить.
 
-Системные таблицы `metric_log`, `query_log`, `query_thread_log`, `trace_log` системные таблицы хранят данные в файловой системе. Остальные системные таблицы хранят свои данные в оперативной памяти. Сервер ClickHouse создает такие системные таблицы при запуске.
+Большинство системных таблиц хранят свои данные в оперативной памяти. Сервер ClickHouse создает эти системные таблицы при старте.
+
+В отличие от других системных таблиц, таблицы с системными логами [metric_log](../../operations/system-tables/metric_log.md), [query_log](../../operations/system-tables/query_log.md), [query_thread_log](../../operations/system-tables/query_thread_log.md), [trace_log](../../operations/system-tables/trace_log.md), [part_log](../../operations/system-tables/part_log.md), [crash_log](../../operations/system-tables/crash-log.md) и [text_log](../../operations/system-tables/text_log.md) используют движок таблиц [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) и по умолчанию хранят свои данные в файловой системе. Если удалить таблицу из файловой системы, сервер ClickHouse снова создаст пустую таблицу во время следующей записи данных. Если схема системной таблицы изменилась в новом релизе, то ClickHouse переименует текущую таблицу и создаст новую.
+
+Таблицы с системными логами `log` можно настроить, создав конфигурационный файл с тем же именем, что и таблица в разделе `/etc/clickhouse-server/config.d/`, или указав соответствующие элементы в `/etc/clickhouse-server/config.xml`. Настраиваться могут следующие элементы:
+
+-   `database` — база данных, к которой принадлежит системная таблица. Эта опция на текущий момент устарела. Все системные таблицы находятся в базе данных `system`.
+-   `table` — таблица для добавления данных.
+-   `partition_by` — [ключ партиционирования](../../engines/table-engines/mergetree-family/custom-partitioning-key.md).
+-   `ttl` — [время жизни](../../sql-reference/statements/alter/ttl.md) таблицы.
+-   `flush_interval_milliseconds` — интервал сброса данных на диск, в миллисекундах.
+-   `engine` — полное имя движка (начиная с `ENGINE =` ) с параметрами. Эта опция противоречит `partition_by` и `ttl`. Если указать оба параметра вместе, сервер вернет ошибку и завершит работу.
+
+Пример:
+
+```xml
+
+    
+        system
+        query_log
+ toYYYYMM(event_date) + event_date + INTERVAL 30 DAY DELETE + + 7500 +
+
+``` + +По умолчанию размер таблицы не ограничен. Управлять размером таблицы можно используя [TTL](../../sql-reference/statements/alter/ttl.md#manipuliatsii-s-ttl-tablitsy) для удаления устаревших записей журнала. Также вы можете использовать функцию партиционирования для таблиц `MergeTree`. ### Источники системных показателей Для сбора системных показателей сервер ClickHouse использует: -- Возможности `CAP_NET_ADMIN`. +- возможности `CAP_NET_ADMIN`. - [procfs](https://ru.wikipedia.org/wiki/Procfs) (только Linux). -**procfs** Если для сервера ClickHouse не включено `CAP_NET_ADMIN`, он пытается обратиться к `ProcfsMetricsProvider`. `ProcfsMetricsProvider` позволяет собирать системные показатели для каждого запроса (для CPU и I/O). diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index 3bb11b638b2..77275b65a05 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -572,7 +572,7 @@ SOURCE(CLICKHOUSE( или ``` sql -SOURCE(MONGO( +SOURCE(MONGODB( host 'localhost' port 27017 user '' diff --git a/docs/ru/sql-reference/statements/select/all.md b/docs/ru/sql-reference/statements/select/all.md new file mode 100644 index 00000000000..4049d77a173 --- /dev/null +++ b/docs/ru/sql-reference/statements/select/all.md @@ -0,0 +1,22 @@ +--- +toc_title: ALL +--- + +# Секция ALL {#select-all} + +Если в таблице несколько совпадающих строк, то `ALL` возвращает все из них. Поведение запроса `SELECT ALL` точно такое же, как и `SELECT` без аргумента `DISTINCT`. Если указаны оба аргумента: `ALL` и `DISTINCT`, функция вернет исключение. + + +`ALL` может быть указан внутри агрегатной функции, например, результат выполнения запроса: + +```sql +SELECT sum(ALL number) FROM numbers(10); +``` + +равен результату выполнения запроса: + +```sql +SELECT sum(number) FROM numbers(10); +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/sql-reference/statements/select/all) From 14513e520e46c51a8811638814978e52d520bdbe Mon Sep 17 00:00:00 2001 From: Denis Glazachev Date: Mon, 1 Mar 2021 18:03:29 +0400 Subject: [PATCH 474/510] Fix privileges --- tests/performance/prewhere_with_row_level_filter.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/performance/prewhere_with_row_level_filter.xml b/tests/performance/prewhere_with_row_level_filter.xml index 03c73a88f89..b7b41c0a30f 100644 --- a/tests/performance/prewhere_with_row_level_filter.xml +++ b/tests/performance/prewhere_with_row_level_filter.xml @@ -1,6 +1,7 @@ DROP TABLE IF EXISTS test_prl; CREATE TABLE test_prl (n UInt64) ENGINE MergeTree ORDER BY n; + GRANT CREATE ROW POLICY ON *.* TO CURRENT_USER CREATE ROW POLICY OR REPLACE test_prl_policy ON test_prl AS PERMISSIVE FOR SELECT USING n % 7 TO ALL; INSERT INTO test_prl SELECT number FROM numbers(50000000); From 10aa26576cf120f9c2e9fa2996667973a55027a2 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 1 Mar 2021 17:11:45 +0300 Subject: [PATCH 475/510] Update DatabaseReplicatedWorker.cpp --- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index 25a19c4dfb2..ee5a3b5eed0 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -63,7 +63,7 @@ void DatabaseReplicatedDDLWorker::initializeReplication() if (our_log_ptr == 0 || our_log_ptr + logs_to_keep < max_log_ptr) database->recoverLostReplica(current_zookeeper, our_log_ptr, max_log_ptr); else - last_skipped_entry_name.emplace(log_ptr_str); + last_skipped_entry_name.emplace(DDLTaskBase::getLogEntryName(our_log_ptr)); } String DatabaseReplicatedDDLWorker::enqueueQuery(DDLLogEntry & entry) From 3764a2a2beb106d595ba5c90b2c72ead12b58c58 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Mar 2021 19:15:59 +0300 Subject: [PATCH 476/510] AggregateFunctionSumMap better comment message --- src/AggregateFunctions/AggregateFunctionSumMap.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index f6a473546f9..30efb3d5fa0 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -118,6 +118,8 @@ public: WhichDataType value_type_to_check(value_type); /// Do not promote decimal because of implementation issues of this function design + /// Currently we cannot get result column type in case of decimal we cannot get decimal scale + /// in method void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override /// If we decide to make this function more efficient we should promote decimal type during summ if (value_type_to_check.isDecimal()) result_type = value_type_without_nullable; @@ -337,7 +339,11 @@ public: if (elem.second[col].isNull()) to_values_col.insertDefault(); else + { + auto element_field = elem.second[col]; + to_values_col.insert(elem.second[col]); + } } } } From 019a2090c71898719858ad451a4e83de842fcf30 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Mar 2021 19:18:14 +0300 Subject: [PATCH 477/510] Fix unused variable --- src/AggregateFunctions/AggregateFunctionSumMap.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 30efb3d5fa0..8af20fe0e2d 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -340,8 +340,6 @@ public: to_values_col.insertDefault(); else { - auto element_field = elem.second[col]; - to_values_col.insert(elem.second[col]); } } From 7ed5900251ef5945dcca32d0b0c1280b2c920800 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 1 Mar 2021 19:18:56 +0300 Subject: [PATCH 478/510] Updated style --- src/AggregateFunctions/AggregateFunctionSumMap.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSumMap.h b/src/AggregateFunctions/AggregateFunctionSumMap.h index 8af20fe0e2d..3233199f01e 100644 --- a/src/AggregateFunctions/AggregateFunctionSumMap.h +++ b/src/AggregateFunctions/AggregateFunctionSumMap.h @@ -339,9 +339,7 @@ public: if (elem.second[col].isNull()) to_values_col.insertDefault(); else - { to_values_col.insert(elem.second[col]); - } } } } From 15b3f379a56c8f648cff38695ec2aa617c13fe58 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Mon, 1 Mar 2021 19:41:16 +0300 Subject: [PATCH 479/510] DOCSUP-4915: documented geo data types (#21294) * Initial * Fix links * Fix master * Mistakes corrected * Minor fix * Fixes and intro. * Update geo.md * Apply suggestions from code review Co-authored-by: Anton Popov Co-authored-by: Olga Revyakina Co-authored-by: Anton Popov --- .../template-data-type.md | 2 +- docs/en/operations/settings/settings.md | 15 ++- docs/en/sql-reference/data-types/geo.md | 106 ++++++++++++++++++ docs/ru/operations/settings/settings.md | 11 ++ docs/ru/sql-reference/data-types/geo.md | 106 ++++++++++++++++++ 5 files changed, 237 insertions(+), 3 deletions(-) create mode 100644 docs/en/sql-reference/data-types/geo.md create mode 100644 docs/ru/sql-reference/data-types/geo.md diff --git a/docs/_description_templates/template-data-type.md b/docs/_description_templates/template-data-type.md index edb6586ee7d..5e560b9325d 100644 --- a/docs/_description_templates/template-data-type.md +++ b/docs/_description_templates/template-data-type.md @@ -26,4 +26,4 @@ The name of an additional section can be any, for example, **Usage**. - [link](#) -[Original article](https://clickhouse.tech/docs/en/data_types//) +[Original article](https://clickhouse.tech/docs/en/data-types//) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 1988e2ec6fa..3c343e09fd3 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -2658,8 +2658,6 @@ Result: Note that this setting influences [Materialized view](../../sql-reference/statements/create/view.md#materialized) and [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md) behaviour. -[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) - ## engine_file_empty_if_not_exists {#engine-file-empty_if-not-exists} Allows to select data from a file engine table without file. @@ -2679,3 +2677,16 @@ Possible values: - 1 — Enabled. Default value: `0`. + +## allow_experimental_geo_types {#allow-experimental-geo-types} + +Allows working with experimental [geo data types](../../sql-reference/data-types/geo.md). + +Possible values: + +- 0 — Working with geo data types is disabled. +- 1 — Working with geo data types is enabled. + +Default value: `0`. + +[Original article](https://clickhouse.tech/docs/en/operations/settings/settings/) diff --git a/docs/en/sql-reference/data-types/geo.md b/docs/en/sql-reference/data-types/geo.md new file mode 100644 index 00000000000..9ed328e0de6 --- /dev/null +++ b/docs/en/sql-reference/data-types/geo.md @@ -0,0 +1,106 @@ +--- +toc_priority: 62 +toc_title: Geo +--- + +# Geo Data Types {#geo-data-types} + +Clickhouse supports data types for representing geographical objects — locations, lands, etc. + +!!! warning "Warning" + Currently geo data types are an experimental feature. To work with them you must set `allow_experimental_geo_types = 1`. + +**See Also** +- [Representing simple geographical features](https://en.wikipedia.org/wiki/GeoJSON). +- [allow_experimental_geo_types](../../operations/settings/settings.md#allow-experimental-geo-types) setting. + +## Point {#point-data-type} + +`Point` is represented by its X and Y coordinates, stored as a [Tuple](tuple.md)([Float64](float.md), [Float64](float.md)). + +**Example** + +Query: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_point (p Point) ENGINE = Memory(); +INSERT INTO geo_point VALUES((10, 10)); +SELECT p, toTypeName(p) FROM geo_point; +``` +Result: + +``` text +┌─p─────┬─toTypeName(p)─┐ +│ (10,10) │ Point │ +└───────┴───────────────┘ +``` + +## Ring {#ring-data-type} + +`Ring` is a simple polygon without holes stored as an array of points: [Array](array.md)([Point](#point-data-type)). + +**Example** + +Query: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_ring (r Ring) ENGINE = Memory(); +INSERT INTO geo_ring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]); +SELECT r, toTypeName(r) FROM geo_ring; +``` +Result: + +``` text +┌─r─────────────────────────────┬─toTypeName(r)─┐ +│ [(0,0),(10,0),(10,10),(0,10)] │ Ring │ +└───────────────────────────────┴───────────────┘ +``` + +## Polygon {#polygon-data-type} + +`Polygon` is a polygon with holes stored as an array of rings: [Array](array.md)([Ring](#ring-data-type)). First element of outer array is the outer shape of polygon and all the following elements are holes. + +**Example** + +This is a polygon with one hole: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_polygon (pg Polygon) ENGINE = Memory(); +INSERT INTO geo_polygon VALUES([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]]); +SELECT pg, toTypeName(pg) FROM geo_polygon; +``` + +Result: + +``` text +┌─pg────────────────────────────────────────────────────────────┬─toTypeName(pg)─┐ +│ [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] │ Polygon │ +└───────────────────────────────────────────────────────────────┴────────────────┘ +``` + +## MultiPolygon {#multipolygon-data-type} + +`MultiPolygon` consists of multiple polygons and is stored as an array of polygons: [Array](array.md)([Polygon](#polygon-data-type)). + +**Example** + +This multipolygon consists of two separate polygons — the first one without holes, and the second with one hole: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_multipolygon (mpg MultiPolygon) ENGINE = Memory(); +INSERT INTO geo_multipolygon VALUES([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); +SELECT mpg, toTypeName(mpg) FROM geo_multipolygon; +``` +Result: + +``` text +┌─mpg─────────────────────────────────────────────────────────────────────────────────────────────┬─toTypeName(mpg)─┐ +│ [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] │ MultiPolygon │ +└─────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────┘ +``` + +[Original article](https://clickhouse.tech/docs/en/data-types/geo/) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index a67b4a283fa..f8f587c8a36 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2552,4 +2552,15 @@ SELECT * FROM test2; Обратите внимание на то, что эта настройка влияет на поведение [материализованных представлений](../../sql-reference/statements/create/view.md#materialized) и БД [MaterializeMySQL](../../engines/database-engines/materialize-mysql.md). +## allow_experimental_geo_types {#allow-experimental-geo-types} + +Разрешает использование экспериментальных типов данных для работы с [географическими структурами](../../sql-reference/data-types/geo.md). + +Возможные значения: + +- 0 — Использование типов данных для работы с географическими структурами не поддерживается. +- 1 — Использование типов данных для работы с географическими структурами поддерживается. + +Значение по умолчанию: `0`. + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) diff --git a/docs/ru/sql-reference/data-types/geo.md b/docs/ru/sql-reference/data-types/geo.md new file mode 100644 index 00000000000..23293b30927 --- /dev/null +++ b/docs/ru/sql-reference/data-types/geo.md @@ -0,0 +1,106 @@ +--- +toc_priority: 62 +toc_title: Географические структуры +--- + +# Типы данных для работы с географическими структурами {#geo-data-types} + +ClickHouse поддерживает типы данных для отображения географических объектов — точек (местоположений), территорий и т.п. + +!!! warning "Предупреждение" + Сейчас использование типов данных для работы с географическими структурами является экспериментальной возможностью. Чтобы использовать эти типы данных, включите настройку `allow_experimental_geo_types = 1`. + +**См. также** +- [Хранение географических структур данных](https://ru.wikipedia.org/wiki/GeoJSON). +- Настройка [allow_experimental_geo_types](../../operations/settings/settings.md#allow-experimental-geo-types). + +## Point {#point-data-type} + +Тип `Point` (точка) определяется парой координат X и Y и хранится в виде кортежа [Tuple](tuple.md)([Float64](float.md), [Float64](float.md)). + +**Пример** + +Запрос: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_point (p Point) ENGINE = Memory(); +INSERT INTO geo_point VALUES((10, 10)); +SELECT p, toTypeName(p) FROM geo_point; +``` +Результат: + +``` text +┌─p─────┬─toTypeName(p)─┐ +│ (10,10) │ Point │ +└───────┴───────────────┘ +``` + +## Ring {#ring-data-type} + +Тип `Ring` описывает простой многоугольник без внутренних областей (дыр) и хранится в виде массива точек: [Array](array.md)([Point](#point-data-type)). + +**Пример** + +Запрос: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_ring (r Ring) ENGINE = Memory(); +INSERT INTO geo_ring VALUES([(0, 0), (10, 0), (10, 10), (0, 10)]); +SELECT r, toTypeName(r) FROM geo_ring; +``` +Результат: + +``` text +┌─r─────────────────────────────┬─toTypeName(r)─┐ +│ [(0,0),(10,0),(10,10),(0,10)] │ Ring │ +└───────────────────────────────┴───────────────┘ +``` + +## Polygon {#polygon-data-type} + +Тип `Polygon` описывает многоугольник с внутренними областями (дырами) и хранится в виде массива: [Array](array.md)([Ring](#ring-data-type)). Первый элемент массива описывает внешний многоугольник (контур), а остальные элементы описывают дыры. + +**Пример** + +Запись в этой таблице описывает многоугольник с одной дырой: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_polygon (pg Polygon) ENGINE = Memory(); +INSERT INTO geo_polygon VALUES([[(20, 20), (50, 20), (50, 50), (20, 50)], [(30, 30), (50, 50), (50, 30)]]); +SELECT pg, toTypeName(pg) FROM geo_polygon; +``` + +Результат: + +``` text +┌─pg────────────────────────────────────────────────────────────┬─toTypeName(pg)─┐ +│ [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]] │ Polygon │ +└───────────────────────────────────────────────────────────────┴────────────────┘ +``` + +## MultiPolygon {#multipolygon-data-type} + +Тип `MultiPolygon` описывает элемент, состоящий из нескольких простых многоугольников (полигональную сетку). Он хранится в виде массива многоугольников: [Array](array.md)([Polygon](#polygon-data-type)). + +**Пример** + +Запись в этой таблице описывает элемент, состоящий из двух многоугольников — первый без дыр, а второй с одной дырой: + +```sql +SET allow_experimental_geo_types = 1; +CREATE TABLE geo_multipolygon (mpg MultiPolygon) ENGINE = Memory(); +INSERT INTO geo_multipolygon VALUES([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]); +SELECT mpg, toTypeName(mpg) FROM geo_multipolygon; +``` +Result: + +``` text +┌─mpg─────────────────────────────────────────────────────────────────────────────────────────────┬─toTypeName(mpg)─┐ +│ [[[(0,0),(10,0),(10,10),(0,10)]],[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]] │ MultiPolygon │ +└─────────────────────────────────────────────────────────────────────────────────────────────────┴─────────────────┘ +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/data-types/geo/) From f82e00ff3c47fd99e4bb7ec912392dcf269e5206 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 1 Mar 2021 23:10:50 +0300 Subject: [PATCH 480/510] Fix inconsistent column type for Values input format. --- .../Formats/Impl/ConstantExpressionTemplate.cpp | 14 +++++++++----- .../Formats/Impl/ConstantExpressionTemplate.h | 2 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 13 +++++++++---- .../01746_lc_values_format_bug.reference | 1 + .../0_stateless/01746_lc_values_format_bug.sql | 14 ++++++++++++++ 5 files changed, 34 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/01746_lc_values_format_bug.reference create mode 100644 tests/queries/0_stateless/01746_lc_values_format_bug.sql diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp index 4aed9979ab2..f599b7c87e9 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -589,7 +590,7 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co } } -ColumnPtr ConstantExpressionTemplate::evaluateAll(BlockMissingValues & nulls, size_t column_idx, size_t offset) +ColumnPtr ConstantExpressionTemplate::evaluateAll(BlockMissingValues & nulls, size_t column_idx, const DataTypePtr & expected_type, size_t offset) { Block evaluated = structure->literals.cloneWithColumns(std::move(columns)); columns = structure->literals.cloneEmptyColumns(); @@ -607,12 +608,13 @@ ColumnPtr ConstantExpressionTemplate::evaluateAll(BlockMissingValues & nulls, si ErrorCodes::LOGICAL_ERROR); rows_count = 0; - ColumnPtr res = evaluated.getByName(structure->result_column_name).column->convertToFullColumnIfConst(); + auto res = evaluated.getByName(structure->result_column_name); + res.column = res.column->convertToFullColumnIfConst(); if (!structure->null_as_default) - return res; + return castColumn(res, expected_type); /// Extract column with evaluated expression and mask for NULLs - const auto & tuple = assert_cast(*res); + const auto & tuple = assert_cast(*res.column); if (tuple.tupleSize() != 2) throw Exception("Invalid tuple size, it'a a bug", ErrorCodes::LOGICAL_ERROR); const auto & is_null = assert_cast(tuple.getColumn(1)); @@ -621,7 +623,9 @@ ColumnPtr ConstantExpressionTemplate::evaluateAll(BlockMissingValues & nulls, si if (is_null.getUInt(i)) nulls.setBit(column_idx, offset + i); - return tuple.getColumnPtr(0); + res.column = tuple.getColumnPtr(0); + res.type = assert_cast(*res.type).getElements()[0]; + return castColumn(res, expected_type); } void ConstantExpressionTemplate::TemplateStructure::addNodesToCastResult(const IDataType & result_column_type, ASTPtr & expr, bool null_as_default) diff --git a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h index 931b05673c6..299ce4c9925 100644 --- a/src/Processors/Formats/Impl/ConstantExpressionTemplate.h +++ b/src/Processors/Formats/Impl/ConstantExpressionTemplate.h @@ -72,7 +72,7 @@ public: /// Evaluate batch of expressions were parsed using template. /// If template was deduced with null_as_default == true, set bits in nulls for NULL values in column_idx, starting from offset. - ColumnPtr evaluateAll(BlockMissingValues & nulls, size_t column_idx, size_t offset = 0); + ColumnPtr evaluateAll(BlockMissingValues & nulls, size_t column_idx, const DataTypePtr & expected_type, size_t offset = 0); size_t rowsCount() const { return rows_count; } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 3fde84a4090..1455b8f6740 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -73,11 +73,13 @@ Chunk ValuesBlockInputFormat::generate() { if (!templates[i] || !templates[i]->rowsCount()) continue; + + const auto & expected_type = header.getByPosition(i).type; if (columns[i]->empty()) - columns[i] = IColumn::mutate(templates[i]->evaluateAll(block_missing_values, i)); + columns[i] = IColumn::mutate(templates[i]->evaluateAll(block_missing_values, i, expected_type)); else { - ColumnPtr evaluated = templates[i]->evaluateAll(block_missing_values, i, columns[i]->size()); + ColumnPtr evaluated = templates[i]->evaluateAll(block_missing_values, i, expected_type, columns[i]->size()); columns[i]->insertRangeFrom(*evaluated, 0, evaluated->size()); } } @@ -135,13 +137,16 @@ bool ValuesBlockInputFormat::tryParseExpressionUsingTemplate(MutableColumnPtr & return true; } + const auto & header = getPort().getHeader(); + const auto & expected_type = header.getByPosition(column_idx).type; + /// Expression in the current row is not match template deduced on the first row. /// Evaluate expressions, which were parsed using this template. if (column->empty()) - column = IColumn::mutate(templates[column_idx]->evaluateAll(block_missing_values, column_idx)); + column = IColumn::mutate(templates[column_idx]->evaluateAll(block_missing_values, column_idx, expected_type)); else { - ColumnPtr evaluated = templates[column_idx]->evaluateAll(block_missing_values, column_idx, column->size()); + ColumnPtr evaluated = templates[column_idx]->evaluateAll(block_missing_values, column_idx, expected_type, column->size()); column->insertRangeFrom(*evaluated, 0, evaluated->size()); } /// Do not use this template anymore diff --git a/tests/queries/0_stateless/01746_lc_values_format_bug.reference b/tests/queries/0_stateless/01746_lc_values_format_bug.reference new file mode 100644 index 00000000000..78981922613 --- /dev/null +++ b/tests/queries/0_stateless/01746_lc_values_format_bug.reference @@ -0,0 +1 @@ +a diff --git a/tests/queries/0_stateless/01746_lc_values_format_bug.sql b/tests/queries/0_stateless/01746_lc_values_format_bug.sql new file mode 100644 index 00000000000..6717b9ae5e3 --- /dev/null +++ b/tests/queries/0_stateless/01746_lc_values_format_bug.sql @@ -0,0 +1,14 @@ +drop table if exists lc_test; + +CREATE TABLE lc_test +( + `id` LowCardinality(String) +) +ENGINE = MergeTree +PARTITION BY tuple() +ORDER BY id; + +insert into lc_test values (toString('a')); + +select id from lc_test; +drop table if exists lc_test; From a26c9e64a99f0f4d36a4fd47a1814c5f799462c1 Mon Sep 17 00:00:00 2001 From: feng lv Date: Sun, 28 Feb 2021 07:42:08 +0000 Subject: [PATCH 481/510] fix fix --- src/Storages/IStorage.cpp | 13 ++++++++----- src/Storages/IStorage.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- src/Storages/StorageBuffer.cpp | 4 ++-- src/Storages/StorageDistributed.cpp | 4 ++-- src/Storages/StorageMerge.cpp | 4 ++-- src/Storages/StorageNull.cpp | 4 ++-- .../01746_forbid_drop_column_referenced_by_mv.sql | 6 +++--- 8 files changed, 22 insertions(+), 19 deletions(-) diff --git a/src/Storages/IStorage.cpp b/src/Storages/IStorage.cpp index 7b68e84e6a1..5129d03cdee 100644 --- a/src/Storages/IStorage.cpp +++ b/src/Storages/IStorage.cpp @@ -182,17 +182,20 @@ Names IStorage::getAllRegisteredNames() const return result; } -NameDependencies IStorage::getColumnNamesAndReferencedMvMap(const Context & context) const +NameDependencies IStorage::getDependentViewsByColumn(const Context & context) const { NameDependencies name_deps; auto dependencies = DatabaseCatalog::instance().getDependencies(storage_id); for (const auto & depend_id : dependencies) { auto depend_table = DatabaseCatalog::instance().getTable(depend_id, context); - const auto & select_query = depend_table->getInMemoryMetadataPtr()->select.inner_query; - auto required_columns = InterpreterSelectQuery(select_query, context, SelectQueryOptions{}.noModify()).getRequiredColumns(); - for (const auto & col_name : required_columns) - name_deps[col_name].push_back(depend_id.table_name); + if (depend_table->getInMemoryMetadataPtr()->select.inner_query) + { + const auto & select_query = depend_table->getInMemoryMetadataPtr()->select.inner_query; + auto required_columns = InterpreterSelectQuery(select_query, context, SelectQueryOptions{}.noModify()).getRequiredColumns(); + for (const auto & col_name : required_columns) + name_deps[col_name].push_back(depend_id.table_name); + } } return name_deps; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index 72eb50e8a85..4dfd2ca50f3 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -176,7 +176,7 @@ public: Names getAllRegisteredNames() const override; - NameDependencies getColumnNamesAndReferencedMvMap(const Context & context) const; + NameDependencies getDependentViewsByColumn(const Context & context) const; protected: /// Returns whether the column is virtual - by default all columns are real. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ed6873f8370..292cd36e696 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -1487,7 +1487,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const C old_types.emplace(column.name, column.type.get()); NamesAndTypesList columns_to_check_conversion; - auto name_deps = getColumnNamesAndReferencedMvMap(context); + auto name_deps = getDependentViewsByColumn(context); for (const AlterCommand & command : commands) { /// Just validate partition expression @@ -1568,7 +1568,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, const C ErrorCodes::ALTER_OF_COLUMN_IS_FORBIDDEN); } - auto deps_mv = name_deps[command.column_name]; + const auto & deps_mv = name_deps[command.column_name]; if (!deps_mv.empty()) { throw Exception( diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index 46cdf237d91..e99f26c1a30 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -913,7 +913,7 @@ void StorageBuffer::reschedule() void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const { - auto name_deps = getColumnNamesAndReferencedMvMap(context); + auto name_deps = getDependentViewsByColumn(context); for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN @@ -923,7 +923,7 @@ void StorageBuffer::checkAlterIsPossible(const AlterCommands & commands, const C ErrorCodes::NOT_IMPLEMENTED); if (command.type == AlterCommand::Type::DROP_COLUMN) { - auto deps_mv = name_deps[command.column_name]; + const auto & deps_mv = name_deps[command.column_name]; if (!deps_mv.empty()) { throw Exception( diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index 337b89af017..ad904994e91 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -580,7 +580,7 @@ BlockOutputStreamPtr StorageDistributed::write(const ASTPtr &, const StorageMeta void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const { - auto name_deps = getColumnNamesAndReferencedMvMap(context); + auto name_deps = getDependentViewsByColumn(context); for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN @@ -593,7 +593,7 @@ void StorageDistributed::checkAlterIsPossible(const AlterCommands & commands, co ErrorCodes::NOT_IMPLEMENTED); if (command.type == AlterCommand::DROP_COLUMN) { - auto deps_mv = name_deps[command.column_name]; + const auto & deps_mv = name_deps[command.column_name]; if (!deps_mv.empty()) { throw Exception( diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 1048cbc6aa3..46be91ba258 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -475,7 +475,7 @@ DatabaseTablesIteratorPtr StorageMerge::getDatabaseIterator(const Context & cont void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const { - auto name_deps = getColumnNamesAndReferencedMvMap(context); + auto name_deps = getDependentViewsByColumn(context); for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN @@ -485,7 +485,7 @@ void StorageMerge::checkAlterIsPossible(const AlterCommands & commands, const Co ErrorCodes::NOT_IMPLEMENTED); if (command.type == AlterCommand::Type::DROP_COLUMN) { - auto deps_mv = name_deps[command.column_name]; + const auto & deps_mv = name_deps[command.column_name]; if (!deps_mv.empty()) { throw Exception( diff --git a/src/Storages/StorageNull.cpp b/src/Storages/StorageNull.cpp index 8b2c96ba436..ed9a7fffc63 100644 --- a/src/Storages/StorageNull.cpp +++ b/src/Storages/StorageNull.cpp @@ -38,7 +38,7 @@ void registerStorageNull(StorageFactory & factory) void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Context & context) const { - auto name_deps = getColumnNamesAndReferencedMvMap(context); + auto name_deps = getDependentViewsByColumn(context); for (const auto & command : commands) { if (command.type != AlterCommand::Type::ADD_COLUMN && command.type != AlterCommand::Type::MODIFY_COLUMN @@ -48,7 +48,7 @@ void StorageNull::checkAlterIsPossible(const AlterCommands & commands, const Con ErrorCodes::NOT_IMPLEMENTED); if (command.type == AlterCommand::DROP_COLUMN) { - auto deps_mv = name_deps[command.column_name]; + const auto & deps_mv = name_deps[command.column_name]; if (!deps_mv.empty()) { throw Exception( diff --git a/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql b/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql index c05a7beff44..f084cae7780 100644 --- a/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql +++ b/tests/queries/0_stateless/01746_forbid_drop_column_referenced_by_mv.sql @@ -75,7 +75,7 @@ ENGINE = Memory; DROP TABLE IF EXISTS `01746_dist`; CREATE TABLE `01746_dist` AS `01746_local` -ENGINE = Distributed('test_shard_localhost', default, `01746_local`, rand()); +ENGINE = Distributed('test_shard_localhost', currentDatabase(), `01746_local`, rand()); DROP TABLE IF EXISTS `01746_dist_mv`; CREATE MATERIALIZED VIEW `01746_dist_mv` @@ -111,7 +111,7 @@ ENGINE = Memory; DROP TABLE IF EXISTS `01746_merge`; CREATE TABLE `01746_merge` AS `01746_merge_t` -ENGINE = Merge(default, '01746_merge_t'); +ENGINE = Merge(currentDatabase(), '01746_merge_t'); DROP TABLE IF EXISTS `01746_merge_mv`; CREATE MATERIALIZED VIEW `01746_merge_mv` @@ -147,7 +147,7 @@ ENGINE = Memory; DROP TABLE IF EXISTS `01746_buffer`; CREATE TABLE `01746_buffer` AS `01746_buffer_t` -ENGINE = Buffer(default, `01746_buffer_t`, 16, 10, 100, 10000, 1000000, 10000000, 100000000); +ENGINE = Buffer(currentDatabase(), `01746_buffer_t`, 16, 10, 100, 10000, 1000000, 10000000, 100000000); DROP TABLE IF EXISTS `01746_buffer_mv`; CREATE MATERIALIZED VIEW `01746_buffer_mv` From 62e8684ad1ac6ced473ac4c753cc3d4937cc3bb6 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 26 Feb 2021 12:50:04 +0800 Subject: [PATCH 482/510] IColunm::hasEqualValues() --- src/Columns/ColumnAggregateFunction.h | 5 +++++ src/Columns/ColumnArray.cpp | 4 ++++ src/Columns/ColumnArray.h | 1 + src/Columns/ColumnCompressed.h | 4 ++++ src/Columns/ColumnConst.h | 2 ++ src/Columns/ColumnDecimal.cpp | 6 ++++++ src/Columns/ColumnDecimal.h | 1 + src/Columns/ColumnFixedString.h | 5 +++++ src/Columns/ColumnFunction.h | 5 +++++ src/Columns/ColumnLowCardinality.cpp | 5 +++++ src/Columns/ColumnLowCardinality.h | 2 ++ src/Columns/ColumnMap.cpp | 5 +++++ src/Columns/ColumnMap.h | 1 + src/Columns/ColumnNullable.cpp | 5 +++++ src/Columns/ColumnNullable.h | 1 + src/Columns/ColumnString.cpp | 5 +++++ src/Columns/ColumnString.h | 2 ++ src/Columns/ColumnTuple.cpp | 5 +++++ src/Columns/ColumnTuple.h | 1 + src/Columns/ColumnVector.h | 5 +++++ src/Columns/IColumn.h | 6 ++++++ src/Columns/IColumnDummy.h | 2 ++ src/Columns/IColumnImpl.h | 12 ++++++++++++ src/Columns/IColumnUnique.h | 5 +++++ 24 files changed, 95 insertions(+) diff --git a/src/Columns/ColumnAggregateFunction.h b/src/Columns/ColumnAggregateFunction.h index cd45cf583a0..f023177d7f2 100644 --- a/src/Columns/ColumnAggregateFunction.h +++ b/src/Columns/ColumnAggregateFunction.h @@ -198,6 +198,11 @@ public: throw Exception("Method compareColumn is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); } + bool hasEqualValues() const override + { + throw Exception("Method hasEqualValues is not supported for ColumnAggregateFunction", ErrorCodes::NOT_IMPLEMENTED); + } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, Permutation & res, EqualRanges & equal_range) const override; diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp index e8a48672435..d8821a646ae 100644 --- a/src/Columns/ColumnArray.cpp +++ b/src/Columns/ColumnArray.cpp @@ -370,6 +370,10 @@ void ColumnArray::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } +bool ColumnArray::hasEqualValues() const +{ + return hasEqualValuesImpl(); +} namespace { diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 1caaf672d49..7d01d04735b 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -78,6 +78,7 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint, const Collator & collator) const override; + bool hasEqualValues() const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const override; void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; diff --git a/src/Columns/ColumnCompressed.h b/src/Columns/ColumnCompressed.h index f6b6bf22177..a31147b0702 100644 --- a/src/Columns/ColumnCompressed.h +++ b/src/Columns/ColumnCompressed.h @@ -96,6 +96,10 @@ public: { throwMustBeDecompressed(); } + bool hasEqualValues() const override + { + throwMustBeDecompressed(); + } void getPermutation(bool, size_t, int, Permutation &) const override { throwMustBeDecompressed(); } void updatePermutation(bool, size_t, int, Permutation &, EqualRanges &) const override { throwMustBeDecompressed(); } ColumnPtr replicate(const Offsets &) const override { throwMustBeDecompressed(); } diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index 99c997ab269..9441f339085 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -206,6 +206,8 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + bool hasEqualValues() const override { return true; } + MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; void gather(ColumnGathererStream &) override diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index bad3a4c3402..4a47919adf1 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -58,6 +58,12 @@ void ColumnDecimal::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } +template +bool ColumnDecimal::hasEqualValues() const +{ + return this->template hasEqualValuesImpl>(); +} + template StringRef ColumnDecimal::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const { diff --git a/src/Columns/ColumnDecimal.h b/src/Columns/ColumnDecimal.h index 5016ddca791..33eb2946122 100644 --- a/src/Columns/ColumnDecimal.h +++ b/src/Columns/ColumnDecimal.h @@ -136,6 +136,7 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + bool hasEqualValues() const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges& equal_range) const override; diff --git a/src/Columns/ColumnFixedString.h b/src/Columns/ColumnFixedString.h index 5e7b7f360f0..58f6d8142fb 100644 --- a/src/Columns/ColumnFixedString.h +++ b/src/Columns/ColumnFixedString.h @@ -132,6 +132,11 @@ public: compare_results, direction, nan_direction_hint); } + bool hasEqualValues() const override + { + return hasEqualValuesImpl(); + } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, EqualRanges & equal_range) const override; diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index f97f41a8627..6080a94d1fb 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -128,6 +128,11 @@ public: throw Exception("compareColumn is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + bool hasEqualValues() const override + { + throw Exception("hasEqualValues is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); + } + void getPermutation(bool, size_t, int, Permutation &) const override { throw Exception("getPermutation is not implemented for " + getName(), ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index 8af3b240cb9..fc822b7c620 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -311,6 +311,11 @@ void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num compare_results, direction, nan_direction_hint); } +bool ColumnLowCardinality::hasEqualValues() const +{ + return hasEqualValuesImpl(); +} + void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator) const { if (limit == 0) diff --git a/src/Columns/ColumnLowCardinality.h b/src/Columns/ColumnLowCardinality.h index a497be8847d..54ddb8ce68b 100644 --- a/src/Columns/ColumnLowCardinality.h +++ b/src/Columns/ColumnLowCardinality.h @@ -126,6 +126,8 @@ public: int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator &) const override; + bool hasEqualValues() const override; + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int, IColumn::Permutation & res, EqualRanges & equal_range) const override; diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp index e0e79d071b4..883a70db435 100644 --- a/src/Columns/ColumnMap.cpp +++ b/src/Columns/ColumnMap.cpp @@ -187,6 +187,11 @@ void ColumnMap::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } +bool ColumnMap::hasEqualValues() const +{ + return hasEqualValuesImpl(); +} + void ColumnMap::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { nested->getPermutation(reverse, limit, nan_direction_hint, res); diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index acae1574f4c..3987d36b19d 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -72,6 +72,7 @@ public: void compareColumn(const IColumn & rhs, size_t rhs_row_num, PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + bool hasEqualValues() const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_range) const override; diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 4e5cc2b4cf7..df5b8789bfc 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -271,6 +271,11 @@ void ColumnNullable::compareColumn(const IColumn & rhs, size_t rhs_row_num, compare_results, direction, nan_direction_hint); } +bool ColumnNullable::hasEqualValues() const +{ + return hasEqualValuesImpl(); +} + void ColumnNullable::getPermutationImpl(bool reverse, size_t limit, int null_direction_hint, Permutation & res, const Collator * collator) const { /// Cannot pass limit because of unknown amount of NULLs. diff --git a/src/Columns/ColumnNullable.h b/src/Columns/ColumnNullable.h index 8d267de8644..0d68a6a0a3f 100644 --- a/src/Columns/ColumnNullable.h +++ b/src/Columns/ColumnNullable.h @@ -94,6 +94,7 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int null_direction_hint, const Collator &) const override; + bool hasEqualValues() const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res, EqualRanges & equal_range) const override; void getPermutationWithCollation(const Collator & collator, bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp index 8fd22e85e10..31f2b2f9275 100644 --- a/src/Columns/ColumnString.cpp +++ b/src/Columns/ColumnString.cpp @@ -287,6 +287,11 @@ void ColumnString::compareColumn( compare_results, direction, nan_direction_hint); } +bool ColumnString::hasEqualValues() const +{ + return hasEqualValuesImpl(); +} + template struct ColumnString::Cmp { diff --git a/src/Columns/ColumnString.h b/src/Columns/ColumnString.h index 843e445d1a0..cf053d59b4d 100644 --- a/src/Columns/ColumnString.h +++ b/src/Columns/ColumnString.h @@ -240,6 +240,8 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; + bool hasEqualValues() const override; + /// Variant of compareAt for string comparison with respect of collation. int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs_, int, const Collator & collator) const override; diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index c7c5f7b97c6..7128b428b1a 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -312,6 +312,11 @@ int ColumnTuple::compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, return compareAtImpl(n, m, rhs, nan_direction_hint, &collator); } +bool ColumnTuple::hasEqualValues() const +{ + return hasEqualValuesImpl(); +} + template struct ColumnTuple::Less { diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 818b29937bd..858eff7a75a 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -76,6 +76,7 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const override; int compareAtWithCollation(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint, const Collator & collator) const override; + bool hasEqualValues() const override; void getExtremes(Field & min, Field & max) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges & equal_ranges) const override; diff --git a/src/Columns/ColumnVector.h b/src/Columns/ColumnVector.h index f0aa4a3bab5..5af5ef20310 100644 --- a/src/Columns/ColumnVector.h +++ b/src/Columns/ColumnVector.h @@ -205,6 +205,11 @@ public: compare_results, direction, nan_direction_hint); } + bool hasEqualValues() const override + { + return this->template hasEqualValuesImpl(); + } + void getPermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res) const override; void updatePermutation(bool reverse, size_t limit, int nan_direction_hint, IColumn::Permutation & res, EqualRanges& equal_range) const override; diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 2b4b633f9a5..7b4141c41d9 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -266,6 +266,9 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const = 0; + /// Check if all elements in the column has equal values. Return true if column is empty. + virtual bool hasEqualValues() const = 0; + /** Returns a permutation that sorts elements of this column, * i.e. perm[i]-th element of source column should be i-th element of sorted column. * reverse - reverse ordering (acsending). @@ -467,6 +470,9 @@ protected: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const; + + template + bool hasEqualValuesImpl() const; }; using ColumnPtr = IColumn::Ptr; diff --git a/src/Columns/IColumnDummy.h b/src/Columns/IColumnDummy.h index 10ef692dc6a..bb08e86bb30 100644 --- a/src/Columns/IColumnDummy.h +++ b/src/Columns/IColumnDummy.h @@ -40,6 +40,8 @@ public: { } + bool hasEqualValues() const override { return true; } + Field operator[](size_t) const override { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void get(size_t, Field &) const override { throw Exception("Cannot get value from " + getName(), ErrorCodes::NOT_IMPLEMENTED); } void insert(const Field &) override { throw Exception("Cannot insert element into " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index d2286981ac7..5a6102835ed 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -127,4 +127,16 @@ void IColumn::doCompareColumn(const Derived & rhs, size_t rhs_row_num, } } +template +bool IColumn::hasEqualValuesImpl() const +{ + size_t num_rows = size(); + for (size_t i = 0; i < num_rows; ++i) + { + if (compareAt(i, 0, static_cast(*this), false) != 0) + return false; + } + return true; +} + } diff --git a/src/Columns/IColumnUnique.h b/src/Columns/IColumnUnique.h index 99e134675f6..5558f493b92 100644 --- a/src/Columns/IColumnUnique.h +++ b/src/Columns/IColumnUnique.h @@ -172,6 +172,11 @@ public: { throw Exception("Method compareColumn is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); } + + bool hasEqualValues() const override + { + throw Exception("Method hasEqualValues is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); + } }; using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr; From 220d3cae7f78043c8834b7a907ed38a090833713 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Mar 2021 22:26:20 +0300 Subject: [PATCH 483/510] Update IColumn.h --- src/Columns/IColumn.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 7b4141c41d9..9ed064ede14 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -266,7 +266,7 @@ public: PaddedPODArray * row_indexes, PaddedPODArray & compare_results, int direction, int nan_direction_hint) const = 0; - /// Check if all elements in the column has equal values. Return true if column is empty. + /// Check if all elements in the column have equal values. Return true if column is empty. virtual bool hasEqualValues() const = 0; /** Returns a permutation that sorts elements of this column, From 32c011052f022afffe175f41f36e9ecace1902ce Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 1 Mar 2021 22:26:55 +0300 Subject: [PATCH 484/510] Update IColumnImpl.h --- src/Columns/IColumnImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/IColumnImpl.h b/src/Columns/IColumnImpl.h index 5a6102835ed..a1ee6a17982 100644 --- a/src/Columns/IColumnImpl.h +++ b/src/Columns/IColumnImpl.h @@ -131,7 +131,7 @@ template bool IColumn::hasEqualValuesImpl() const { size_t num_rows = size(); - for (size_t i = 0; i < num_rows; ++i) + for (size_t i = 1; i < num_rows; ++i) { if (compareAt(i, 0, static_cast(*this), false) != 0) return false; From c41dedc9223d1c2cd049949a298880ff9557f741 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 2 Mar 2021 11:07:42 +0800 Subject: [PATCH 485/510] optimize for ColumnLowCardinality --- src/Columns/ColumnLowCardinality.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index fc822b7c620..f434ff32f3a 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -313,7 +313,7 @@ void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num bool ColumnLowCardinality::hasEqualValues() const { - return hasEqualValuesImpl(); + return getDictionary().size() <= 1; } void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator) const From 41a8c2f8faaff656e19f5f311400dddb3a4b0c0e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Tue, 2 Mar 2021 09:37:47 +0300 Subject: [PATCH 486/510] clickhouse stop: wait for the server to be killed (process exited) Since killing doesn't happen instantly, and the start will fail [1]: The process with pid = 157 is running. Will terminate forcefully. Sent kill signal. /var/run/clickhouse-server/clickhouse-server.pid file exists and contains pid = 157. The process with pid = 157 is already running. + for _ in {1..120} + clickhouse-client --query 'SELECT 1' Code: 210. DB::NetException: Connection refused (localhost:9000) [1]: https://clickhouse-test-reports.s3.yandex.net/21318/4327e9e1d1e4c9c3576b00f41a8444237549dffd/functional_stateful_tests_(debug).html#fail1 --- programs/install/Install.cpp | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/programs/install/Install.cpp b/programs/install/Install.cpp index 8404586d394..ef72624e7ab 100644 --- a/programs/install/Install.cpp +++ b/programs/install/Install.cpp @@ -66,6 +66,7 @@ namespace ErrorCodes extern const int CANNOT_OPEN_FILE; extern const int SYSTEM_ERROR; extern const int NOT_ENOUGH_SPACE; + extern const int CANNOT_KILL; } } @@ -886,6 +887,27 @@ namespace fmt::print("Sent kill signal.\n", pid); else throwFromErrno("Cannot send kill signal", ErrorCodes::SYSTEM_ERROR); + + /// Wait for the process (100 seconds). + constexpr size_t num_kill_check_tries = 1000; + constexpr size_t kill_check_delay_ms = 100; + for (size_t i = 0; i < num_kill_check_tries; ++i) + { + fmt::print("Waiting for server to be killed\n"); + if (!isRunning(pid_file)) + { + fmt::print("Server exited\n"); + break; + } + sleepForMilliseconds(kill_check_delay_ms); + } + + if (isRunning(pid_file)) + { + throw Exception(ErrorCodes::CANNOT_KILL, + "The server process still exists after %zu ms", + num_kill_check_tries, kill_check_delay_ms); + } } return 0; From 3a6307a990f6e574d124c7f831beedec30cc1737 Mon Sep 17 00:00:00 2001 From: turbo jason Date: Tue, 2 Mar 2021 15:13:54 +0800 Subject: [PATCH 487/510] [ClickHouse][LOG]correct shutdown timeout log --- src/Coordination/NuKeeperServer.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Coordination/NuKeeperServer.cpp b/src/Coordination/NuKeeperServer.cpp index 400ef8fd58b..9ff1e2801c9 100644 --- a/src/Coordination/NuKeeperServer.cpp +++ b/src/Coordination/NuKeeperServer.cpp @@ -80,8 +80,9 @@ void NuKeeperServer::shutdown() { state_machine->shutdownStorage(); state_manager->flushLogStore(); - if (!launcher.shutdown(coordination_settings->shutdown_timeout.totalSeconds())) - LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", 5); + auto timeout = coordination_settings->shutdown_timeout.totalSeconds(); + if (!launcher.shutdown(timeout)) + LOG_WARNING(&Poco::Logger::get("NuKeeperServer"), "Failed to shutdown RAFT server in {} seconds", timeout); } namespace From 3df57da2d82a9690ae9afa60d25967aabf8c76e5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 2 Mar 2021 17:06:29 +0800 Subject: [PATCH 488/510] mysql client compatibility --- src/Functions/globalVariable.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Functions/globalVariable.cpp b/src/Functions/globalVariable.cpp index 55561af4d10..68b71eebff0 100644 --- a/src/Functions/globalVariable.cpp +++ b/src/Functions/globalVariable.cpp @@ -77,8 +77,10 @@ private: DataTypePtr type; Field value; }; - std::unordered_map global_variable_map = { - {"max_allowed_packet", {std::make_shared(), 67108864}}, {"version", {std::make_shared(), "5.7.30"}}}; + std::unordered_map global_variable_map + = {{"max_allowed_packet", {std::make_shared(), 67108864}}, + {"version", {std::make_shared(), "5.7.30"}}, + {"transaction_isolation", {std::make_shared(), "READ-UNCOMMITTED"}}}; }; } From 59a2c45555ed7e1bceaf8b31bf0e5fa39cd60ab7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 2 Mar 2021 13:09:29 +0300 Subject: [PATCH 489/510] Update executeQuery.cpp --- src/Interpreters/executeQuery.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 62986793376..1a0aa031d6f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -473,15 +473,12 @@ static std::tuple executeQueryImpl( if (settings.enable_global_with_statement) { ApplyWithGlobalVisitor().visit(ast); - query = serializeAST(*ast); } /// Normalize SelectWithUnionQuery NormalizeSelectWithUnionQueryVisitor::Data data{context.getSettingsRef().union_default_mode}; NormalizeSelectWithUnionQueryVisitor{data}.visit(ast); - query = serializeAST(*ast); - /// Check the limits. checkASTSizeLimits(*ast, settings); From 9ebf1b4fad1cca6e8b3bd990273ff56f1a5389af Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Mar 2021 13:33:54 +0300 Subject: [PATCH 490/510] Get rid of separate minmax index fields --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 24 ++++++--- src/Storages/MergeTree/MergeTreeData.cpp | 53 ++++++++++++------- src/Storages/MergeTree/MergeTreeData.h | 7 +-- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 21 +++++--- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 + .../MergeTree/MergeTreeDataWriter.cpp | 2 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../ReplicatedMergeTreeTableMetadata.cpp | 5 +- tests/queries/skip_list.json | 4 +- 11 files changed, 82 insertions(+), 42 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index f80020991b0..de7f3b6c0f4 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -363,7 +363,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->uuid = part_uuid; new_data_part->is_temp = true; new_data_part->setColumns(block.getNamesAndTypesList()); - new_data_part->minmax_idx.update(block, data.minmax_idx_columns); + new_data_part->minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); new_data_part->partition.create(metadata_snapshot, block, 0); MergedBlockOutputStream part_out(new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 2f6513bbb12..1568ca16254 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -57,13 +57,18 @@ static std::unique_ptr openForReading(const DiskPtr & di void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const DiskPtr & disk_, const String & part_path) { - size_t minmax_idx_size = data.minmax_idx_column_types.size(); + auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + + auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); + size_t minmax_idx_size = minmax_column_types.size(); hyperrectangle.reserve(minmax_idx_size); for (size_t i = 0; i < minmax_idx_size; ++i) { - String file_name = part_path + "minmax_" + escapeForFileName(data.minmax_idx_columns[i]) + ".idx"; + String file_name = part_path + "minmax_" + escapeForFileName(minmax_column_names[i]) + ".idx"; auto file = openForReading(disk_, file_name); - const DataTypePtr & data_type = data.minmax_idx_column_types[i]; + const DataTypePtr & data_type = minmax_column_types[i]; Field min_val; data_type->deserializeBinary(min_val, *file); @@ -78,7 +83,13 @@ void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const Dis void IMergeTreeDataPart::MinMaxIndex::store( const MergeTreeData & data, const DiskPtr & disk_, const String & part_path, Checksums & out_checksums) const { - store(data.minmax_idx_columns, data.minmax_idx_column_types, disk_, part_path, out_checksums); + auto metadata_snapshot = data.getInMemoryMetadataPtr(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + + auto minmax_column_names = data.getMinMaxColumnsNames(partition_key); + auto minmax_column_types = data.getMinMaxColumnsTypes(partition_key); + + store(minmax_column_names, minmax_column_types, disk_, part_path, out_checksums); } void IMergeTreeDataPart::MinMaxIndex::store( @@ -1168,6 +1179,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const auto metadata_snapshot = storage.getInMemoryMetadataPtr(); const auto & pk = metadata_snapshot->getPrimaryKey(); + const auto & partition_key = metadata_snapshot->getPartitionKey(); if (!checksums.empty()) { if (!pk.column_names.empty() && !checksums.files.count("primary.idx")) @@ -1183,7 +1195,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const if (!isEmpty()) { - for (const String & col_name : storage.minmax_idx_columns) + for (const String & col_name : storage.getMinMaxColumnsNames(partition_key)) { if (!checksums.files.count("minmax_" + escapeForFileName(col_name) + ".idx")) throw Exception("No minmax idx file checksum for column " + col_name, ErrorCodes::NO_FILE_IN_DATA_PART); @@ -1214,7 +1226,7 @@ void IMergeTreeDataPart::checkConsistencyBase() const if (metadata_snapshot->hasPartitionKey()) check_file_not_empty(volume->getDisk(), path + "partition.dat"); - for (const String & col_name : storage.minmax_idx_columns) + for (const String & col_name : storage.getMinMaxColumnsNames(partition_key)) check_file_not_empty(volume->getDisk(), path + "minmax_" + escapeForFileName(col_name) + ".idx"); } } diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0cedae9a2f4..caba9709785 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -165,6 +165,8 @@ MergeTreeData::MergeTreeData( { try { + + checkPartitionKeyAndInitMinMax(metadata_.partition_key); setProperties(metadata_, metadata_, attach); if (minmax_idx_date_column_pos == -1) throw Exception("Could not find Date column", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -179,9 +181,10 @@ MergeTreeData::MergeTreeData( else { is_custom_partitioned = true; - setProperties(metadata_, metadata_, attach); + checkPartitionKeyAndInitMinMax(metadata_.partition_key); min_format_version = MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING; } + setProperties(metadata_, metadata_, attach); /// NOTE: using the same columns list as is read when performing actual merges. merging_params.check(metadata_); @@ -396,7 +399,6 @@ void MergeTreeData::checkProperties( void MergeTreeData::setProperties(const StorageInMemoryMetadata & new_metadata, const StorageInMemoryMetadata & old_metadata, bool attach) { checkProperties(new_metadata, old_metadata, attach); - checkPartitionKeyAndInitMinMax(new_metadata.partition_key); setInMemoryMetadata(new_metadata); } @@ -421,6 +423,29 @@ ExpressionActionsPtr getCombinedIndicesExpression( } +ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partition_key) const +{ + NamesAndTypesList partition_key_columns; + if (!partition_key.column_names.empty()) + partition_key_columns = partition_key.expression->getRequiredColumnsWithTypes(); + + return std::make_shared(std::make_shared(partition_key_columns)); +} + +Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) const +{ + if (!partition_key.column_names.empty()) + return partition_key.expression->getRequiredColumns(); + return {}; +} + +DataTypes MergeTreeData::getMinMaxColumnsTypes(const KeyDescription & partition_key) const +{ + if (!partition_key.column_names.empty()) + return partition_key.expression->getRequiredColumnsWithTypes().getTypes(); + return {}; +} + ExpressionActionsPtr MergeTreeData::getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const { return getCombinedIndicesExpression(metadata_snapshot->getPrimaryKey(), metadata_snapshot->getSecondaryIndices(), metadata_snapshot->getColumns(), global_context); @@ -439,26 +464,14 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa checkKeyExpression(*new_partition_key.expression, new_partition_key.sample_block, "Partition", allow_nullable_key); - /// Reset filled fields - minmax_idx_columns.clear(); - minmax_idx_column_types.clear(); - minmax_idx_date_column_pos = -1; - minmax_idx_time_column_pos = -1; - /// Add all columns used in the partition key to the min-max index. - const NamesAndTypesList & minmax_idx_columns_with_types = new_partition_key.expression->getRequiredColumnsWithTypes(); - minmax_idx_expr = std::make_shared(std::make_shared(minmax_idx_columns_with_types)); - for (const NameAndTypePair & column : minmax_idx_columns_with_types) - { - minmax_idx_columns.emplace_back(column.name); - minmax_idx_column_types.emplace_back(column.type); - } + DataTypes minmax_idx_columns_types = getMinMaxColumnsTypes(new_partition_key); /// Try to find the date column in columns used by the partition key (a common case). bool encountered_date_column = false; - for (size_t i = 0; i < minmax_idx_column_types.size(); ++i) + for (size_t i = 0; i < minmax_idx_columns_types.size(); ++i) { - if (typeid_cast(minmax_idx_column_types[i].get())) + if (typeid_cast(minmax_idx_columns_types[i].get())) { if (!encountered_date_column) { @@ -474,9 +487,9 @@ void MergeTreeData::checkPartitionKeyAndInitMinMax(const KeyDescription & new_pa } if (!encountered_date_column) { - for (size_t i = 0; i < minmax_idx_column_types.size(); ++i) + for (size_t i = 0; i < minmax_idx_columns_types.size(); ++i) { - if (typeid_cast(minmax_idx_column_types[i].get())) + if (typeid_cast(minmax_idx_columns_types[i].get())) { if (!encountered_date_column) { @@ -3508,7 +3521,7 @@ bool MergeTreeData::isPrimaryOrMinMaxKeyColumnPossiblyWrappedInFunctions( if (column_name == name) return true; - for (const auto & name : minmax_idx_columns) + for (const auto & name : getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())) if (column_name == name) return true; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index f03f3f1dd8c..6f0ca34e753 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -692,12 +692,13 @@ public: bool is_custom_partitioned = false; - ExpressionActionsPtr minmax_idx_expr; - Names minmax_idx_columns; - DataTypes minmax_idx_column_types; Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column. Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column. + ExpressionActionsPtr getMinMaxExpr(const KeyDescription & partition_key) const; + Names getMinMaxColumnsNames(const KeyDescription & partition_key) const; + DataTypes getMinMaxColumnsTypes(const KeyDescription & partition_key) const; + ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index c571a53d4c8..f2f8172837c 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -1779,7 +1779,7 @@ void MergeTreeDataMergerMutator::mutateAllPartColumns( Block block; while (checkOperationIsNotCanceled(merge_entry) && (block = mutating_stream->read())) { - minmax_idx.update(block, data.minmax_idx_columns); + minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); out.write(block); merge_entry->rows_written += block.rows(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index d23413f4a84..c309bdd533f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -242,16 +242,21 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( std::optional minmax_idx_condition; std::optional partition_pruner; - if (data.minmax_idx_expr) + DataTypes minmax_columns_types; + if (metadata_snapshot->hasPartitionKey()) { - minmax_idx_condition.emplace(query_info, context, data.minmax_idx_columns, data.minmax_idx_expr); + const auto & partition_key = metadata_snapshot->getPartitionKey(); + auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key); + minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); + + minmax_idx_condition.emplace(query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key)); partition_pruner.emplace(metadata_snapshot->getPartitionKey(), query_info, context, false /* strict */); if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) { String msg = "Neither MinMax index by columns ("; bool first = true; - for (const String & col : data.minmax_idx_columns) + for (const String & col : minmax_columns_names) { if (first) first = false; @@ -268,9 +273,9 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( const Context & query_context = context.hasQueryContext() ? context.getQueryContext() : context; if (query_context.getSettingsRef().allow_experimental_query_deduplication) - selectPartsToReadWithUUIDFilter(parts, part_values, minmax_idx_condition, partition_pruner, max_block_numbers_to_read, query_context); + selectPartsToReadWithUUIDFilter(parts, part_values, minmax_idx_condition, minmax_columns_types, partition_pruner, max_block_numbers_to_read, query_context); else - selectPartsToRead(parts, part_values, minmax_idx_condition, partition_pruner, max_block_numbers_to_read); + selectPartsToRead(parts, part_values, minmax_idx_condition, minmax_columns_types, partition_pruner, max_block_numbers_to_read); /// Sampling. @@ -1885,6 +1890,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, const std::optional & minmax_idx_condition, + const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read) const { @@ -1900,7 +1906,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( continue; if (minmax_idx_condition && !minmax_idx_condition->checkInHyperrectangle( - part->minmax_idx.hyperrectangle, data.minmax_idx_column_types).can_be_true) + part->minmax_idx.hyperrectangle, minmax_columns_types).can_be_true) continue; if (partition_pruner) @@ -1924,6 +1930,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, const std::optional & minmax_idx_condition, + const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, const Context & query_context) const @@ -1950,7 +1957,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( continue; if (minmax_idx_condition - && !minmax_idx_condition->checkInHyperrectangle(part->minmax_idx.hyperrectangle, data.minmax_idx_column_types) + && !minmax_idx_condition->checkInHyperrectangle(part->minmax_idx.hyperrectangle, minmax_columns_types) .can_be_true) continue; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 7692424dfb5..8efdcf35082 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -123,6 +123,7 @@ private: MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, const std::optional & minmax_idx_condition, + const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read) const; @@ -131,6 +132,7 @@ private: MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, const std::optional & minmax_idx_condition, + const DataTypes & minmax_columns_types, std::optional & partition_pruner, const PartitionIdToMaxBlock * max_block_numbers_to_read, const Context & query_context) const; diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 5a9bdd90bc8..f478cdba40a 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -268,7 +268,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa Int64 temp_index = data.insert_increment.get(); IMergeTreeDataPart::MinMaxIndex minmax_idx; - minmax_idx.update(block, data.minmax_idx_columns); + minmax_idx.update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); MergeTreePartition partition(std::move(block_with_partition.partition)); diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 7ddc8d93b03..4ca20572e90 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -191,7 +191,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor { MergedBlockOutputStream part_out(part, metadata_snapshot, block.getNamesAndTypesList(), {}, CompressionCodecFactory::instance().get("NONE", {})); - part->minmax_idx.update(block, storage.minmax_idx_columns); + part->minmax_idx.update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0); if (metadata_snapshot->hasSortingKey()) metadata_snapshot->getSortingKey().expression->execute(block); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp index d06706f9109..ac1c92849d5 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeTableMetadata.cpp @@ -26,7 +26,10 @@ static String formattedAST(const ASTPtr & ast) ReplicatedMergeTreeTableMetadata::ReplicatedMergeTreeTableMetadata(const MergeTreeData & data, const StorageMetadataPtr & metadata_snapshot) { if (data.format_version < MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING) - date_column = data.minmax_idx_columns[data.minmax_idx_date_column_pos]; + { + auto minmax_idx_column_names = data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey()); + date_column = minmax_idx_column_names[data.minmax_idx_date_column_pos]; + } const auto data_settings = data.getSettings(); sampling_expression = formattedAST(metadata_snapshot->getSamplingKeyAST()); diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 36cca55779d..45d569fc131 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -261,7 +261,8 @@ "00116_storage_set", "00083_create_merge_tree_zookeeper", "00062_replicated_merge_tree_alter_zookeeper", - "01720_constraints_complex_types" + "01720_constraints_complex_types", + "01747_alter_partition_key_enum_zookeeper" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed @@ -748,6 +749,7 @@ "01676_dictget_in_default_expression", "01700_system_zookeeper_path_in", "01715_background_checker_blather_zookeeper", + "01747_alter_partition_key_enum_zookeeper", "attach", "ddl_dictionaries", "dictionary", From ad4fd75fb4d6e1701a4af548eec2f4bbb3d30884 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 2 Mar 2021 18:53:06 +0800 Subject: [PATCH 491/510] mycli compatibility --- src/Functions/connectionID.cpp | 44 +++++++++++++++++++ src/Functions/globalVariable.cpp | 1 + .../registerFunctionsMiscellaneous.cpp | 2 + src/Functions/ya.make | 1 + src/Interpreters/ClientInfo.h | 3 ++ src/Server/MySQLHandler.cpp | 1 + src/Server/MySQLHandler.h | 2 +- 7 files changed, 53 insertions(+), 1 deletion(-) create mode 100644 src/Functions/connectionID.cpp diff --git a/src/Functions/connectionID.cpp b/src/Functions/connectionID.cpp new file mode 100644 index 00000000000..877a2fdf85d --- /dev/null +++ b/src/Functions/connectionID.cpp @@ -0,0 +1,44 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + /// Get the connection ID. It's used for MySQL handler only. + class FunctionConnectionID : public IFunction + { + public: + static constexpr auto name = "connectionID"; + + FunctionConnectionID(const Context & context_) : context(context_) {} + + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override + { + return result_type->createColumnConst(input_rows_count, context.getClientInfo().connection_id); + } + + private: + const Context & context; + }; + +} + +void registerFunctionConnectionID(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerAlias("connection_id", "connectionID"); +} + +} diff --git a/src/Functions/globalVariable.cpp b/src/Functions/globalVariable.cpp index 68b71eebff0..6482b3bb976 100644 --- a/src/Functions/globalVariable.cpp +++ b/src/Functions/globalVariable.cpp @@ -80,6 +80,7 @@ private: std::unordered_map global_variable_map = {{"max_allowed_packet", {std::make_shared(), 67108864}}, {"version", {std::make_shared(), "5.7.30"}}, + {"version_comment", {std::make_shared(), ""}}, {"transaction_isolation", {std::make_shared(), "READ-UNCOMMITTED"}}}; }; diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 3438145981b..592f0d6774d 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -69,6 +69,7 @@ void registerFunctionErrorCodeToName(FunctionFactory &); void registerFunctionTcpPort(FunctionFactory &); void registerFunctionByteSize(FunctionFactory &); void registerFunctionFile(FunctionFactory & factory); +void registerFunctionConnectionID(FunctionFactory & factory); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -138,6 +139,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionTcpPort(factory); registerFunctionByteSize(factory); registerFunctionFile(factory); + registerFunctionConnectionID(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index f8beaa8540c..7a4deae4d04 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -210,6 +210,7 @@ SRCS( cbrt.cpp coalesce.cpp concat.cpp + connectionID.cpp convertCharset.cpp cos.cpp cosh.cpp diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h index d2b7beb7d8c..bc471dc3aa4 100644 --- a/src/Interpreters/ClientInfo.h +++ b/src/Interpreters/ClientInfo.h @@ -84,6 +84,9 @@ public: String http_user_agent; String http_referer; + /// For mysql + UInt64 connection_id = 0; + /// Comma separated list of forwarded IP addresses (from X-Forwarded-For for HTTP interface). /// It's expected that proxy appends the forwarded address to the end of the list. /// The element can be trusted only if you trust the corresponding proxy. diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index ea2813cf639..75c88a6ff93 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -92,6 +92,7 @@ void MySQLHandler::run() connection_context.makeSessionContext(); connection_context.getClientInfo().interface = ClientInfo::Interface::MYSQL; connection_context.setDefaultFormat("MySQLWire"); + connection_context.getClientInfo().connection_id = connection_id; in = std::make_shared(socket()); out = std::make_shared(socket()); diff --git a/src/Server/MySQLHandler.h b/src/Server/MySQLHandler.h index 5568805cdfe..1418d068ffd 100644 --- a/src/Server/MySQLHandler.h +++ b/src/Server/MySQLHandler.h @@ -61,7 +61,7 @@ protected: std::shared_ptr packet_endpoint; private: - size_t connection_id = 0; + UInt64 connection_id = 0; size_t server_capability_flags = 0; size_t client_capability_flags = 0; From 4e33587043f9cc7b637207771c8a49da7c7c71be Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Mar 2021 13:57:09 +0300 Subject: [PATCH 492/510] Comments --- src/Storages/MergeTree/MergeTreeData.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 6f0ca34e753..e629536a106 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -692,11 +692,15 @@ public: bool is_custom_partitioned = false; + /// Used only for old syntax tables. Never changes after init. Int64 minmax_idx_date_column_pos = -1; /// In a common case minmax index includes a date column. Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column. + /// Get partition key expression on required columns ExpressionActionsPtr getMinMaxExpr(const KeyDescription & partition_key) const; + /// Get column names required for partition key Names getMinMaxColumnsNames(const KeyDescription & partition_key) const; + /// Get column types required for partition key DataTypes getMinMaxColumnsTypes(const KeyDescription & partition_key) const; ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; From 494c2dba6ca25006d51510f58bdb11844f5d87cb Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 2 Mar 2021 19:12:51 +0800 Subject: [PATCH 493/510] Fix error --- src/Columns/ColumnLowCardinality.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp index f434ff32f3a..e420fd78a39 100644 --- a/src/Columns/ColumnLowCardinality.cpp +++ b/src/Columns/ColumnLowCardinality.cpp @@ -313,7 +313,9 @@ void ColumnLowCardinality::compareColumn(const IColumn & rhs, size_t rhs_row_num bool ColumnLowCardinality::hasEqualValues() const { - return getDictionary().size() <= 1; + if (getDictionary().size() <= 1) + return true; + return getIndexes().hasEqualValues(); } void ColumnLowCardinality::getPermutationImpl(bool reverse, size_t limit, int nan_direction_hint, Permutation & res, const Collator * collator) const From 658a62b7fc99826d2240151403c6f1a7a6660095 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Mar 2021 14:54:01 +0300 Subject: [PATCH 494/510] Update version_date.tsv after release 21.2.5.5 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 4ba92864020..cd18e950905 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v21.2.5.5-stable 2021-03-02 v21.2.4.6-stable 2021-02-20 v21.2.3.15-stable 2021-02-14 v21.2.2.8-stable 2021-02-07 From 24860bd29175be6105f6e19cf4c5f23f74af854f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Mar 2021 15:17:44 +0300 Subject: [PATCH 495/510] Update version_date.tsv after release 21.1.6.13 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index cd18e950905..fe249db4b10 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -2,6 +2,7 @@ v21.2.5.5-stable 2021-03-02 v21.2.4.6-stable 2021-02-20 v21.2.3.15-stable 2021-02-14 v21.2.2.8-stable 2021-02-07 +v21.1.6.13-stable 2021-03-02 v21.1.5.4-stable 2021-02-20 v21.1.4.46-stable 2021-02-14 v21.1.3.32-stable 2021-02-03 From 4166ae1f45d1cfdb2601e6fae487db7f7dd37fb6 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Mar 2021 15:40:24 +0300 Subject: [PATCH 496/510] Restart timeout when we make progress --- src/Client/HedgedConnections.cpp | 14 ++++++++++++-- src/Client/HedgedConnections.h | 1 + src/Server/TCPHandler.cpp | 2 +- tests/integration/test_hedged_requests/test.py | 9 +++++++++ 4 files changed, 23 insertions(+), 3 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index 2c9c47ed767..fbc02197805 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -318,6 +318,7 @@ HedgedConnections::ReplicaLocation HedgedConnections::getReadyReplicaLocation(As { ReplicaLocation location = timeout_fd_to_replica_location[event_fd]; offset_states[location.offset].replicas[location.index].change_replica_timeout.reset(); + offset_states[location.offset].replicas[location.index].is_change_replica_timeout_expired = true; offset_states[location.offset].next_replica_in_process = true; offsets_queue.push(location.offset); startNewReplica(); @@ -370,12 +371,21 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli switch (packet.type) { case Protocol::Server::Data: - if (!offset_states[replica_location.offset].first_packet_of_data_received) + if (!offset_states[replica_location.offset].first_packet_of_data_received && packet.block.rows() > 0) processReceivedFirstDataPacket(replica_location); replica_with_last_received_packet = replica_location; break; - case Protocol::Server::PartUUIDs: case Protocol::Server::Progress: + /// If we haven't received the first data packet (except header), we have made + /// some progress and timeout hasn't expired yet, we restart timeout for changing replica. + if (!replica.is_change_replica_timeout_expired && !offset_states[replica_location.offset].first_packet_of_data_received && packet.progress.read_bytes > 0) + { + /// Restart change replica timeout. + replica.change_replica_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_data_timeout); + } + replica_with_last_received_packet = replica_location; + break; + case Protocol::Server::PartUUIDs: case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: case Protocol::Server::Extremes: diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index cd47e01db2f..4cca66f53e4 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -33,6 +33,7 @@ public: Connection * connection = nullptr; PacketReceiverPtr packet_receiver; TimerDescriptor change_replica_timeout; + bool is_change_replica_timeout_expired = false; }; struct OffsetState diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index b330724f334..5765c3ec43e 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -1415,7 +1415,7 @@ void TCPHandler::sendData(const Block & block) /// For testing hedged requests const Settings & settings = query_context->getSettingsRef(); - if (settings.sleep_in_send_data) + if (block.rows() > 0 && settings.sleep_in_send_data) { out->next(); std::chrono::seconds sec(settings.sleep_in_send_data); diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index de16b8d361a..5785a9d8f81 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -246,3 +246,12 @@ def test_receive_timeout2(started_cluster): time.sleep(2) check_query(expected_replica="node_2", receive_timeout=3) + +def test_long_query(started_cluster): + NODES['node'].query("INSERT INTO distributed select number, toDate(number) from numbers(100);") + while TSV(NODES['node'].query("SELECT count() FROM distributed;")) != TSV("101"): + time.sleep(0.1) + + result = NODES['node'].query("select hostName(), max(id + sleep(1.5)) from distributed settings max_block_size = 1, max_threads = 1;") + assert TSV(result) == TSV("node_1\t99") + From d9e22ba646d9946a4c3e3abee39a1842b552dfc3 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 2 Mar 2021 15:53:39 +0300 Subject: [PATCH 497/510] Update version_date.tsv after release 20.12.8.5 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index fe249db4b10..7a3e315aa69 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -7,6 +7,7 @@ v21.1.5.4-stable 2021-02-20 v21.1.4.46-stable 2021-02-14 v21.1.3.32-stable 2021-02-03 v21.1.2.15-stable 2021-01-18 +v20.12.8.5-stable 2021-03-02 v20.12.7.3-stable 2021-02-20 v20.12.6.29-stable 2021-02-14 v20.12.5.18-stable 2021-02-03 From 134216dc4ae05cd04eb1a0c3c289aa839248d9fb Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sun, 28 Feb 2021 16:28:30 +0800 Subject: [PATCH 498/510] Specialize intDiv/module --- src/Functions/intDiv.cpp | 18 +++++++++++++++++- src/Functions/modulo.cpp | 20 ++++++++++++++++++-- 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/src/Functions/intDiv.cpp b/src/Functions/intDiv.cpp index e13c46e9cd1..55396b1d1c7 100644 --- a/src/Functions/intDiv.cpp +++ b/src/Functions/intDiv.cpp @@ -24,9 +24,25 @@ template struct DivideIntegralByConstantImpl : BinaryOperation> { - using ResultType = typename DivideIntegralImpl::ResultType; + using Op = DivideIntegralImpl; + using ResultType = typename Op::ResultType; static const constexpr bool allow_fixed_string = false; + template + static void NO_INLINE process(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size) + { + if constexpr (op_case == OpCase::Vector) + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(a[i], b[i]); + else if constexpr (op_case == OpCase::LeftConstant) + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(*a, b[i]); + else + vectorConstant(a, *b, c, size); + } + + static ResultType process(A a, B b) { return Op::template apply(a, b); } + static NO_INLINE void vectorConstant(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size) { #pragma GCC diagnostic push diff --git a/src/Functions/modulo.cpp b/src/Functions/modulo.cpp index 388e6ab9fb9..d9bf74ccaf5 100644 --- a/src/Functions/modulo.cpp +++ b/src/Functions/modulo.cpp @@ -24,10 +24,26 @@ template struct ModuloByConstantImpl : BinaryOperation> { - using ResultType = typename ModuloImpl::ResultType; + using Op = ModuloImpl; + using ResultType = typename Op::ResultType; static const constexpr bool allow_fixed_string = false; - static NO_INLINE void vectorConstant(const A * __restrict src, B b, ResultType * __restrict dst, size_t size) + template + static void NO_INLINE process(const A * __restrict a, const B * __restrict b, ResultType * __restrict c, size_t size) + { + if constexpr (op_case == OpCase::Vector) + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(a[i], b[i]); + else if constexpr (op_case == OpCase::LeftConstant) + for (size_t i = 0; i < size; ++i) + c[i] = Op::template apply(*a, b[i]); + else + vectorConstant(a, *b, c, size); + } + + static ResultType process(A a, B b) { return Op::template apply(a, b); } + + static void NO_INLINE vectorConstant(const A * __restrict src, B b, ResultType * __restrict dst, size_t size) { #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wsign-compare" From 4269eaa25275321071e168ade603714bc3e957ae Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Tue, 2 Mar 2021 17:05:33 +0300 Subject: [PATCH 499/510] Change tests --- .../integration/test_hedged_requests/test.py | 71 +++++++++++++++---- .../test_hedged_requests_parallel/test.py | 34 +++++++-- 2 files changed, 88 insertions(+), 17 deletions(-) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 5785a9d8f81..93f48e642ed 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -70,6 +70,19 @@ def check_query(expected_replica, receive_timeout=300): assert query_time < 10 +def check_settings(node_name, sleep_in_send_tables_status, sleep_in_send_data): + attempts = 0 + while attempts < 1000: + setting1 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_tables_status'") + setting2 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_data'") + if int(setting1) == sleep_in_send_tables_status and int(setting2) == sleep_in_send_data: + return + time.sleep(0.1) + attempts += 1 + + assert attempts < 1000 + + def test_stuck_replica(started_cluster): cluster.pause_container("node_1") check_query(expected_replica="node_2") @@ -88,10 +101,14 @@ def test_send_table_status_sleep(started_cluster): NODES['node_3'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + + check_settings('node_1', sleep_time, 0) + check_settings('node_2', 0, 0) + check_settings('node_3', 0, 0) - time.sleep(2) check_query(expected_replica="node_2") + def test_send_table_status_sleep2(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', @@ -104,8 +121,11 @@ def test_send_table_status_sleep2(started_cluster): NODES['node_3'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + + check_settings('node_1', sleep_time, 0) + check_settings('node_2', sleep_time, 0) + check_settings('node_3', 0, 0) - time.sleep(2) check_query(expected_replica="node_3") @@ -122,8 +142,10 @@ def test_send_data(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) - time.sleep(2) - + check_settings('node_1', 0, sleep_time) + check_settings('node_2', 0, 0) + check_settings('node_3', 0, 0) + check_query(expected_replica="node_2") @@ -139,8 +161,11 @@ def test_send_data2(started_cluster): NODES['node_3'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + + check_settings('node_1', 0, sleep_time) + check_settings('node_2', 0, sleep_time) + check_settings('node_3', 0, 0) - time.sleep(2) check_query(expected_replica="node_3") @@ -152,8 +177,15 @@ def test_combination1(started_cluster): NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + + NODES['node_3'].replace_config( + '/etc/clickhouse-server/users.d/users1.xml', + config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) + + check_settings('node_1', sleep_time, 0) + check_settings('node_2', 0, sleep_time) + check_settings('node_3', 0, 0) - time.sleep(2) check_query(expected_replica="node_3") @@ -170,7 +202,10 @@ def test_combination2(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=0)) - time.sleep(2) + check_settings('node_1', 0, sleep_time) + check_settings('node_2', sleep_time, 0) + check_settings('node_3', 0, 0) + check_query(expected_replica="node_3") @@ -186,8 +221,11 @@ def test_combination3(started_cluster): NODES['node_3'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) + + check_settings('node_1', 0, sleep_time) + check_settings('node_2', 1, 0) + check_settings('node_3', 0, sleep_time) - time.sleep(2) check_query(expected_replica="node_2") @@ -204,7 +242,10 @@ def test_combination4(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0)) - time.sleep(2) + check_settings('node_1', 1, sleep_time) + check_settings('node_2', 1, 0) + check_settings('node_3', 2, 0) + check_query(expected_replica="node_2") @@ -222,8 +263,11 @@ def test_receive_timeout1(started_cluster): NODES['node_3'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=1)) + + check_settings('node_1', 3, 0) + check_settings('node_2', 3, 0) + check_settings('node_3', 0, 1) - time.sleep(2) check_query(expected_replica="node_3", receive_timeout=2) @@ -243,7 +287,10 @@ def test_receive_timeout2(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=2, sleep_in_send_data=0)) - time.sleep(2) + check_settings('node_1', 0, 4) + check_settings('node_2', 2, 0) + check_settings('node_3', 2, 0) + check_query(expected_replica="node_2", receive_timeout=3) @@ -254,4 +301,4 @@ def test_long_query(started_cluster): result = NODES['node'].query("select hostName(), max(id + sleep(1.5)) from distributed settings max_block_size = 1, max_threads = 1;") assert TSV(result) == TSV("node_1\t99") - + diff --git a/tests/integration/test_hedged_requests_parallel/test.py b/tests/integration/test_hedged_requests_parallel/test.py index b713cf14af4..17db4af5d41 100644 --- a/tests/integration/test_hedged_requests_parallel/test.py +++ b/tests/integration/test_hedged_requests_parallel/test.py @@ -68,6 +68,19 @@ def check_query(): assert query_time < 5 +def check_settings(node_name, sleep_in_send_tables_status, sleep_in_send_data): + attempts = 0 + while attempts < 1000: + setting1 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_tables_status'") + setting2 = NODES[node_name].http_query("SELECT value FROM system.settings WHERE name='sleep_in_send_data'") + if int(setting1) == sleep_in_send_tables_status and int(setting2) == sleep_in_send_data: + return + time.sleep(0.1) + attempts += 1 + + assert attempts < 1000 + + def test_send_table_status_sleep(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', @@ -76,8 +89,10 @@ def test_send_table_status_sleep(started_cluster): NODES['node_2'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=sleep_time, sleep_in_send_data=0)) + + check_settings('node_1', sleep_time, 0) + check_settings('node_2', sleep_time, 0) - time.sleep(2) check_query() @@ -90,7 +105,9 @@ def test_send_data(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) - time.sleep(2) + check_settings('node_1', 0, sleep_time) + check_settings('node_2', 0, sleep_time) + check_query() @@ -107,7 +124,10 @@ def test_combination1(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=0, sleep_in_send_data=sleep_time)) - time.sleep(2) + check_settings('node_1', 1, 0) + check_settings('node_2', 1, 0) + check_settings('node_3', 0, sleep_time) + check_query() @@ -128,7 +148,11 @@ def test_combination2(started_cluster): '/etc/clickhouse-server/users.d/users1.xml', config.format(sleep_in_send_tables_status=1, sleep_in_send_data=0)) - time.sleep(2) + + check_settings('node_1', 0, sleep_time) + check_settings('node_2', 1, 0) + check_settings('node_3', 0, sleep_time) + check_settings('node_4', 1, 0) + check_query() - From 9411d496d9957419b0a6cd2928ba1447d32eb5d3 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 2 Mar 2021 23:17:02 +0800 Subject: [PATCH 500/510] Fix build --- src/Functions/connectionID.cpp | 47 ++++++++++++++++------------------ 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/src/Functions/connectionID.cpp b/src/Functions/connectionID.cpp index 877a2fdf85d..8e9c81aed6c 100644 --- a/src/Functions/connectionID.cpp +++ b/src/Functions/connectionID.cpp @@ -6,34 +6,31 @@ namespace DB { -namespace + +/// Get the connection ID. It's used for MySQL handler only. +class FunctionConnectionID : public IFunction { - /// Get the connection ID. It's used for MySQL handler only. - class FunctionConnectionID : public IFunction +public: + static constexpr auto name = "connectionID"; + + explicit FunctionConnectionID(const Context & context_) : context(context_) {} + + static FunctionPtr create(const Context & context) { return std::make_shared(context); } + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 0; } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override { - public: - static constexpr auto name = "connectionID"; + return result_type->createColumnConst(input_rows_count, context.getClientInfo().connection_id); + } - FunctionConnectionID(const Context & context_) : context(context_) {} - - static FunctionPtr create(const Context & context) { return std::make_shared(context); } - - String getName() const override { return name; } - - size_t getNumberOfArguments() const override { return 0; } - - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return std::make_shared(); } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr & result_type, size_t input_rows_count) const override - { - return result_type->createColumnConst(input_rows_count, context.getClientInfo().connection_id); - } - - private: - const Context & context; - }; - -} +private: + const Context & context; +}; void registerFunctionConnectionID(FunctionFactory & factory) { From c29d7c7f4959c406c67f4fdca8901c7333f4f7fb Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 2 Mar 2021 19:13:36 +0300 Subject: [PATCH 501/510] Shutup clang tidy --- src/Storages/MergeTree/MergeTreeData.cpp | 6 +++--- src/Storages/MergeTree/MergeTreeData.h | 6 +++--- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 4 ++-- 4 files changed, 9 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index caba9709785..277f525f4d9 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -423,7 +423,7 @@ ExpressionActionsPtr getCombinedIndicesExpression( } -ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partition_key) const +ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partition_key) { NamesAndTypesList partition_key_columns; if (!partition_key.column_names.empty()) @@ -432,14 +432,14 @@ ExpressionActionsPtr MergeTreeData::getMinMaxExpr(const KeyDescription & partiti return std::make_shared(std::make_shared(partition_key_columns)); } -Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) const +Names MergeTreeData::getMinMaxColumnsNames(const KeyDescription & partition_key) { if (!partition_key.column_names.empty()) return partition_key.expression->getRequiredColumns(); return {}; } -DataTypes MergeTreeData::getMinMaxColumnsTypes(const KeyDescription & partition_key) const +DataTypes MergeTreeData::getMinMaxColumnsTypes(const KeyDescription & partition_key) { if (!partition_key.column_names.empty()) return partition_key.expression->getRequiredColumnsWithTypes().getTypes(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index e629536a106..bfc1a5f6b6f 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -697,11 +697,11 @@ public: Int64 minmax_idx_time_column_pos = -1; /// In other cases, minmax index often includes a dateTime column. /// Get partition key expression on required columns - ExpressionActionsPtr getMinMaxExpr(const KeyDescription & partition_key) const; + static ExpressionActionsPtr getMinMaxExpr(const KeyDescription & partition_key); /// Get column names required for partition key - Names getMinMaxColumnsNames(const KeyDescription & partition_key) const; + static Names getMinMaxColumnsNames(const KeyDescription & partition_key); /// Get column types required for partition key - DataTypes getMinMaxColumnsTypes(const KeyDescription & partition_key) const; + static DataTypes getMinMaxColumnsTypes(const KeyDescription & partition_key); ExpressionActionsPtr getPrimaryKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; ExpressionActionsPtr getSortingKeyAndSkipIndicesExpression(const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c309bdd533f..b1f3f524beb 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1892,7 +1892,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const + const PartitionIdToMaxBlock * max_block_numbers_to_read) { auto prev_parts = parts; parts.clear(); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8efdcf35082..634719639ad 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -119,13 +119,13 @@ private: /// Select the parts in which there can be data that satisfy `minmax_idx_condition` and that match the condition on `_part`, /// as well as `max_block_number_to_read`. - void selectPartsToRead( + static void selectPartsToRead( MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, const std::optional & minmax_idx_condition, const DataTypes & minmax_columns_types, std::optional & partition_pruner, - const PartitionIdToMaxBlock * max_block_numbers_to_read) const; + const PartitionIdToMaxBlock * max_block_numbers_to_read); /// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded. void selectPartsToReadWithUUIDFilter( From 08148e062f1c7ccc84c2e6d3cbd4f34ed3cb0faf Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Tue, 2 Mar 2021 19:21:30 +0300 Subject: [PATCH 502/510] Fix a rare false negative in perf tests --- docker/test/performance-comparison/compare.sh | 25 ++++++++++++++++--- docker/test/performance-comparison/eqmed.sql | 15 ++++++++--- tests/performance/modulo.xml | 3 --- 3 files changed, 33 insertions(+), 10 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index 2b19a5e75a8..4d862cf987e 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -358,6 +358,8 @@ mkdir analyze analyze/tmp ||: build_log_column_definitions # Split the raw test output into files suitable for analysis. +# To debug calculations only for a particular test, substitute a suitable +# wildcard here, e.g. `for test_file in modulo-raw.tsv`. for test_file in *-raw.tsv do test_name=$(basename "$test_file" "-raw.tsv") @@ -467,7 +469,13 @@ create view broken_queries as create table query_run_metrics_for_stats engine File( TSV, -- do not add header -- will parse with grep 'analyze/query-run-metrics-for-stats.tsv') - as select test, query_index, 0 run, version, metric_values + as select test, query_index, 0 run, version, + -- For debugging, add a filter for a particular metric like this: + -- arrayFilter(m, n -> n = 'client_time', metric_values, metric_names) + -- metric_values + -- Note that further reporting may break, because the metric names are + -- not filtered. + metric_values from query_run_metric_arrays where (test, query_index) not in broken_queries order by test, query_index, run, version @@ -585,8 +593,19 @@ create view query_metric_stats as -- Main statistics for queries -- query time as reported in query log. create table queries engine File(TSVWithNamesAndTypes, 'report/queries.tsv') as select - abs(diff) > report_threshold and abs(diff) > stat_threshold as changed_fail, - abs(diff) > report_threshold - 0.05 and abs(diff) > stat_threshold as changed_show, + -- It is important to have a non-strict inequality with stat_threshold + -- here. The randomization distribution is actually discrete, and when + -- the number of runs is small, the quantile we need (e.g. 0.99) turns + -- out to be the maximum value of the distribution. We can also hit this + -- maximum possible value with our test run, and this obviously means + -- that we have observed the difference to the best precision possible + -- for the given number of runs. If we use a strict equality here, we + -- will miss such cases. This happened in the wild and lead to some + -- uncaught regressions, because for the default 7 runs we do for PRs, + -- the randomization distribution has only 16 values, so the max quantile + -- is actually 0.9375. + abs(diff) > report_threshold and abs(diff) >= stat_threshold as changed_fail, + abs(diff) > report_threshold - 0.05 and abs(diff) >= stat_threshold as changed_show, not changed_fail and stat_threshold > report_threshold + 0.10 as unstable_fail, not changed_show and stat_threshold > report_threshold - 0.05 as unstable_show, diff --git a/docker/test/performance-comparison/eqmed.sql b/docker/test/performance-comparison/eqmed.sql index 139f0758798..d0111550ee6 100644 --- a/docker/test/performance-comparison/eqmed.sql +++ b/docker/test/performance-comparison/eqmed.sql @@ -1,4 +1,6 @@ --- input is table(test text, query text, run UInt32, version int, metrics Array(float)) +-- The input is table(test text, query text, run UInt32, version UInt8, metrics Array(float)). +-- Run like this: +-- clickhouse-local --queries-file eqmed.sql -S 'test text, query text, run UInt32, version UInt8, metrics Array(float)' --file analyze/tmp/modulo_0.tsv select arrayMap(x -> floor(x, 4), original_medians_array.medians_by_version[1] as l) l_rounded, arrayMap(x -> floor(x, 4), original_medians_array.medians_by_version[2] as r) r_rounded, @@ -8,14 +10,19 @@ select from ( -- quantiles of randomization distributions + -- note that for small number of runs, the exact quantile might not make + -- sense, because the last possible value of randomization distribution + -- might take a larger percentage of distirbution (i.e. the distribution + -- actually has discrete values, and the last step can be large). select quantileExactForEach(0.99)( arrayMap(x, y -> abs(x - y), metrics_by_label[1], metrics_by_label[2]) as d ) threshold - ---- uncomment to see what the distribution is really like - --, uniqExact(d.1) u + ---- Uncomment to see what the distribution is really like. This debug + ---- code only works for single (the first) metric. + --, uniqExact(d[1]) u --, arraySort(x->x.1, -- arrayZip( - -- (sumMap([d.1], [1]) as f).1, + -- (sumMap([d[1]], [1]) as f).1, -- f.2)) full_histogram from ( diff --git a/tests/performance/modulo.xml b/tests/performance/modulo.xml index 77b544ff389..7c33855ff32 100644 --- a/tests/performance/modulo.xml +++ b/tests/performance/modulo.xml @@ -1,7 +1,4 @@ - - - SELECT number % 128 FROM numbers(300000000) FORMAT Null SELECT number % 255 FROM numbers(300000000) FORMAT Null SELECT number % 256 FROM numbers(300000000) FORMAT Null From 1278b5c71a485971d812d39c22c6c4a6954e6407 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Tue, 2 Mar 2021 16:29:11 -0400 Subject: [PATCH 503/510] test for issue#21369 --- ...1747_join_view_filter_dictionary.reference | 2 + .../01747_join_view_filter_dictionary.sql | 49 +++++++++++++++++++ 2 files changed, 51 insertions(+) create mode 100644 tests/queries/0_stateless/01747_join_view_filter_dictionary.reference create mode 100644 tests/queries/0_stateless/01747_join_view_filter_dictionary.sql diff --git a/tests/queries/0_stateless/01747_join_view_filter_dictionary.reference b/tests/queries/0_stateless/01747_join_view_filter_dictionary.reference new file mode 100644 index 00000000000..24ed2375adf --- /dev/null +++ b/tests/queries/0_stateless/01747_join_view_filter_dictionary.reference @@ -0,0 +1,2 @@ +name test 33 1.2 +33 diff --git a/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql b/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql new file mode 100644 index 00000000000..b43fa336485 --- /dev/null +++ b/tests/queries/0_stateless/01747_join_view_filter_dictionary.sql @@ -0,0 +1,49 @@ +drop table if exists summing_table01747; +drop view if exists rates01747; +drop view if exists agg_view01747; +drop table if exists dictst01747; +drop DICTIONARY if exists default.dict01747; + +CREATE TABLE summing_table01747 + ( + some_name String, + user_id UInt64, + amount Int64, + currency String + ) +ENGINE = SummingMergeTree() +ORDER BY (some_name); + +CREATE VIEW rates01747 AS + SELECT 'USD' as from_currency, 'EUR' as to_currency, 1.2 as rates01747; + +insert into summing_table01747 values ('name', 2, 20, 'USD'),('name', 1, 10, 'USD'); + +create table dictst01747(some_name String, field1 String, field2 UInt8) Engine = Memory +as select 'name', 'test', 33; + +CREATE DICTIONARY default.dict01747 (some_name String, field1 String, field2 UInt8) +PRIMARY KEY some_name SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 +TABLE dictst01747 DB currentDatabase() USER 'default')) +LIFETIME(MIN 0 MAX 0) LAYOUT(COMPLEX_KEY_HASHED()); + + +CREATE VIEW agg_view01747 AS + SELECT + summing_table01747.some_name as some_name, + dictGet('default.dict01747', 'field1', tuple(some_name)) as field1, + dictGet('default.dict01747', 'field2', tuple(some_name)) as field2, + rates01747.rates01747 as rates01747 + FROM summing_table01747 + ANY LEFT JOIN rates01747 + ON rates01747.from_currency = summing_table01747.currency; + +select * from agg_view01747; + +SELECT field2 FROM agg_view01747 WHERE field1 = 'test'; + +drop table summing_table01747; +drop view rates01747; +drop view agg_view01747; +drop table dictst01747; +drop DICTIONARY default.dict01747; From 82a6be44cfb06d644e5ea33a3089366d41246b0f Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 3 Mar 2021 01:03:03 +0300 Subject: [PATCH 504/510] Disable changing replica when we have made progress --- src/Client/HedgedConnections.cpp | 34 ++++++++++++++++++-------------- src/Client/HedgedConnections.h | 30 +++++++++++++++------------- src/Core/Settings.h | 3 ++- 3 files changed, 37 insertions(+), 30 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index fbc02197805..c197671cdfa 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -47,7 +47,7 @@ HedgedConnections::HedgedConnections( } active_connection_count = connections.size(); - offsets_with_received_first_data_packet = 0; + offsets_with_disabled_changing_replica = 0; pipeline_for_new_replicas.add([throttler_](ReplicaState & replica_) { replica_.connection->setThrottler(throttler_); }); } @@ -371,17 +371,22 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli switch (packet.type) { case Protocol::Server::Data: - if (!offset_states[replica_location.offset].first_packet_of_data_received && packet.block.rows() > 0) - processReceivedFirstDataPacket(replica_location); + /// If we received the first not empty data packet and still can change replica, + /// disable changing replica with this offset. + if (offset_states[replica_location.offset].can_change_replica && packet.block.rows() > 0) + disableChangingReplica(replica_location); replica_with_last_received_packet = replica_location; break; case Protocol::Server::Progress: - /// If we haven't received the first data packet (except header), we have made - /// some progress and timeout hasn't expired yet, we restart timeout for changing replica. - if (!replica.is_change_replica_timeout_expired && !offset_states[replica_location.offset].first_packet_of_data_received && packet.progress.read_bytes > 0) + /// Check if we have made some progress and still can change replica. + if (offset_states[replica_location.offset].can_change_replica && packet.progress.read_bytes > 0) { - /// Restart change replica timeout. - replica.change_replica_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_data_timeout); + /// If we are allowed to change replica until the first data packet, + /// just restart timeout (if it hasn't expired yet). Otherwise disable changing replica with this offset. + if (settings.allow_changeing_replica_until_first_data_packet && !replica.is_change_replica_timeout_expired) + replica.change_replica_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_data_timeout); + else + disableChangingReplica(replica_location); } replica_with_last_received_packet = replica_location; break; @@ -406,14 +411,13 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli return packet; } -void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & replica_location) +void HedgedConnections::disableChangingReplica(const ReplicaLocation & replica_location) { - /// When we receive first packet of data from replica, we stop working with replicas, that are - /// responsible for the same offset. + /// Stop working with replicas, that are responsible for the same offset. OffsetState & offset_state = offset_states[replica_location.offset]; offset_state.replicas[replica_location.index].change_replica_timeout.reset(); - ++offsets_with_received_first_data_packet; - offset_state.first_packet_of_data_received = true; + ++offsets_with_disabled_changing_replica; + offset_state.can_change_replica = false; for (size_t i = 0; i != offset_state.replicas.size(); ++i) { @@ -424,8 +428,8 @@ void HedgedConnections::processReceivedFirstDataPacket(const ReplicaLocation & r } } - /// If we received data from replicas with all offsets, we need to stop choosing new replicas. - if (hedged_connections_factory.hasEventsInProcess() && offsets_with_received_first_data_packet == offset_states.size()) + /// If we disabled changing replica with all offsets, we need to stop choosing new replicas. + if (hedged_connections_factory.hasEventsInProcess() && offsets_with_disabled_changing_replica == offset_states.size()) { if (hedged_connections_factory.numberOfProcessingReplicas() > 0) epoll.remove(hedged_connections_factory.getFileDescriptor()); diff --git a/src/Client/HedgedConnections.h b/src/Client/HedgedConnections.h index 4cca66f53e4..f1675108349 100644 --- a/src/Client/HedgedConnections.h +++ b/src/Client/HedgedConnections.h @@ -14,10 +14,11 @@ namespace DB { -/** To receive data from multiple replicas (connections) from one shard asynchronously, +/** To receive data from multiple replicas (connections) from one shard asynchronously. * The principe of Hedged Connections is used to reduce tail latency: - * (if we don't receive data from replica for a long time, we try to get new replica - * and send query to it, without cancelling working with previous replica). This class + * if we don't receive data from replica and there is no progress in query execution + * for a long time, we try to get new replica and send query to it, + * without cancelling working with previous replica. This class * supports all functionality that MultipleConnections has. */ class HedgedConnections : public IConnections @@ -40,11 +41,11 @@ public: { /// Replicas with the same offset. std::vector replicas; - /// An amount of active replicas, when first_packet_of_data_received is true, - /// active_connection_count is always <= 1 (because we stop working with - /// other replicas when we receive first data packet from one of them) + /// An amount of active replicas. When can_change_replica is false, + /// active_connection_count is always <= 1 (because we stopped working with + /// other replicas with the same offset) size_t active_connection_count = 0; - bool first_packet_of_data_received = false; + bool can_change_replica = true; /// This flag is true when this offset is in queue for /// new replicas. It's needed to process receive timeout @@ -102,8 +103,9 @@ public: bool hasActiveConnections() const override { return active_connection_count > 0; } private: - /// If we don't receive data from replica for receive_data_timeout, we are trying - /// to get new replica and send query to it. Beside sending query, there are some + /// If we don't receive data from replica and there is no progress in query + /// execution for receive_data_timeout, we are trying to get new + /// replica and send query to it. Beside sending query, there are some /// additional actions like sendScalarsData or sendExternalTablesData and we need /// to perform these actions in the same order on the new replica. So, we will /// save actions with replicas in pipeline to perform them on the new replicas. @@ -123,7 +125,7 @@ private: bool resumePacketReceiver(const ReplicaLocation & replica_location); - void processReceivedFirstDataPacket(const ReplicaLocation & replica_location); + void disableChangingReplica(const ReplicaLocation & replica_location); void startNewReplica(); @@ -156,10 +158,10 @@ private: /// The current number of valid connections to the replicas of this shard. size_t active_connection_count; - /// We count offsets which received first packet of data, - /// it's needed to cancel choosing new replicas when all offsets - /// received their first packet of data. - size_t offsets_with_received_first_data_packet; + /// We count offsets in which we can't change replica anymore, + /// it's needed to cancel choosing new replicas when we + /// disabled replica changing in all offsets. + size_t offsets_with_disabled_changing_replica; Pipeline pipeline_for_new_replicas; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 27fed7c2f8f..1326eb6f1c1 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -56,8 +56,9 @@ class IColumn; M(Seconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "", 0) \ M(Seconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes", 0) \ M(Milliseconds, hedged_connection_timeout, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \ - M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data from replica", 0) \ + M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \ M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \ + M(Bool, allow_changeing_replica_until_first_data_packet, false, "Allow HedgedConnections to change replica until receiving first data packet", 0) \ M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ M(Milliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \ M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \ From d1524b749eabbbc42eadbe1d99f5a9905c3c53dd Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 3 Mar 2021 01:24:41 +0300 Subject: [PATCH 505/510] Fix tests --- .../integration/test_hedged_requests/test.py | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/tests/integration/test_hedged_requests/test.py b/tests/integration/test_hedged_requests/test.py index 93f48e642ed..27fe7905b3a 100644 --- a/tests/integration/test_hedged_requests/test.py +++ b/tests/integration/test_hedged_requests/test.py @@ -36,7 +36,7 @@ def started_cluster(): NODES['node'].query('''CREATE TABLE distributed (id UInt32, date Date) ENGINE = Distributed('test_cluster', 'default', 'replicated')''') - NODES['node'].query("INSERT INTO distributed VALUES (1, '2020-01-01')") + NODES['node'].query("INSERT INTO distributed select number, toDate(number) from numbers(100);") yield cluster @@ -61,10 +61,10 @@ def check_query(expected_replica, receive_timeout=300): # with hedged requests it will last just around 1-2 second start = time.time() - result = NODES['node'].query("SELECT hostName(), id FROM distributed SETTINGS receive_timeout={}".format(receive_timeout)); + result = NODES['node'].query("SELECT hostName(), id FROM distributed ORDER BY id LIMIT 1 SETTINGS receive_timeout={}".format(receive_timeout)); query_time = time.time() - start - assert TSV(result) == TSV(expected_replica + "\t1") + assert TSV(result) == TSV(expected_replica + "\t0") print("Query time:", query_time) assert query_time < 10 @@ -89,6 +89,13 @@ def test_stuck_replica(started_cluster): cluster.unpause_container("node_1") +def test_long_query(started_cluster): + result = NODES['node'].query("select hostName(), max(id + sleep(1.5)) from distributed settings max_block_size = 1, max_threads = 1;") + assert TSV(result) == TSV("node_1\t99") + + NODES['node'].query("INSERT INTO distributed select number, toDate(number) from numbers(100);") + + def test_send_table_status_sleep(started_cluster): NODES['node_1'].replace_config( '/etc/clickhouse-server/users.d/users1.xml', @@ -293,12 +300,3 @@ def test_receive_timeout2(started_cluster): check_query(expected_replica="node_2", receive_timeout=3) - -def test_long_query(started_cluster): - NODES['node'].query("INSERT INTO distributed select number, toDate(number) from numbers(100);") - while TSV(NODES['node'].query("SELECT count() FROM distributed;")) != TSV("101"): - time.sleep(0.1) - - result = NODES['node'].query("select hostName(), max(id + sleep(1.5)) from distributed settings max_block_size = 1, max_threads = 1;") - assert TSV(result) == TSV("node_1\t99") - From a93e29ef4a20e3d81bd390822e1b22cead88dfbe Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 3 Mar 2021 02:00:19 +0300 Subject: [PATCH 506/510] Add a test for #14740 --- .../01747_transform_empty_arrays.reference | 1 + .../01747_transform_empty_arrays.sql | 27 +++++++++++++++++++ 2 files changed, 28 insertions(+) create mode 100644 tests/queries/0_stateless/01747_transform_empty_arrays.reference create mode 100644 tests/queries/0_stateless/01747_transform_empty_arrays.sql diff --git a/tests/queries/0_stateless/01747_transform_empty_arrays.reference b/tests/queries/0_stateless/01747_transform_empty_arrays.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/01747_transform_empty_arrays.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/01747_transform_empty_arrays.sql b/tests/queries/0_stateless/01747_transform_empty_arrays.sql new file mode 100644 index 00000000000..cc9a00a4555 --- /dev/null +++ b/tests/queries/0_stateless/01747_transform_empty_arrays.sql @@ -0,0 +1,27 @@ +SELECT * +FROM +( + WITH + + ( + SELECT groupArray(a) + FROM + ( + SELECT 1 AS a + ) + ) AS keys, + + ( + SELECT groupArray(a) + FROM + ( + SELECT 2 AS a + ) + ) AS values + SELECT * + FROM + ( + SELECT 1 AS a + ) + WHERE transform(a, keys, values, 0) +) AS wrap; From 019513d869aaa6aa18cf1bdcfbbee22a447aa065 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 3 Mar 2021 13:42:44 +0300 Subject: [PATCH 507/510] Try fix perftests. --- .../performance-comparison/config/config.d/user_files.xml | 7 +++++++ .../config/users.d/perf-comparison-tweaks-users.xml | 5 +++++ 2 files changed, 12 insertions(+) create mode 100644 docker/test/performance-comparison/config/config.d/user_files.xml diff --git a/docker/test/performance-comparison/config/config.d/user_files.xml b/docker/test/performance-comparison/config/config.d/user_files.xml new file mode 100644 index 00000000000..6611e986a90 --- /dev/null +++ b/docker/test/performance-comparison/config/config.d/user_files.xml @@ -0,0 +1,7 @@ + + + /var/lib/clickhouse/user_files/ + + + users.xml + \ No newline at end of file diff --git a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml index f3609bcfcdb..41bc7f777bf 100644 --- a/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml +++ b/docker/test/performance-comparison/config/users.d/perf-comparison-tweaks-users.xml @@ -19,4 +19,9 @@ 12 + + + 1 + +
From dc9dd8ccdcdc16fe35cab291152590b0e818dec8 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 3 Mar 2021 16:16:17 +0300 Subject: [PATCH 508/510] Fix typo --- src/Client/HedgedConnections.cpp | 2 +- src/Core/Settings.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/HedgedConnections.cpp b/src/Client/HedgedConnections.cpp index c197671cdfa..41397df2e79 100644 --- a/src/Client/HedgedConnections.cpp +++ b/src/Client/HedgedConnections.cpp @@ -383,7 +383,7 @@ Packet HedgedConnections::receivePacketFromReplica(const ReplicaLocation & repli { /// If we are allowed to change replica until the first data packet, /// just restart timeout (if it hasn't expired yet). Otherwise disable changing replica with this offset. - if (settings.allow_changeing_replica_until_first_data_packet && !replica.is_change_replica_timeout_expired) + if (settings.allow_changing_replica_until_first_data_packet && !replica.is_change_replica_timeout_expired) replica.change_replica_timeout.setRelative(hedged_connections_factory.getConnectionTimeouts().receive_data_timeout); else disableChangingReplica(replica_location); diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 1326eb6f1c1..8afc08da21a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -58,7 +58,7 @@ class IColumn; M(Milliseconds, hedged_connection_timeout, DBMS_DEFAULT_HEDGED_CONNECTION_TIMEOUT_MS, "Connection timeout for establishing connection with replica for Hedged requests", 0) \ M(Seconds, receive_data_timeout, DBMS_DEFAULT_RECEIVE_DATA_TIMEOUT_SEC, "Connection timeout for receiving first packet of data or packet with positive progress from replica", 0) \ M(Bool, use_hedged_requests, true, "Use hedged requests for distributed queries", 0) \ - M(Bool, allow_changeing_replica_until_first_data_packet, false, "Allow HedgedConnections to change replica until receiving first data packet", 0) \ + M(Bool, allow_changing_replica_until_first_data_packet, false, "Allow HedgedConnections to change replica until receiving first data packet", 0) \ M(Milliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.", 0) \ M(Milliseconds, connection_pool_max_wait_ms, 0, "The wait time when the connection pool is full.", 0) \ M(Milliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.", 0) \ From 6a6bc3924d2cfe09d42fea5b9e03e314d67cb352 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Wed, 3 Mar 2021 16:53:26 +0300 Subject: [PATCH 509/510] Update version_date.tsv after release 20.8.14.4 --- utils/list-versions/version_date.tsv | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 7a3e315aa69..3e63f8898c0 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -33,6 +33,7 @@ v20.9.5.5-stable 2020-11-13 v20.9.4.76-stable 2020-10-29 v20.9.3.45-stable 2020-10-09 v20.9.2.20-stable 2020-09-22 +v20.8.14.4-lts 2021-03-03 v20.8.13.15-lts 2021-02-20 v20.8.12.2-lts 2021-01-16 v20.8.11.17-lts 2020-12-25 From e8df9971f14880729fe20336388ea09e4dc44712 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 3 Mar 2021 18:12:39 +0300 Subject: [PATCH 510/510] Fix Arcadia --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index 4e523545938..a823ba38c1c 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -207,3 +207,4 @@ 01702_bitmap_native_integers 01686_event_time_microseconds_part_log 01017_uniqCombined_memory_usage +01747_join_view_filter_dictionary