From aad55ab55faa9ed64d802c73080db77503c6363d Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 12 Jun 2024 14:49:06 +0000 Subject: [PATCH 001/154] IO scheduling on HTTP session level --- base/poco/Net/include/Poco/Net/HTTPSession.h | 43 ++++++++++ base/poco/Net/src/HTTPSession.cpp | 24 ++++-- base/poco/Net/src/SocketImpl.cpp | 36 ++++----- src/Common/CurrentMetrics.cpp | 3 + src/Common/CurrentThread.cpp | 50 ++++++++++++ src/Common/CurrentThread.h | 43 +++++++++- src/Common/HTTPConnectionPool.cpp | 64 ++++++++++++++- src/Common/ProfileEvents.cpp | 7 ++ .../Scheduler/Nodes/tests/ResourceTest.h | 6 +- .../tests/gtest_dynamic_resource_manager.cpp | 4 +- .../Nodes/tests/gtest_resource_scheduler.cpp | 8 +- src/Common/Scheduler/ResourceGuard.h | 79 +++++++++++++++++-- src/Common/Scheduler/ResourceLink.h | 23 +++++- src/Common/ThreadStatus.h | 6 +- .../IO/ReadBufferFromAzureBlobStorage.cpp | 5 ++ .../IO/WriteBufferFromAzureBlobStorage.cpp | 6 +- .../ObjectStorages/DiskObjectStorage.cpp | 13 +-- src/IO/ReadBufferFromS3.cpp | 12 +-- src/IO/ReadSettings.h | 3 +- src/IO/WriteBufferFromS3.cpp | 13 +-- src/IO/WriteSettings.h | 3 +- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 8 +- .../HDFS/WriteBufferFromHDFS.cpp | 10 +-- 23 files changed, 379 insertions(+), 90 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index cac14f479db..b0e59443f9b 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -19,6 +19,8 @@ #include +#include +#include #include "Poco/Any.h" #include "Poco/Buffer.h" #include "Poco/Exception.h" @@ -33,6 +35,27 @@ namespace Net { + class IHTTPSessionDataHooks + /// Interface to control stream of data bytes being sent or received though socket by HTTPSession + /// It allows to monitor, throttle and schedule data streams with syscall granulatrity + { + public: + virtual ~IHTTPSessionDataHooks() = default; + + virtual void start(int bytes) = 0; + /// Called before sending/receiving data `bytes` to/from socket. + + virtual void finish(int bytes) = 0; + /// Called when sending/receiving of data `bytes` is successfully finished. + + virtual void fail() = 0; + /// If an error occured during send/receive `fail()` is called instead of `finish()`. + }; + + + using HTTPSessionDataHooksPtr = std::shared_ptr; + + class Net_API HTTPSession /// HTTPSession implements basic HTTP session management /// for both HTTP clients and HTTP servers. @@ -73,6 +96,12 @@ namespace Net Poco::Timespan getReceiveTimeout() const; /// Returns receive timeout for the HTTP session. + void setSendDataHooks(const HTTPSessionDataHooksPtr & sendDataHooks = {}); + /// Sets data hooks that will be called on every sent to the socket. + + void setReceiveDataHooks(const HTTPSessionDataHooksPtr & receiveDataHooks = {}); + /// Sets data hooks that will be called on every receive from the socket. + bool connected() const; /// Returns true if the underlying socket is connected. @@ -211,6 +240,10 @@ namespace Net Poco::Exception * _pException; Poco::Any _data; + // Data hooks + HTTPSessionDataHooksPtr _sendDataHooks; + HTTPSessionDataHooksPtr _receiveDataHooks; + friend class HTTPStreamBuf; friend class HTTPHeaderStreamBuf; friend class HTTPFixedLengthStreamBuf; @@ -246,6 +279,16 @@ namespace Net return _receiveTimeout; } + inline void HTTPSession::setSendDataHooks(const HTTPSessionDataHooksPtr & sendDataHooks) + { + _sendDataHooks = sendDataHooks; + } + + inline void HTTPSession::setReceiveDataHooks(const HTTPSessionDataHooksPtr & receiveDataHooks) + { + _receiveDataHooks = receiveDataHooks; + } + inline StreamSocket & HTTPSession::socket() { return _socket; diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 8f951b3102c..596185703fa 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -128,14 +128,14 @@ int HTTPSession::get() { if (_pCurrent == _pEnd) refill(); - + if (_pCurrent < _pEnd) return *_pCurrent++; else return std::char_traits::eof(); } - + int HTTPSession::peek() { if (_pCurrent == _pEnd) @@ -147,7 +147,7 @@ int HTTPSession::peek() return std::char_traits::eof(); } - + int HTTPSession::read(char* buffer, std::streamsize length) { if (_pCurrent < _pEnd) @@ -166,10 +166,17 @@ int HTTPSession::write(const char* buffer, std::streamsize length) { try { - return _socket.sendBytes(buffer, (int) length); + if (_sendDataHooks) + _sendDataHooks->start((int) length); + int result = _socket.sendBytes(buffer, (int) length); + if (_sendDataHooks) + _sendDataHooks->finish(result); + return result; } catch (Poco::Exception& exc) { + if (_sendDataHooks) + _sendDataHooks->fail(); setException(exc); throw; } @@ -180,10 +187,17 @@ int HTTPSession::receive(char* buffer, int length) { try { - return _socket.receiveBytes(buffer, length); + if (_receiveDataHooks) + _receiveDataHooks->start(length); + int result = _socket.receiveBytes(buffer, length); + if (_receiveDataHooks) + _receiveDataHooks->finish(result); + return result; } catch (Poco::Exception& exc) { + if (_receiveDataHooks) + _receiveDataHooks->fail(); setException(exc); throw; } diff --git a/base/poco/Net/src/SocketImpl.cpp b/base/poco/Net/src/SocketImpl.cpp index 484b8cfeec3..65456a287fb 100644 --- a/base/poco/Net/src/SocketImpl.cpp +++ b/base/poco/Net/src/SocketImpl.cpp @@ -62,7 +62,7 @@ bool checkIsBrokenTimeout() SocketImpl::SocketImpl(): _sockfd(POCO_INVALID_SOCKET), - _blocking(true), + _blocking(true), _isBrokenTimeout(checkIsBrokenTimeout()) { } @@ -81,7 +81,7 @@ SocketImpl::~SocketImpl() close(); } - + SocketImpl* SocketImpl::acceptConnection(SocketAddress& clientAddr) { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -117,7 +117,7 @@ void SocketImpl::connect(const SocketAddress& address) rc = ::connect(_sockfd, address.addr(), address.length()); } while (rc != 0 && lastError() == POCO_EINTR); - if (rc != 0) + if (rc != 0) { int err = lastError(); error(err, address.toString()); @@ -204,7 +204,7 @@ void SocketImpl::bind6(const SocketAddress& address, bool reuseAddress, bool reu #if defined(POCO_HAVE_IPv6) if (address.family() != SocketAddress::IPv6) throw Poco::InvalidArgumentException("SocketAddress must be an IPv6 address"); - + if (_sockfd == POCO_INVALID_SOCKET) { init(address.af()); @@ -225,11 +225,11 @@ void SocketImpl::bind6(const SocketAddress& address, bool reuseAddress, bool reu #endif } - + void SocketImpl::listen(int backlog) { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); - + int rc = ::listen(_sockfd, backlog); if (rc != 0) error(); } @@ -253,7 +253,7 @@ void SocketImpl::shutdownReceive() if (rc != 0) error(); } - + void SocketImpl::shutdownSend() { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -262,7 +262,7 @@ void SocketImpl::shutdownSend() if (rc != 0) error(); } - + void SocketImpl::shutdown() { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -317,7 +317,7 @@ int SocketImpl::receiveBytes(void* buffer, int length, int flags) throw TimeoutException(); } } - + int rc; do { @@ -325,7 +325,7 @@ int SocketImpl::receiveBytes(void* buffer, int length, int flags) rc = ::recv(_sockfd, reinterpret_cast(buffer), length, flags); } while (blocking && rc < 0 && lastError() == POCO_EINTR); - if (rc < 0) + if (rc < 0) { int err = lastError(); if ((err == POCO_EAGAIN || err == POCO_EWOULDBLOCK) && !blocking) @@ -363,7 +363,7 @@ int SocketImpl::receiveFrom(void* buffer, int length, SocketAddress& address, in throw TimeoutException(); } } - + sockaddr_storage abuffer; struct sockaddr* pSA = reinterpret_cast(&abuffer); poco_socklen_t saLen = sizeof(abuffer); @@ -450,7 +450,7 @@ bool SocketImpl::pollImpl(Poco::Timespan& remainingTime, int mode) } while (rc < 0 && lastError() == POCO_EINTR); if (rc < 0) error(); - return rc > 0; + return rc > 0; #else @@ -493,7 +493,7 @@ bool SocketImpl::pollImpl(Poco::Timespan& remainingTime, int mode) } while (rc < 0 && errorCode == POCO_EINTR); if (rc < 0) error(errorCode); - return rc > 0; + return rc > 0; #endif // POCO_HAVE_FD_POLL } @@ -503,13 +503,13 @@ bool SocketImpl::poll(const Poco::Timespan& timeout, int mode) Poco::Timespan remainingTime(timeout); return pollImpl(remainingTime, mode); } - + void SocketImpl::setSendBufferSize(int size) { setOption(SOL_SOCKET, SO_SNDBUF, size); } - + int SocketImpl::getSendBufferSize() { int result; @@ -523,7 +523,7 @@ void SocketImpl::setReceiveBufferSize(int size) setOption(SOL_SOCKET, SO_RCVBUF, size); } - + int SocketImpl::getReceiveBufferSize() { int result; @@ -569,7 +569,7 @@ Poco::Timespan SocketImpl::getReceiveTimeout() return result; } - + SocketAddress SocketImpl::address() { if (_sockfd == POCO_INVALID_SOCKET) throw InvalidSocketException(); @@ -580,7 +580,7 @@ SocketAddress SocketImpl::address() int rc = ::getsockname(_sockfd, pSA, &saLen); if (rc == 0) return SocketAddress(pSA, saLen); - else + else error(); return SocketAddress(); } diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 731c72d65f2..63156f5291d 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -291,6 +291,9 @@ M(DistrCacheWriteRequests, "Number of executed Write requests to Distributed Cache") \ M(DistrCacheServerConnections, "Number of open connections to ClickHouse server from Distributed Cache") \ \ + M(SchedulerIOReadScheduled, "Number of IO reads are being scheduled currently") \ + M(SchedulerIOWriteScheduled, "Number of IO writes are being scheduled currently") \ + \ M(StorageConnectionsStored, "Total count of sessions stored in the session pool for storages") \ M(StorageConnectionsTotal, "Total count of all sessions: stored in the pool and actively used right now for storages") \ \ diff --git a/src/Common/CurrentThread.cpp b/src/Common/CurrentThread.cpp index 70b69d6bcc7..ba7087ca7f1 100644 --- a/src/Common/CurrentThread.cpp +++ b/src/Common/CurrentThread.cpp @@ -113,6 +113,56 @@ std::string_view CurrentThread::getQueryId() return current_thread->getQueryId(); } +void CurrentThread::attachReadResource(ResourceLink link) +{ + if (unlikely(!current_thread)) + return; + if (current_thread->read_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has been already attached to read resource", std::to_string(getThreadId())); + current_thread->read_resource_link = link; +} + +void CurrentThread::detachReadResource() +{ + if (unlikely(!current_thread)) + return; + if (!current_thread->read_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has not been attached to read resource", std::to_string(getThreadId())); + current_thread->read_resource_link.reset(); +} + +ResourceLink CurrentThread::getReadResourceLink() +{ + if (unlikely(!current_thread)) + return {}; + return current_thread->read_resource_link; +} + +void CurrentThread::attachWriteResource(ResourceLink link) +{ + if (unlikely(!current_thread)) + return; + if (current_thread->write_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has been already attached to write resource", std::to_string(getThreadId())); + current_thread->write_resource_link = link; +} + +void CurrentThread::detachWriteResource() +{ + if (unlikely(!current_thread)) + return; + if (!current_thread->write_resource_link) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Thread #{} has not been attached to write resource", std::to_string(getThreadId())); + current_thread->write_resource_link.reset(); +} + +ResourceLink CurrentThread::getWriteResourceLink() +{ + if (unlikely(!current_thread)) + return {}; + return current_thread->write_resource_link; +} + MemoryTracker * CurrentThread::getUserMemoryTracker() { if (unlikely(!current_thread)) diff --git a/src/Common/CurrentThread.h b/src/Common/CurrentThread.h index 53b61ba315f..787e8369a83 100644 --- a/src/Common/CurrentThread.h +++ b/src/Common/CurrentThread.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -23,7 +24,6 @@ class QueryStatus; struct Progress; class InternalTextLogsQueue; - /** Collection of static methods to work with thread-local objects. * Allows to attach and detach query/process (thread group) to a thread * (to calculate query-related metrics and to allow to obtain query-related data from a thread). @@ -92,6 +92,14 @@ public: static std::string_view getQueryId(); + // For IO Scheduling + static void attachReadResource(ResourceLink link); + static void detachReadResource(); + static ResourceLink getReadResourceLink(); + static void attachWriteResource(ResourceLink link); + static void detachWriteResource(); + static ResourceLink getWriteResourceLink(); + /// Initializes query with current thread as master thread in constructor, and detaches it in destructor struct QueryScope : private boost::noncopyable { @@ -102,6 +110,39 @@ public: void logPeakMemoryUsage(); bool log_peak_memory_usage_in_destructor = true; }; + + /// Scoped attach/detach of IO resource links + struct IOScope : private boost::noncopyable + { + explicit IOScope(ResourceLink read_resource_link, ResourceLink write_resource_link) + { + if (read_resource_link) + { + attachReadResource(read_resource_link); + read_attached = true; + } + if (write_resource_link) + { + attachWriteResource(write_resource_link); + write_attached = true; + } + } + + explicit IOScope(const IOSchedulingSettings & settings) + : IOScope(settings.read_resource_link, settings.write_resource_link) + {} + + ~IOScope() + { + if (read_attached) + detachReadResource(); + if (write_attached) + detachWriteResource(); + } + + bool read_attached = false; + bool write_attached = false; + }; }; } diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 167aeee68f3..de7e10d044a 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include @@ -249,6 +250,54 @@ public: }; +// Session data hooks implementation for integration with resource scheduler. +// Hooks are created per every request-response pair and are registered/unregistered in HTTP session. +// * `start()` send resource request to the scheduler every time HTTP session is going to send or receive +// data to/from socket. `start()` waits for the scheduler confirmation. This way scheduler might +// throttle and/or schedule socket data streams. +// * `finish()` hook is called on successful socket read/write operation. +// It informs the scheduler that operation is complete, which allows the scheduler to control the total +// amount of in-flight bytes and/or operations. +// * `fail()` hook is called on failure of socket operation. The purpose is to correct the amount of bytes +// passed through the scheduler queue to ensure fair bandwidth allocation even in presence of errors. +struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks +{ + explicit ResourceGuardSessionDataHooks(const ResourceGuard::Metrics * metrics, ResourceLink link_) + : link(link_) + { + request.metrics = metrics; + chassert(link); + } + + ~ResourceGuardSessionDataHooks() override + { + request.assertFinished(); // Never destruct with an active request + } + + void start(int bytes) override + { + // TODO(serxa): add metrics here or better in scheduler code (e.g. during enqueue, or better in REsourceGuard::REquest)? + request.enqueue(bytes, link); + request.wait(); + } + + void finish(int bytes) override + { + request.finish(); + link.adjust(request.cost, bytes); // success + } + + void fail() override + { + request.finish(); + link.accumulate(request.cost); // We assume no resource was used in case of failure + } + + ResourceLink link; + ResourceGuard::Request request; +}; + + // EndpointConnectionPool manage connections to the endpoint // Features: // - it uses HostResolver for address selecting. See Common/HostResolver.h for more info. @@ -259,8 +308,6 @@ public: // - `Session::reconnect()` uses the pool as well // - comprehensive sensors // - session is reused according its inner state, automatically - - template class EndpointConnectionPool : public std::enable_shared_from_this>, public IExtendedPool { @@ -350,6 +397,19 @@ private: std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override { auto idle = idleTime(); + + // Reset data hooks for IO scheduling + if (ResourceLink link = CurrentThread::getReadResourceLink()) { + Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); + } else { + Session::setSendDataHooks(); + } + if (ResourceLink link = CurrentThread::getWriteResourceLink()) { + Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + } else { + Session::setReceiveDataHooks(); + } + std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index f73e16c517d..b305614e54c 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -104,6 +104,13 @@ M(PartsWithAppliedMutationsOnFly, "Total number of parts for which there was any mutation applied on fly") \ M(MutationsAppliedOnFlyInAllParts, "The sum of number of applied mutations on-fly for part among all read parts") \ \ + M(SchedulerIOReadRequests, "Resource requests passed through scheduler for IO reads.") \ + M(SchedulerIOReadBytes, "Bytes passed through scheduler for IO reads.") \ + M(SchedulerIOReadWaitMicroseconds, "Total time a query was waiting on resource requests for IO reads.") \ + M(SchedulerIOWriteRequests, "Resource requests passed through scheduler for IO writes.") \ + M(SchedulerIOWriteBytes, "Bytes passed through scheduler for IO writes.") \ + M(SchedulerIOWriteWaitMicroseconds, "Total time a query was waiting on resource requests for IO writes.") \ + \ M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \ \ M(ReplicatedPartFetches, "Number of times a data part was downloaded from replica of a ReplicatedMergeTree table.") \ diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index ea3f9edf765..a5eb98f2a2f 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -232,7 +232,7 @@ struct ResourceTestManager : public ResourceTestBase ResourceTestManager & t; Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(link_, cost, PostponeLocking) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, PostponeLocking) , t(t_) { t.onEnqueue(link); @@ -310,7 +310,7 @@ struct ResourceTestManager : public ResourceTestBase // NOTE: actually leader's request(s) make their own small busy period. void blockResource(ResourceLink link) { - ResourceGuard g(link, 1, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::PostponeLocking); g.lock(); // NOTE: at this point we assume resource to be blocked by single request (1) busy_period.arrive_and_wait(); // (1) notify all followers that resource is blocked @@ -320,7 +320,7 @@ struct ResourceTestManager : public ResourceTestBase { getLinkData(link).left += total_requests + 1; busy_period.arrive_and_wait(); // (1) wait leader to block resource - ResourceGuard g(link, cost, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::PostponeLocking); onEnqueue(link); busy_period.arrive_and_wait(); // (2) notify leader to unblock g.lock(); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 1901a4fd120..4ac79977663 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -36,11 +36,11 @@ TEST(SchedulerDynamicResourceManager, Smoke) for (int i = 0; i < 10; i++) { - ResourceGuard gA(cA->get("res1"), ResourceGuard::PostponeLocking); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), ResourceGuard::PostponeLocking); gA.lock(); gA.unlock(); - ResourceGuard gB(cB->get("res1")); + ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index f8196d15819..ba573bf0c85 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -109,22 +109,22 @@ TEST(SchedulerRoot, Smoke) r2.registerResource(); { - ResourceGuard rg(a); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); EXPECT_TRUE(fc1->requests.contains(&rg.request)); } { - ResourceGuard rg(b); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); EXPECT_TRUE(fc1->requests.contains(&rg.request)); } { - ResourceGuard rg(c); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); EXPECT_TRUE(fc2->requests.contains(&rg.request)); } { - ResourceGuard rg(d); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); EXPECT_TRUE(fc2->requests.contains(&rg.request)); } } diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 3c29f588fba..73aea4afdb6 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -7,10 +7,30 @@ #include #include +#include +#include +#include + #include #include +namespace ProfileEvents +{ + extern const Event SchedulerIOReadRequests; + extern const Event SchedulerIOReadBytes; + extern const Event SchedulerIOReadWaitMicroseconds; + extern const Event SchedulerIOWriteRequests; + extern const Event SchedulerIOWriteBytes; + extern const Event SchedulerIOWriteWaitMicroseconds; +} + +namespace CurrentMetrics +{ + extern const Metric SchedulerIOReadScheduled; + extern const Metric SchedulerIOWriteScheduled; +} + namespace DB { @@ -30,6 +50,36 @@ public: PostponeLocking /// Don't lock in constructor, but send request }; + struct Metrics + { + const ProfileEvents::Event requests = ProfileEvents::end(); + const ProfileEvents::Event cost = ProfileEvents::end(); + const ProfileEvents::Event wait_microseconds = ProfileEvents::end(); + const CurrentMetrics::Metric scheduled_count = CurrentMetrics::end(); + + static const Metrics * getIORead() + { + static Metrics metrics{ + .requests = ProfileEvents::SchedulerIOReadRequests, + .cost = ProfileEvents::SchedulerIOReadBytes, + .wait_microseconds = ProfileEvents::SchedulerIOReadWaitMicroseconds, + .scheduled_count = CurrentMetrics::SchedulerIOReadScheduled + }; + return &metrics; + } + + static const Metrics * getIOWrite() + { + static Metrics metrics{ + .requests = ProfileEvents::SchedulerIOWriteRequests, + .cost = ProfileEvents::SchedulerIOWriteBytes, + .wait_microseconds = ProfileEvents::SchedulerIOWriteWaitMicroseconds, + .scheduled_count = CurrentMetrics::SchedulerIOWriteScheduled + }; + return &metrics; + } + }; + enum RequestState { Finished, // Last request has already finished; no concurrent access is possible @@ -46,6 +96,8 @@ public: chassert(state == Finished); state = Enqueued; ResourceRequest::reset(cost_); + ProfileEvents::increment(metrics->requests); + ProfileEvents::increment(metrics->cost, cost_); link_.queue->enqueueRequestUsingBudget(this); } @@ -63,6 +115,8 @@ public: void wait() { + CurrentMetrics::Increment scheduled(metrics->scheduled_count); + auto timer = CurrentThread::getProfileEvents().timer(metrics->wait_microseconds); std::unique_lock lock(mutex); dequeued_cv.wait(lock, [this] { return state == Dequeued; }); } @@ -75,14 +129,23 @@ public: ResourceRequest::finish(); } - static Request & local() + void assertFinished() + { + // lock(mutex) is not required because `Finished` request cannot be used by the scheduler thread + chassert(state == Finished); + } + + static Request & local(const Metrics * metrics) { // Since single thread cannot use more than one resource request simultaneously, // we can reuse thread-local request to avoid allocations static thread_local Request instance; + instance.metrics = metrics; return instance; } + const Metrics * metrics = nullptr; // Must be initialized before use + private: std::mutex mutex; std::condition_variable dequeued_cv; @@ -90,13 +153,13 @@ public: }; /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) - explicit ResourceGuard(ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) + explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) : link(link_) - , request(Request::local()) + , request(Request::local(metrics)) { if (cost == 0) - link.queue = nullptr; // Ignore zero-cost requests - else if (link.queue) + link.reset(); // Ignore zero-cost requests + else if (link) { request.enqueue(cost, link); if (ctor == LockStraightAway) @@ -112,17 +175,17 @@ public: /// Blocks until resource is available void lock() { - if (link.queue) + if (link) request.wait(); } /// Report resource consumption has finished void unlock() { - if (link.queue) + if (link) { request.finish(); - link.queue = nullptr; + link.reset(); } } diff --git a/src/Common/Scheduler/ResourceLink.h b/src/Common/Scheduler/ResourceLink.h index 450d9bc1efa..6dd3be930ca 100644 --- a/src/Common/Scheduler/ResourceLink.h +++ b/src/Common/Scheduler/ResourceLink.h @@ -13,13 +13,32 @@ using ResourceCost = Int64; struct ResourceLink { ISchedulerQueue * queue = nullptr; + bool operator==(const ResourceLink &) const = default; + explicit operator bool() const { return queue != nullptr; } void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const; - void consumed(ResourceCost cost) const; - void accumulate(ResourceCost cost) const; + + void reset() + { + queue = nullptr; + } +}; + +/* + * Everything required for IO scheduling. + * Note that raw pointer are stored inside, so make sure that `ClassifierPtr` that produced + * resource links will outlive them. Usually classifier is stored in query `Context`. + */ +struct IOSchedulingSettings +{ + ResourceLink read_resource_link; + ResourceLink write_resource_link; + + bool operator==(const IOSchedulingSettings &) const = default; + explicit operator bool() const { return read_resource_link && write_resource_link; } }; } diff --git a/src/Common/ThreadStatus.h b/src/Common/ThreadStatus.h index 0c02ab8fdb0..6ea0a9a848c 100644 --- a/src/Common/ThreadStatus.h +++ b/src/Common/ThreadStatus.h @@ -7,11 +7,11 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -188,6 +188,10 @@ public: Progress progress_in; Progress progress_out; + /// IO scheduling + ResourceLink read_resource_link; + ResourceLink write_resource_link; + private: /// Group of threads, to which this thread attached ThreadGroupPtr thread_group; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index da1ea65f2ea..ecc4168c729 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -113,13 +114,17 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() { try { + ResourceGuard rlock(ResourceGuard::Metrics::getIORead(), read_settings.io_scheduling.read_resource_link, to_read_bytes); bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); + read_settings.io_scheduling.read_resource_link.adjust(to_read_bytes, bytes_read); + rlock.unlock(); // Do not hold resource under bandwidth throttler if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); break; } catch (const Azure::Core::RequestFailedException & e) { + read_settings.io_scheduling.read_resource_link.accumulate(to_read_bytes); // We assume no resource was used in case of failure ProfileEvents::increment(ProfileEvents::ReadBufferFromAzureRequestsErrors); LOG_DEBUG(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 2c90e3a9003..355f70b5e33 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -88,14 +88,14 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { try { - ResourceGuard rlock(write_settings.resource_link, cost); // Note that zero-cost requests are ignored + ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, cost); // Note that zero-cost requests are ignored func(); break; } catch (const Azure::Core::RequestFailedException & e) { if (cost) - write_settings.resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it + write_settings.io_scheduling.write_resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it if (i == num_tries - 1 || !isRetryableAzureException(e)) throw; @@ -105,7 +105,7 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, catch (...) { if (cost) - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure + write_settings.io_scheduling.write_resource_link.accumulate(cost); // We assume no resource was used in case of failure throw; } } diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 5803a985000..ccdb321c904 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -461,14 +461,17 @@ DiskObjectStoragePtr DiskObjectStorage::createDiskObjectStorage() } template -static inline Settings updateResourceLink(const Settings & settings, const String & resource_name) +static inline Settings updateIOSchedulingSettings(const Settings & settings, const String & read_resource_name, const String & write_resource_name) { - if (resource_name.empty()) + if (read_resource_name.empty() && write_resource_name.empty()) return settings; if (auto query_context = CurrentThread::getQueryContext()) { Settings result(settings); - result.resource_link = query_context->getWorkloadClassifier()->get(resource_name); + if (!read_resource_name.empty()) + result.io_scheduling.read_resource_link = query_context->getWorkloadClassifier()->get(read_resource_name); + if (!write_resource_name.empty()) + result.io_scheduling.write_resource_link = query_context->getWorkloadClassifier()->get(write_resource_name); return result; } return settings; @@ -500,7 +503,7 @@ std::unique_ptr DiskObjectStorage::readFile( return object_storage->readObjects( storage_objects, - updateResourceLink(settings, getReadResourceName()), + updateIOSchedulingSettings(settings, getReadResourceName(), getWriteResourceName()), read_hint, file_size); } @@ -513,7 +516,7 @@ std::unique_ptr DiskObjectStorage::writeFile( { LOG_TEST(log, "Write file: {}", path); - WriteSettings write_settings = updateResourceLink(settings, getWriteResourceName()); + WriteSettings write_settings = updateIOSchedulingSettings(settings, getReadResourceName(), getWriteResourceName()); auto transaction = createObjectStorageTransaction(); return transaction->writeFile(path, buf_size, mode, write_settings); } diff --git a/src/IO/ReadBufferFromS3.cpp b/src/IO/ReadBufferFromS3.cpp index 8823af55936..0c90ae25626 100644 --- a/src/IO/ReadBufferFromS3.cpp +++ b/src/IO/ReadBufferFromS3.cpp @@ -6,7 +6,6 @@ #include #include -#include #include #include @@ -425,22 +424,13 @@ Aws::S3::Model::GetObjectResult ReadBufferFromS3::sendRequest(size_t attempt, si ProfileEventTimeIncrement watch(ProfileEvents::ReadBufferFromS3InitMicroseconds); // We do not know in advance how many bytes we are going to consume, to avoid blocking estimated it from below - constexpr ResourceCost estimated_cost = 1; - ResourceGuard rlock(read_settings.resource_link, estimated_cost); - + CurrentThread::IOScope io_scope(read_settings.io_scheduling); Aws::S3::Model::GetObjectOutcome outcome = client_ptr->GetObject(req); - rlock.unlock(); - if (outcome.IsSuccess()) - { - ResourceCost bytes_read = outcome.GetResult().GetContentLength(); - read_settings.resource_link.adjust(estimated_cost, bytes_read); return outcome.GetResultWithOwnership(); - } else { - read_settings.resource_link.accumulate(estimated_cost); const auto & error = outcome.GetError(); throw S3Exception(error.GetMessage(), error.GetErrorType()); } diff --git a/src/IO/ReadSettings.h b/src/IO/ReadSettings.h index e73a9054928..7c22682dc76 100644 --- a/src/IO/ReadSettings.h +++ b/src/IO/ReadSettings.h @@ -118,8 +118,7 @@ struct ReadSettings ThrottlerPtr remote_throttler; ThrottlerPtr local_throttler; - // Resource to be used during reading - ResourceLink resource_link; + IOSchedulingSettings io_scheduling; size_t http_max_tries = 10; size_t http_retry_initial_backoff_ms = 100; diff --git a/src/IO/WriteBufferFromS3.cpp b/src/IO/WriteBufferFromS3.cpp index ff18a77f09f..160816ebbaa 100644 --- a/src/IO/WriteBufferFromS3.cpp +++ b/src/IO/WriteBufferFromS3.cpp @@ -11,7 +11,6 @@ #include #include -#include #include #include #include @@ -538,12 +537,11 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) auto & request = std::get<0>(*worker_data); - ResourceCost cost = request.GetContentLength(); - ResourceGuard rlock(write_settings.resource_link, cost); + CurrentThread::IOScope io_scope(write_settings.io_scheduling); + Stopwatch watch; auto outcome = client_ptr->UploadPart(request); watch.stop(); - rlock.unlock(); // Avoid acquiring other locks under resource lock ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); @@ -557,7 +555,6 @@ void WriteBufferFromS3::writePart(WriteBufferFromS3::PartData && data) if (!outcome.IsSuccess()) { ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure throw S3Exception(outcome.GetError().GetMessage(), outcome.GetError().GetErrorType()); } @@ -695,12 +692,11 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data if (client_ptr->isClientForDisk()) ProfileEvents::increment(ProfileEvents::DiskS3PutObject); - ResourceCost cost = request.GetContentLength(); - ResourceGuard rlock(write_settings.resource_link, cost); + CurrentThread::IOScope io_scope(write_settings.io_scheduling); + Stopwatch watch; auto outcome = client_ptr->PutObject(request); watch.stop(); - rlock.unlock(); ProfileEvents::increment(ProfileEvents::WriteBufferFromS3Microseconds, watch.elapsedMicroseconds()); if (blob_log) @@ -714,7 +710,6 @@ void WriteBufferFromS3::makeSinglepartUpload(WriteBufferFromS3::PartData && data } ProfileEvents::increment(ProfileEvents::WriteBufferFromS3RequestsErrors, 1); - write_settings.resource_link.accumulate(cost); // We assume no resource was used in case of failure if (outcome.GetError().GetErrorType() == Aws::S3::S3Errors::NO_SUCH_KEY) { diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 84bb25439b5..cdc75e8c0e9 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -13,8 +13,7 @@ struct WriteSettings ThrottlerPtr remote_throttler; ThrottlerPtr local_throttler; - // Resource to be used during reading - ResourceLink resource_link; + IOSchedulingSettings io_scheduling; /// Filesystem cache settings bool enable_filesystem_cache_on_write_operations = false; diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index be339d021dc..d1502577bf7 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -119,27 +119,25 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); + read_settings.io_scheduling.read_resource_link.adjust(num_bytes_to_read, std::max(0, bytes_read)); } catch (...) { - read_settings.resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure + read_settings.io_scheduling.read_resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure throw; } - rlock.unlock(); if (bytes_read < 0) { - read_settings.resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to read from HDFS: {}, file path: {}. Error: {}", hdfs_uri, hdfs_file_path, std::string(hdfsGetLastError())); } - read_settings.resource_link.adjust(num_bytes_to_read, bytes_read); if (bytes_read) { diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 2c14b38ce01..65a5f45cd96 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -66,25 +66,21 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) { - ResourceGuard rlock(write_settings.resource_link, size); int bytes_written; try { + ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, size); bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + write_settings.io_scheduling.write_resource_link.adjust(size, std::max(0, bytes_written)); } catch (...) { - write_settings.resource_link.accumulate(size); // We assume no resource was used in case of failure + write_settings.io_scheduling.write_resource_link.accumulate(size); // We assume no resource was used in case of failure throw; } - rlock.unlock(); if (bytes_written < 0) - { - write_settings.resource_link.accumulate(size); // We assume no resource was used in case of failure throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); - } - write_settings.resource_link.adjust(size, bytes_written); if (write_settings.remote_throttler) write_settings.remote_throttler->add(bytes_written, ProfileEvents::RemoteWriteThrottlerBytes, ProfileEvents::RemoteWriteThrottlerSleepMicroseconds); From eb3c61915d4bfae9f1e1007ca02fcd2c0f50b585 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 14:27:39 +0000 Subject: [PATCH 002/154] fix mixed read and write links --- src/Common/HTTPConnectionPool.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index de7e10d044a..b6bf98894e4 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -400,15 +400,15 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) { - Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); - } else { - Session::setSendDataHooks(); - } - if (ResourceLink link = CurrentThread::getWriteResourceLink()) { - Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); } else { Session::setReceiveDataHooks(); } + if (ResourceLink link = CurrentThread::getWriteResourceLink()) { + Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + } else { + Session::setSendDataHooks(); + } std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); From f1f354f22b5d1573e970e5d2a5e2ea540e3f99f8 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 15:33:59 +0000 Subject: [PATCH 003/154] add test for granularity and total byte size of resource requests --- tests/integration/test_scheduler/test.py | 67 ++++++++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 8e37bd8d403..9940e16ea42 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -28,6 +28,73 @@ def start_cluster(): cluster.shutdown() +def test_s3_resource_request_granularity(): + node.query( + f""" + drop table if exists data; + create table data (key UInt64 CODEC(NONE), value String CODEC(NONE)) engine=MergeTree() order by key settings min_bytes_for_wide_part=1e9, storage_policy='s3'; + """ + ) + + total_bytes = 50000000 # Approximate data size + max_bytes_per_request = 2000000 # Should be ~1MB or less in general + min_bytes_per_request = 6000 # Small requests are ok, but we don't want hurt performance with too often resource requests + + writes_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + write_bytes_before = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + node.query(f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'") + writes_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + write_bytes_after = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) + + assert write_bytes_after - write_bytes_before > 1.0 * total_bytes + assert write_bytes_after - write_bytes_before < 1.2 * total_bytes + assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) < max_bytes_per_request + assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) > min_bytes_per_request + + reads_before = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + read_bytes_before = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + node.query(f"select count() from data where not ignore(*) SETTINGS workload='admin'") + reads_after = int( + node.query( + f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + read_bytes_after = int( + node.query( + f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) + + assert read_bytes_after - read_bytes_before > 1.0 * total_bytes + assert read_bytes_after - read_bytes_before < 1.2 * total_bytes + assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) < max_bytes_per_request + assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) > min_bytes_per_request + + def test_s3_disk(): node.query( f""" From b74cf1a356fe668721532c1d05fa08aeaa1bf023 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Thu, 13 Jun 2024 15:43:08 +0000 Subject: [PATCH 004/154] Automatic style fix --- tests/integration/test_scheduler/test.py | 30 +++++++++++++++++------- 1 file changed, 21 insertions(+), 9 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 9940e16ea42..0ae297f04d1 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -36,9 +36,9 @@ def test_s3_resource_request_granularity(): """ ) - total_bytes = 50000000 # Approximate data size - max_bytes_per_request = 2000000 # Should be ~1MB or less in general - min_bytes_per_request = 6000 # Small requests are ok, but we don't want hurt performance with too often resource requests + total_bytes = 50000000 # Approximate data size + max_bytes_per_request = 2000000 # Should be ~1MB or less in general + min_bytes_per_request = 6000 # Small requests are ok, but we don't want hurt performance with too often resource requests writes_before = int( node.query( @@ -50,7 +50,9 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" ).strip() ) - node.query(f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'") + node.query( + f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'" + ) writes_after = int( node.query( f"select dequeued_requests from system.scheduler where resource='network_write' and path='/prio/admin'" @@ -64,8 +66,12 @@ def test_s3_resource_request_granularity(): assert write_bytes_after - write_bytes_before > 1.0 * total_bytes assert write_bytes_after - write_bytes_before < 1.2 * total_bytes - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) < max_bytes_per_request - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) > min_bytes_per_request + assert (write_bytes_after - write_bytes_before) / ( + writes_after - writes_before + ) < max_bytes_per_request + assert (write_bytes_after - write_bytes_before) / ( + writes_after - writes_before + ) > min_bytes_per_request reads_before = int( node.query( @@ -77,7 +83,9 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" ).strip() ) - node.query(f"select count() from data where not ignore(*) SETTINGS workload='admin'") + node.query( + f"select count() from data where not ignore(*) SETTINGS workload='admin'" + ) reads_after = int( node.query( f"select dequeued_requests from system.scheduler where resource='network_read' and path='/prio/admin'" @@ -91,8 +99,12 @@ def test_s3_resource_request_granularity(): assert read_bytes_after - read_bytes_before > 1.0 * total_bytes assert read_bytes_after - read_bytes_before < 1.2 * total_bytes - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) < max_bytes_per_request - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) > min_bytes_per_request + assert (read_bytes_after - read_bytes_before) / ( + reads_after - reads_before + ) < max_bytes_per_request + assert (read_bytes_after - read_bytes_before) / ( + reads_after - reads_before + ) > min_bytes_per_request def test_s3_disk(): From 937e1708259c518d27959961849da263163cc5a2 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 13 Jun 2024 18:58:39 +0000 Subject: [PATCH 005/154] make ResourceGuards more straight-forward --- src/Common/HTTPConnectionPool.cpp | 10 +++--- .../Scheduler/Nodes/tests/ResourceTest.h | 9 ++++-- .../tests/gtest_dynamic_resource_manager.cpp | 7 ++++- .../Nodes/tests/gtest_resource_scheduler.cpp | 4 +++ src/Common/Scheduler/ResourceGuard.h | 31 ++++++++++++------- src/Common/Scheduler/ResourceRequest.h | 2 +- .../IO/ReadBufferFromAzureBlobStorage.cpp | 4 +-- .../IO/WriteBufferFromAzureBlobStorage.cpp | 6 +--- .../ObjectStorage/HDFS/ReadBufferFromHDFS.cpp | 15 ++------- .../HDFS/WriteBufferFromHDFS.cpp | 15 ++------- 10 files changed, 50 insertions(+), 53 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index b6bf98894e4..18a6ab13c4f 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -276,21 +276,18 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks void start(int bytes) override { - // TODO(serxa): add metrics here or better in scheduler code (e.g. during enqueue, or better in REsourceGuard::REquest)? request.enqueue(bytes, link); request.wait(); } void finish(int bytes) override { - request.finish(); - link.adjust(request.cost, bytes); // success + request.finish(bytes, link); } void fail() override { - request.finish(); - link.accumulate(request.cost); // We assume no resource was used in case of failure + request.finish(0, link); } ResourceLink link; @@ -466,6 +463,9 @@ private: } } response_stream = nullptr; + // FIXME: We are not sure that response stream is fully read at this moment, so hooks could possible be called after this point, right? + // Session::setSendDataHooks(); + // Session::setReceiveDataHooks(); group->atConnectionDestroy(); diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index a5eb98f2a2f..c440cb176f8 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -232,12 +232,13 @@ struct ResourceTestManager : public ResourceTestBase ResourceTestManager & t; Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, PostponeLocking) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Postpone) , t(t_) { t.onEnqueue(link); lock(); t.onExecute(link); + consume(cost); } }; @@ -310,8 +311,9 @@ struct ResourceTestManager : public ResourceTestBase // NOTE: actually leader's request(s) make their own small busy period. void blockResource(ResourceLink link) { - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::Lock::Postpone); g.lock(); + g.consume(1); // NOTE: at this point we assume resource to be blocked by single request (1) busy_period.arrive_and_wait(); // (1) notify all followers that resource is blocked busy_period.arrive_and_wait(); // (2) wait all followers to enqueue their requests @@ -320,10 +322,11 @@ struct ResourceTestManager : public ResourceTestBase { getLinkData(link).left += total_requests + 1; busy_period.arrive_and_wait(); // (1) wait leader to block resource - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::PostponeLocking); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::Lock::Postpone); onEnqueue(link); busy_period.arrive_and_wait(); // (2) notify leader to unblock g.lock(); + g.consume(cost); onExecute(link); } }; diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 4ac79977663..0f4aaab70aa 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -36,11 +36,16 @@ TEST(SchedulerDynamicResourceManager, Smoke) for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), ResourceGuard::PostponeLocking); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Postpone); gA.lock(); + gA.consume(1); gA.unlock(); ResourceGuard gB(ResourceGuard::Metrics::getIOWrite(), cB->get("res1")); + gB.unlock(); + + ResourceGuard gC(ResourceGuard::Metrics::getIORead(), cB->get("res1")); + gB.consume(2); } } diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ba573bf0c85..ff3054d6b7a 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -111,21 +111,25 @@ TEST(SchedulerRoot, Smoke) { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a); EXPECT_TRUE(fc1->requests.contains(&rg.request)); + rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), b); EXPECT_TRUE(fc1->requests.contains(&rg.request)); + rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), c); EXPECT_TRUE(fc2->requests.contains(&rg.request)); + rg.consume(1); } { ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), d); EXPECT_TRUE(fc2->requests.contains(&rg.request)); + rg.consume(1); } } diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 73aea4afdb6..93b6268a62c 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -42,12 +42,12 @@ namespace DB class ResourceGuard { public: - enum ResourceGuardCtor + enum class Lock { - LockStraightAway, /// Locks inside constructor (default) + StraightAway, /// Locks inside constructor (default) // WARNING: Only for tests. It is not exception-safe because `lock()` must be called after construction. - PostponeLocking /// Don't lock in constructor, but send request + Postpone /// Don't lock in constructor, but send request }; struct Metrics @@ -96,8 +96,6 @@ public: chassert(state == Finished); state = Enqueued; ResourceRequest::reset(cost_); - ProfileEvents::increment(metrics->requests); - ProfileEvents::increment(metrics->cost, cost_); link_.queue->enqueueRequestUsingBudget(this); } @@ -121,12 +119,16 @@ public: dequeued_cv.wait(lock, [this] { return state == Dequeued; }); } - void finish() + void finish(ResourceCost real_cost_, ResourceLink link_) { // lock(mutex) is not required because `Dequeued` request cannot be used by the scheduler thread chassert(state == Dequeued); state = Finished; + if (cost != real_cost_) + link_.adjust(cost, real_cost_); ResourceRequest::finish(); + ProfileEvents::increment(metrics->requests); + ProfileEvents::increment(metrics->cost, real_cost_); } void assertFinished() @@ -153,7 +155,7 @@ public: }; /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) - explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuardCtor ctor = LockStraightAway) + explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuard::Lock type = ResourceGuard::Lock::StraightAway) : link(link_) , request(Request::local(metrics)) { @@ -162,7 +164,7 @@ public: else if (link) { request.enqueue(cost, link); - if (ctor == LockStraightAway) + if (type == Lock::StraightAway) request.wait(); } } @@ -179,18 +181,25 @@ public: request.wait(); } - /// Report resource consumption has finished - void unlock() + void consume(ResourceCost cost) { + real_cost += cost; + } + + /// Report resource consumption has finished + void unlock(ResourceCost consumed = 0) + { + consume(consumed); if (link) { - request.finish(); + request.finish(real_cost, link); link.reset(); } } ResourceLink link; Request & request; + ResourceCost real_cost = 0; }; } diff --git a/src/Common/Scheduler/ResourceRequest.h b/src/Common/Scheduler/ResourceRequest.h index d64f624cec5..7b6a5af0fe6 100644 --- a/src/Common/Scheduler/ResourceRequest.h +++ b/src/Common/Scheduler/ResourceRequest.h @@ -45,7 +45,7 @@ constexpr ResourceCost ResourceCostMax = std::numeric_limits::max(); class ResourceRequest : public boost::intrusive::list_base_hook<> { public: - /// Cost of request execution; should be filled before request enqueueing. + /// Cost of request execution; should be filled before request enqueueing and remain constant until `finish()`. /// NOTE: If cost is not known in advance, ResourceBudget should be used (note that every ISchedulerQueue has it) ResourceCost cost; diff --git a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index ecc4168c729..626220a843e 100644 --- a/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -116,15 +116,13 @@ bool ReadBufferFromAzureBlobStorage::nextImpl() { ResourceGuard rlock(ResourceGuard::Metrics::getIORead(), read_settings.io_scheduling.read_resource_link, to_read_bytes); bytes_read = data_stream->ReadToCount(reinterpret_cast(data_ptr), to_read_bytes); - read_settings.io_scheduling.read_resource_link.adjust(to_read_bytes, bytes_read); - rlock.unlock(); // Do not hold resource under bandwidth throttler + rlock.unlock(bytes_read); // Do not hold resource under bandwidth throttler if (read_settings.remote_throttler) read_settings.remote_throttler->add(bytes_read, ProfileEvents::RemoteReadThrottlerBytes, ProfileEvents::RemoteReadThrottlerSleepMicroseconds); break; } catch (const Azure::Core::RequestFailedException & e) { - read_settings.io_scheduling.read_resource_link.accumulate(to_read_bytes); // We assume no resource was used in case of failure ProfileEvents::increment(ProfileEvents::ReadBufferFromAzureRequestsErrors); LOG_DEBUG(log, "Exception caught during Azure Read for file {} at attempt {}/{}: {}", path, i + 1, max_single_read_retries, e.Message); diff --git a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index 82ce9d32f5a..d040200c31e 100644 --- a/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -94,13 +94,11 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, { ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, cost); // Note that zero-cost requests are ignored func(); + rlock.unlock(cost); break; } catch (const Azure::Core::RequestFailedException & e) { - if (cost) - write_settings.io_scheduling.write_resource_link.accumulate(cost); // Accumulate resource for later use, because we have failed to consume it - if (i == num_tries - 1 || !isRetryableAzureException(e)) throw; @@ -108,8 +106,6 @@ void WriteBufferFromAzureBlobStorage::execWithRetry(std::function func, } catch (...) { - if (cost) - write_settings.io_scheduling.write_resource_link.accumulate(cost); // We assume no resource was used in case of failure throw; } } diff --git a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp index d1502577bf7..0fbd123508e 100644 --- a/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/ReadBufferFromHDFS.cpp @@ -119,18 +119,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl : public BufferWithOwnMemory(num_bytes_to_read)); - read_settings.io_scheduling.read_resource_link.adjust(num_bytes_to_read, std::max(0, bytes_read)); - } - catch (...) - { - read_settings.io_scheduling.read_resource_link.accumulate(num_bytes_to_read); // We assume no resource was used in case of failure - throw; - } + ResourceGuard rlock(ResourceGuard::Metrics::getIORead(), read_settings.io_scheduling.read_resource_link, num_bytes_to_read); + int bytes_read = hdfsRead(fs.get(), fin, internal_buffer.begin(), safe_cast(num_bytes_to_read)); + rlock.unlock(std::max(0, bytes_read)); if (bytes_read < 0) { diff --git a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp index 65a5f45cd96..816de6676f1 100644 --- a/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/ObjectStorage/HDFS/WriteBufferFromHDFS.cpp @@ -66,18 +66,9 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl int write(const char * start, size_t size) { - int bytes_written; - try - { - ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, size); - bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); - write_settings.io_scheduling.write_resource_link.adjust(size, std::max(0, bytes_written)); - } - catch (...) - { - write_settings.io_scheduling.write_resource_link.accumulate(size); // We assume no resource was used in case of failure - throw; - } + ResourceGuard rlock(ResourceGuard::Metrics::getIOWrite(), write_settings.io_scheduling.write_resource_link, size); + int bytes_written = hdfsWrite(fs.get(), fout, start, safe_cast(size)); + rlock.unlock(std::max(0, bytes_written)); if (bytes_written < 0) throw Exception(ErrorCodes::NETWORK_ERROR, "Fail to write HDFS file: {} {}", hdfs_uri, std::string(hdfsGetLastError())); From a5eeeb3422e956212c15382af13ca45012efc96b Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:01:14 +0000 Subject: [PATCH 006/154] bugfix: wrong estimated cost passed for budget adjusting --- src/Common/Scheduler/ISchedulerQueue.h | 19 +++++-------------- src/Common/Scheduler/ResouceLink.cpp | 25 ------------------------- src/Common/Scheduler/ResourceGuard.h | 7 ++++--- src/Common/Scheduler/ResourceLink.h | 4 ---- 4 files changed, 9 insertions(+), 46 deletions(-) delete mode 100644 src/Common/Scheduler/ResouceLink.cpp diff --git a/src/Common/Scheduler/ISchedulerQueue.h b/src/Common/Scheduler/ISchedulerQueue.h index 532f4bf6c63..b7a51870a24 100644 --- a/src/Common/Scheduler/ISchedulerQueue.h +++ b/src/Common/Scheduler/ISchedulerQueue.h @@ -22,10 +22,13 @@ public: {} // Wrapper for `enqueueRequest()` that should be used to account for available resource budget - void enqueueRequestUsingBudget(ResourceRequest * request) + // Returns `estimated_cost` that should be passed later to `adjustBudget()` + [[ nodiscard ]] ResourceCost enqueueRequestUsingBudget(ResourceRequest * request) { - request->cost = budget.ask(request->cost); + ResourceCost estimated_cost = request->cost; + request->cost = budget.ask(estimated_cost); enqueueRequest(request); + return estimated_cost; } // Should be called to account for difference between real and estimated costs @@ -34,18 +37,6 @@ public: budget.adjust(estimated_cost, real_cost); } - // Adjust budget to account for extra consumption of `cost` resource units - void consumeBudget(ResourceCost cost) - { - adjustBudget(0, cost); - } - - // Adjust budget to account for requested, but not consumed `cost` resource units - void accumulateBudget(ResourceCost cost) - { - adjustBudget(cost, 0); - } - /// Enqueue new request to be executed using underlying resource. /// Should be called outside of scheduling subsystem, implementation must be thread-safe. virtual void enqueueRequest(ResourceRequest * request) = 0; diff --git a/src/Common/Scheduler/ResouceLink.cpp b/src/Common/Scheduler/ResouceLink.cpp deleted file mode 100644 index 2da5dba62dc..00000000000 --- a/src/Common/Scheduler/ResouceLink.cpp +++ /dev/null @@ -1,25 +0,0 @@ -#include -#include -#include - -namespace DB -{ -void ResourceLink::adjust(ResourceCost estimated_cost, ResourceCost real_cost) const -{ - if (queue) - queue->adjustBudget(estimated_cost, real_cost); -} - -void ResourceLink::consumed(ResourceCost cost) const -{ - if (queue) - queue->consumeBudget(cost); -} - -void ResourceLink::accumulate(DB::ResourceCost cost) const -{ - if (queue) - queue->accumulateBudget(cost); -} -} - diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 93b6268a62c..c46a3683455 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -96,7 +96,7 @@ public: chassert(state == Finished); state = Enqueued; ResourceRequest::reset(cost_); - link_.queue->enqueueRequestUsingBudget(this); + estimated_cost = link_.queue->enqueueRequestUsingBudget(this); // NOTE: it modifies `cost` and enqueues request } // This function is executed inside scheduler thread and wakes thread issued this `request`. @@ -124,8 +124,8 @@ public: // lock(mutex) is not required because `Dequeued` request cannot be used by the scheduler thread chassert(state == Dequeued); state = Finished; - if (cost != real_cost_) - link_.adjust(cost, real_cost_); + if (estimated_cost != real_cost_) + link_.queue->adjustBudget(estimated_cost, real_cost_); ResourceRequest::finish(); ProfileEvents::increment(metrics->requests); ProfileEvents::increment(metrics->cost, real_cost_); @@ -149,6 +149,7 @@ public: const Metrics * metrics = nullptr; // Must be initialized before use private: + ResourceCost estimated_cost = 0; // Stores initial `cost` value in case budget was used to modify it std::mutex mutex; std::condition_variable dequeued_cv; RequestState state = Finished; diff --git a/src/Common/Scheduler/ResourceLink.h b/src/Common/Scheduler/ResourceLink.h index 6dd3be930ca..a4e2adbd963 100644 --- a/src/Common/Scheduler/ResourceLink.h +++ b/src/Common/Scheduler/ResourceLink.h @@ -17,10 +17,6 @@ struct ResourceLink bool operator==(const ResourceLink &) const = default; explicit operator bool() const { return queue != nullptr; } - void adjust(ResourceCost estimated_cost, ResourceCost real_cost) const; - void consumed(ResourceCost cost) const; - void accumulate(ResourceCost cost) const; - void reset() { queue = nullptr; From ec71d35aa64e22d62715d313f642b81f9e74f36c Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 13:12:23 +0000 Subject: [PATCH 007/154] add test for scheduler queue budget --- .../Nodes/tests/gtest_resource_scheduler.cpp | 42 ++++++++++++++++++- 1 file changed, 40 insertions(+), 2 deletions(-) diff --git a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp index ff3054d6b7a..ddfe0cfbc6f 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_resource_scheduler.cpp @@ -1,11 +1,13 @@ #include -#include - #include +#include +#include + #include #include +#include using namespace DB; @@ -22,6 +24,17 @@ struct ResourceTest : public ResourceTestBase { scheduler.stop(true); } + + std::mutex rng_mutex; + pcg64 rng{randomSeed()}; + + template + T randomInt(T from, T to) + { + std::uniform_int_distribution distribution(from, to); + std::lock_guard lock(rng_mutex); + return distribution(rng); + } }; struct ResourceHolder @@ -133,6 +146,31 @@ TEST(SchedulerRoot, Smoke) } } +TEST(SchedulerRoot, Budget) +{ + ResourceTest t; + + ResourceHolder r1(t); + r1.add("/", "1"); + r1.add("/prio"); + auto a = r1.addQueue("/prio/A", ""); + r1.registerResource(); + + ResourceCost total_real_cost = 0; + int total_requests = 10; + for (int i = 0 ; i < total_requests; i++) + { + ResourceCost est_cost = t.randomInt(1, 10); + ResourceCost real_cost = t.randomInt(0, 10); + ResourceGuard rg(ResourceGuard::Metrics::getIOWrite(), a, est_cost); + rg.consume(real_cost); + total_real_cost += real_cost; + } + + EXPECT_EQ(total_requests, a.queue->dequeued_requests); + EXPECT_EQ(total_real_cost, a.queue->dequeued_cost - a.queue->getBudget()); +} + TEST(SchedulerRoot, Cancel) { ResourceTest t; From 709ef2ba8539bfe0316817374e9fe4c0768e502b Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 14 Jun 2024 16:09:00 +0000 Subject: [PATCH 008/154] add metrics and budget checks --- tests/integration/test_scheduler/test.py | 59 ++++++++++++++++++++---- 1 file changed, 51 insertions(+), 8 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 9940e16ea42..c6338ec3eb1 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -28,6 +28,19 @@ def start_cluster(): cluster.shutdown() +def check_profile_event_for_query(workload, profile_event, amount=1): + node.query("system flush logs") + query_pattern = f"workload='{workload}'".replace("'", "\\'") + assert ( + int( + node.query( + f"select ProfileEvents['{profile_event}'] from system.query_log where query ilike '%{query_pattern}%' and type = 'QueryFinish' order by query_start_time_microseconds desc limit 1" + ) + ) + == amount + ) + + def test_s3_resource_request_granularity(): node.query( f""" @@ -50,6 +63,11 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" ).strip() ) + write_budget_before = int( + node.query( + f"select budget from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) node.query(f"insert into data select number, randomString(10000000) from numbers(5) SETTINGS workload='admin'") writes_after = int( node.query( @@ -61,11 +79,22 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_write' and path='/prio/admin'" ).strip() ) + write_budget_after = int( + node.query( + f"select budget from system.scheduler where resource='network_write' and path='/prio/admin'" + ).strip() + ) - assert write_bytes_after - write_bytes_before > 1.0 * total_bytes - assert write_bytes_after - write_bytes_before < 1.2 * total_bytes - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) < max_bytes_per_request - assert (write_bytes_after - write_bytes_before) / (writes_after - writes_before) > min_bytes_per_request + write_requests = writes_after - writes_before + write_bytes = (write_bytes_after - write_bytes_before) - (write_budget_after - write_budget_before) + assert write_bytes > 1.0 * total_bytes + assert write_bytes < 1.05 * total_bytes + assert write_bytes / write_requests < max_bytes_per_request + assert write_bytes / write_requests > min_bytes_per_request + check_profile_event_for_query("admin", "SchedulerIOWriteRequests", write_requests) + check_profile_event_for_query("admin", "SchedulerIOWriteBytes", write_bytes) + + node.query(f"optimize table data final") reads_before = int( node.query( @@ -77,6 +106,11 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" ).strip() ) + read_budget_before = int( + node.query( + f"select budget from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) node.query(f"select count() from data where not ignore(*) SETTINGS workload='admin'") reads_after = int( node.query( @@ -88,11 +122,20 @@ def test_s3_resource_request_granularity(): f"select dequeued_cost from system.scheduler where resource='network_read' and path='/prio/admin'" ).strip() ) + read_budget_after = int( + node.query( + f"select budget from system.scheduler where resource='network_read' and path='/prio/admin'" + ).strip() + ) - assert read_bytes_after - read_bytes_before > 1.0 * total_bytes - assert read_bytes_after - read_bytes_before < 1.2 * total_bytes - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) < max_bytes_per_request - assert (read_bytes_after - read_bytes_before) / (reads_after - reads_before) > min_bytes_per_request + read_bytes = (read_bytes_after - read_bytes_before) - (read_budget_after - read_budget_before) + read_requests = reads_after - reads_before + assert read_bytes > 1.0 * total_bytes + assert read_bytes < 1.05 * total_bytes + assert read_bytes / read_requests < max_bytes_per_request + assert read_bytes / read_requests > min_bytes_per_request + check_profile_event_for_query("admin", "SchedulerIOReadRequests", read_requests) + check_profile_event_for_query("admin", "SchedulerIOReadBytes", read_bytes) def test_s3_disk(): From f3c3d419bbce4f7edecf807fdbaca8557ee36d1f Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 14 Jun 2024 16:17:57 +0000 Subject: [PATCH 009/154] Automatic style fix --- tests/integration/test_scheduler/test.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_scheduler/test.py b/tests/integration/test_scheduler/test.py index 11525d3fd62..5c7da0e2516 100644 --- a/tests/integration/test_scheduler/test.py +++ b/tests/integration/test_scheduler/test.py @@ -88,7 +88,9 @@ def test_s3_resource_request_granularity(): ) write_requests = writes_after - writes_before - write_bytes = (write_bytes_after - write_bytes_before) - (write_budget_after - write_budget_before) + write_bytes = (write_bytes_after - write_bytes_before) - ( + write_budget_after - write_budget_before + ) assert write_bytes > 1.0 * total_bytes assert write_bytes < 1.05 * total_bytes assert write_bytes / write_requests < max_bytes_per_request @@ -132,7 +134,9 @@ def test_s3_resource_request_granularity(): ).strip() ) - read_bytes = (read_bytes_after - read_bytes_before) - (read_budget_after - read_budget_before) + read_bytes = (read_bytes_after - read_bytes_before) - ( + read_budget_after - read_budget_before + ) read_requests = reads_after - reads_before assert read_bytes > 1.0 * total_bytes assert read_bytes < 1.05 * total_bytes From a9bfaf6454f38ce9e6ef57fdcaba864b47d49e9f Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 18 Jun 2024 11:52:40 +0000 Subject: [PATCH 010/154] style --- src/Common/HTTPConnectionPool.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index a713bd14d62..bac12fd438d 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -383,16 +383,15 @@ private: auto idle = idleTime(); // Reset data hooks for IO scheduling - if (ResourceLink link = CurrentThread::getReadResourceLink()) { + if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); - } else { + else Session::setReceiveDataHooks(); - } - if (ResourceLink link = CurrentThread::getWriteResourceLink()) { + + if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); - } else { + else Session::setSendDataHooks(); - } std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); From 7a01b8189cadb9b720182548c3c49f9267f8a606 Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 19 Jun 2024 00:49:21 +0000 Subject: [PATCH 011/154] fix typo, renames --- base/poco/Net/include/Poco/Net/HTTPSession.h | 2 +- src/Common/Scheduler/Nodes/tests/ResourceTest.h | 6 +++--- .../Nodes/tests/gtest_dynamic_resource_manager.cpp | 2 +- src/Common/Scheduler/ResourceGuard.h | 10 +++++----- 4 files changed, 10 insertions(+), 10 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index b0e59443f9b..b25ad68cc67 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -49,7 +49,7 @@ namespace Net /// Called when sending/receiving of data `bytes` is successfully finished. virtual void fail() = 0; - /// If an error occured during send/receive `fail()` is called instead of `finish()`. + /// If an error occurred during send/receive `fail()` is called instead of `finish()`. }; diff --git a/src/Common/Scheduler/Nodes/tests/ResourceTest.h b/src/Common/Scheduler/Nodes/tests/ResourceTest.h index c440cb176f8..c787a686a09 100644 --- a/src/Common/Scheduler/Nodes/tests/ResourceTest.h +++ b/src/Common/Scheduler/Nodes/tests/ResourceTest.h @@ -232,7 +232,7 @@ struct ResourceTestManager : public ResourceTestBase ResourceTestManager & t; Guard(ResourceTestManager & t_, ResourceLink link_, ResourceCost cost) - : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Postpone) + : ResourceGuard(ResourceGuard::Metrics::getIOWrite(), link_, cost, Lock::Defer) , t(t_) { t.onEnqueue(link); @@ -311,7 +311,7 @@ struct ResourceTestManager : public ResourceTestBase // NOTE: actually leader's request(s) make their own small busy period. void blockResource(ResourceLink link) { - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::Lock::Postpone); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, 1, ResourceGuard::Lock::Defer); g.lock(); g.consume(1); // NOTE: at this point we assume resource to be blocked by single request (1) @@ -322,7 +322,7 @@ struct ResourceTestManager : public ResourceTestBase { getLinkData(link).left += total_requests + 1; busy_period.arrive_and_wait(); // (1) wait leader to block resource - ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::Lock::Postpone); + ResourceGuard g(ResourceGuard::Metrics::getIOWrite(), link, cost, ResourceGuard::Lock::Defer); onEnqueue(link); busy_period.arrive_and_wait(); // (2) notify leader to unblock g.lock(); diff --git a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp index 0f4aaab70aa..3328196cced 100644 --- a/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp +++ b/src/Common/Scheduler/Nodes/tests/gtest_dynamic_resource_manager.cpp @@ -36,7 +36,7 @@ TEST(SchedulerDynamicResourceManager, Smoke) for (int i = 0; i < 10; i++) { - ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Postpone); + ResourceGuard gA(ResourceGuard::Metrics::getIOWrite(), cA->get("res1"), 1, ResourceGuard::Lock::Defer); gA.lock(); gA.consume(1); gA.unlock(); diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index c46a3683455..2e735aae656 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -44,10 +44,10 @@ class ResourceGuard public: enum class Lock { - StraightAway, /// Locks inside constructor (default) + Default, /// Locks inside constructor // WARNING: Only for tests. It is not exception-safe because `lock()` must be called after construction. - Postpone /// Don't lock in constructor, but send request + Defer /// Don't lock in constructor, but send request }; struct Metrics @@ -155,8 +155,8 @@ public: RequestState state = Finished; }; - /// Creates pending request for resource; blocks while resource is not available (unless `PostponeLocking`) - explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuard::Lock type = ResourceGuard::Lock::StraightAway) + /// Creates pending request for resource; blocks while resource is not available (unless `Lock::Defer`) + explicit ResourceGuard(const Metrics * metrics, ResourceLink link_, ResourceCost cost = 1, ResourceGuard::Lock type = ResourceGuard::Lock::Default) : link(link_) , request(Request::local(metrics)) { @@ -165,7 +165,7 @@ public: else if (link) { request.enqueue(cost, link); - if (type == Lock::StraightAway) + if (type == Lock::Default) request.wait(); } } From 54fda05094d1d3d788e0e65591c0d15b0e5838b0 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 10:25:04 +0000 Subject: [PATCH 012/154] reset session send/recv hooks at connection destruction --- src/Common/HTTPConnectionPool.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index bac12fd438d..3798b7624ea 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -385,13 +385,8 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); - else - Session::setReceiveDataHooks(); - if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); - else - Session::setSendDataHooks(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); @@ -449,9 +444,8 @@ private: } } response_stream = nullptr; - // FIXME: We are not sure that response stream is fully read at this moment, so hooks could possible be called after this point, right? - // Session::setSendDataHooks(); - // Session::setReceiveDataHooks(); + Session::setSendDataHooks(); + Session::setReceiveDataHooks(); group->atConnectionDestroy(); From 3fa5ad92b4612b59ca9c35aab5853dcb3318a112 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 10:33:22 +0000 Subject: [PATCH 013/154] rename `IHTTPSessionDataHooks` methods --- base/poco/Net/include/Poco/Net/HTTPSession.h | 6 +++--- base/poco/Net/src/HTTPSession.cpp | 12 ++++++------ src/Common/HTTPConnectionPool.cpp | 12 ++++++------ 3 files changed, 15 insertions(+), 15 deletions(-) diff --git a/base/poco/Net/include/Poco/Net/HTTPSession.h b/base/poco/Net/include/Poco/Net/HTTPSession.h index b25ad68cc67..2038fd2aff5 100644 --- a/base/poco/Net/include/Poco/Net/HTTPSession.h +++ b/base/poco/Net/include/Poco/Net/HTTPSession.h @@ -42,13 +42,13 @@ namespace Net public: virtual ~IHTTPSessionDataHooks() = default; - virtual void start(int bytes) = 0; + virtual void atStart(int bytes) = 0; /// Called before sending/receiving data `bytes` to/from socket. - virtual void finish(int bytes) = 0; + virtual void atFinish(int bytes) = 0; /// Called when sending/receiving of data `bytes` is successfully finished. - virtual void fail() = 0; + virtual void atFail() = 0; /// If an error occurred during send/receive `fail()` is called instead of `finish()`. }; diff --git a/base/poco/Net/src/HTTPSession.cpp b/base/poco/Net/src/HTTPSession.cpp index 596185703fa..f30ccb21129 100644 --- a/base/poco/Net/src/HTTPSession.cpp +++ b/base/poco/Net/src/HTTPSession.cpp @@ -167,16 +167,16 @@ int HTTPSession::write(const char* buffer, std::streamsize length) try { if (_sendDataHooks) - _sendDataHooks->start((int) length); + _sendDataHooks->atStart((int) length); int result = _socket.sendBytes(buffer, (int) length); if (_sendDataHooks) - _sendDataHooks->finish(result); + _sendDataHooks->atFinish(result); return result; } catch (Poco::Exception& exc) { if (_sendDataHooks) - _sendDataHooks->fail(); + _sendDataHooks->atFail(); setException(exc); throw; } @@ -188,16 +188,16 @@ int HTTPSession::receive(char* buffer, int length) try { if (_receiveDataHooks) - _receiveDataHooks->start(length); + _receiveDataHooks->atStart(length); int result = _socket.receiveBytes(buffer, length); if (_receiveDataHooks) - _receiveDataHooks->finish(result); + _receiveDataHooks->atFinish(result); return result; } catch (Poco::Exception& exc) { if (_receiveDataHooks) - _receiveDataHooks->fail(); + _receiveDataHooks->atFail(); setException(exc); throw; } diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 3798b7624ea..62362c61605 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -239,13 +239,13 @@ public: // Session data hooks implementation for integration with resource scheduler. // Hooks are created per every request-response pair and are registered/unregistered in HTTP session. -// * `start()` send resource request to the scheduler every time HTTP session is going to send or receive +// * `atStart()` send resource request to the scheduler every time HTTP session is going to send or receive // data to/from socket. `start()` waits for the scheduler confirmation. This way scheduler might // throttle and/or schedule socket data streams. -// * `finish()` hook is called on successful socket read/write operation. +// * `atFinish()` hook is called on successful socket read/write operation. // It informs the scheduler that operation is complete, which allows the scheduler to control the total // amount of in-flight bytes and/or operations. -// * `fail()` hook is called on failure of socket operation. The purpose is to correct the amount of bytes +// * `atFail()` hook is called on failure of socket operation. The purpose is to correct the amount of bytes // passed through the scheduler queue to ensure fair bandwidth allocation even in presence of errors. struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks { @@ -261,18 +261,18 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks request.assertFinished(); // Never destruct with an active request } - void start(int bytes) override + void atStart(int bytes) override { request.enqueue(bytes, link); request.wait(); } - void finish(int bytes) override + void atFinish(int bytes) override { request.finish(bytes, link); } - void fail() override + void atFail() override { request.finish(0, link); } From fefcc52c24b4560cc2434decdbd0981edfcbc5ca Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 5 Jul 2024 11:09:10 +0000 Subject: [PATCH 014/154] add logging of too long resource requests for http sessions --- src/Common/HTTPConnectionPool.cpp | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 62362c61605..acddcc8530d 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -249,8 +250,10 @@ public: // passed through the scheduler queue to ensure fair bandwidth allocation even in presence of errors. struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks { - explicit ResourceGuardSessionDataHooks(const ResourceGuard::Metrics * metrics, ResourceLink link_) + ResourceGuardSessionDataHooks(ResourceLink link_, const ResourceGuard::Metrics * metrics, LoggerPtr log_, const String & method, const String & uri) : link(link_) + , log(log_) + , http_request(method + " " + uri) { request.metrics = metrics; chassert(link); @@ -263,8 +266,12 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks void atStart(int bytes) override { + Stopwatch timer; request.enqueue(bytes, link); request.wait(); + timer.stop(); + if (timer.elapsedMilliseconds() >= 5000) + LOG_INFO(log, "Resource request took too long to finish: {} ms for {}", timer.elapsedMilliseconds(), http_request); } void atFinish(int bytes) override @@ -279,6 +286,8 @@ struct ResourceGuardSessionDataHooks : public Poco::Net::IHTTPSessionDataHooks ResourceLink link; ResourceGuard::Request request; + LoggerPtr log; + String http_request; }; @@ -384,9 +393,9 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) - Session::setReceiveDataHooks(std::make_shared(ResourceGuard::Metrics::getIORead(), link)); + Session::setReceiveDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIORead(), log, request.getMethod(), request.getURI())); if (ResourceLink link = CurrentThread::getWriteResourceLink()) - Session::setSendDataHooks(std::make_shared(ResourceGuard::Metrics::getIOWrite(), link)); + Session::setSendDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIOWrite(), log, request.getMethod(), request.getURI())); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); From 383221b04d57948364514106b2f48b03140bfbd0 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 23 Jul 2024 10:36:08 +0000 Subject: [PATCH 015/154] Analyzer: Storage Join support joining with nullable columns --- src/Planner/PlannerJoins.cpp | 8 +- ...multiple_joins_with_storage_join.reference | 23 +++++ ...03208_multiple_joins_with_storage_join.sql | 84 +++++++++++++++++++ 3 files changed, 114 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference create mode 100644 tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql diff --git a/src/Planner/PlannerJoins.cpp b/src/Planner/PlannerJoins.cpp index d9360a58240..04011233ca1 100644 --- a/src/Planner/PlannerJoins.cpp +++ b/src/Planner/PlannerJoins.cpp @@ -494,6 +494,12 @@ JoinClausesAndActions buildJoinClausesAndActions( necessary_names.push_back(name); }; + bool is_join_with_special_storage = false; + if (const auto * right_table_node = join_node.getRightTableExpression()->as()) + { + is_join_with_special_storage = dynamic_cast(right_table_node->getStorage().get()); + } + for (auto & join_clause : result.join_clauses) { const auto & left_filter_condition_nodes = join_clause.getLeftFilterConditionNodes(); @@ -561,7 +567,7 @@ JoinClausesAndActions buildJoinClausesAndActions( if (!left_key_node->result_type->equals(*common_type)) left_key_node = &left_join_actions->addCast(*left_key_node, common_type, {}); - if (!right_key_node->result_type->equals(*common_type)) + if (!is_join_with_special_storage && !right_key_node->result_type->equals(*common_type)) right_key_node = &right_join_actions->addCast(*right_key_node, common_type, {}); } diff --git a/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference new file mode 100644 index 00000000000..5ebdaa1c81f --- /dev/null +++ b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.reference @@ -0,0 +1,23 @@ +----- +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +----- +\N \N \N 0 3 B +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +----- +1 1 1 a 1 A 1 A +2 2 2 b 2 B 2 B +\N \N \N \N 3 B \N \N +\N \N \N \N \N \N 3 B +----- +\N \N \N 3 3 B 0 0 +\N \N \N 0 0 3 3 B +1 1 1 a 1 1 A 1 1 A +2 2 2 b 2 2 B 2 2 B +----- +3 3 \N B B +1 1 1 a A A +2 2 2 b B B +----- +7 diff --git a/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql new file mode 100644 index 00000000000..83be4c3f1d2 --- /dev/null +++ b/tests/queries/0_stateless/03208_multiple_joins_with_storage_join.sql @@ -0,0 +1,84 @@ +#!/usr/bin/env -S ${HOME}/clickhouse-client --queries-file + +DROP TABLE IF EXISTS tab; +CREATE TABLE tab ( `k` Nullable(UInt32), `k1` Nullable(UInt32), `k2` Nullable(UInt32), `v` String ) ENGINE = Memory; +INSERT INTO tab VALUES (1, 1, 1, 'a'), (2, 2, 2, 'b'); + +DROP TABLE IF EXISTS mem; +CREATE TABLE mem ( `k` UInt64, `v` String ) ENGINE = Join(ANY, LEFT, k); +INSERT INTO mem VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem2; +CREATE TABLE mem2 ( `k` UInt64, `v` String ) ENGINE = Join(ANY, RIGHT, k); +INSERT INTO mem2 VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem3; +CREATE TABLE mem3 ( `k` UInt64, `v` String ) ENGINE = Join(ALL, FULL, k) SETTINGS join_use_nulls = 1; +INSERT INTO mem3 VALUES (1, 'A'), (2, 'B'), (3, 'B'); + +DROP TABLE IF EXISTS mem4; +CREATE TABLE mem4 ( `k1` UInt64, `k2` UInt64, `v` String ) ENGINE = Join(ALL, FULL, k1, k2); +INSERT INTO mem4 VALUES (1, 1, 'A'), (2, 2, 'B'), (3, 3, 'B'); + +SET allow_experimental_analyzer = 1; + +SELECT '-----'; + +SELECT * +FROM tab +ANY LEFT JOIN mem ON k1 = mem.k +ANY LEFT JOIN mem AS t ON k2 = t.k +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +ANY LEFT JOIN mem ON k1 = mem.k +ANY RIGHT JOIN mem2 ON k2 = mem2.k +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem3 AS t1 ON k1 = t1.k +FULL JOIN mem3 AS t2 ON k2 = t2.k +ORDER BY tab.v +SETTINGS join_use_nulls = 1 +; +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem4 AS t1 ON tab.k1 = t1.k1 AND tab.k2 = t1.k2 +FULL JOIN mem4 AS t2 ON tab.k1 = t2.k1 AND tab.k2 = t2.k2 +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT * +FROM tab +FULL JOIN mem4 AS t1 USING (k1, k2) +FULL JOIN mem4 AS t2 USING (k1, k2) +ORDER BY tab.v +; + +SELECT '-----'; + +SELECT count() FROM ( + EXPLAIN PLAN + SELECT * FROM tab + ANY LEFT JOIN mem AS t1 ON tab.k = t1.k + ANY LEFT JOIN mem AS t2 ON tab.k = t2.k + ANY LEFT JOIN mem AS t3 ON tab.k = t3.k + ANY LEFT JOIN mem AS t4 ON tab.k = t4.k + ANY RIGHT JOIN mem2 AS t5 ON tab.k = t5.k + ANY LEFT JOIN mem AS t6 ON tab.k = t6.k + ANY LEFT JOIN mem AS t7 ON tab.k = t7.k +) +WHERE explain like '%FilledJoin%' +; From 16d7ff11be71df496a960ba833649df2a452a24b Mon Sep 17 00:00:00 2001 From: haozelong Date: Tue, 30 Jul 2024 17:16:19 +0800 Subject: [PATCH 016/154] Fix the problem that alter modfiy order by causes inconsistent metadata --- src/Storages/AlterCommands.cpp | 12 ++++++++++++ .../integration/test_replicated_database/test.py | 16 ++++++++++++++++ 2 files changed, 28 insertions(+) diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7891042bb96..2bef3319020 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -1245,6 +1245,13 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) { auto columns = metadata.columns; + auto ast_to_str = [](const ASTPtr & query) -> String + { + if (!query) + return ""; + return queryToString(query); + }; + for (size_t i = 0; i < size(); ++i) { auto & command = (*this)[i]; @@ -1277,6 +1284,11 @@ void AlterCommands::prepare(const StorageInMemoryMetadata & metadata) if (!has_column && command.if_exists) command.ignore = true; } + else if (command.type == AlterCommand::MODIFY_ORDER_BY) + { + if (ast_to_str(command.order_by) == ast_to_str(metadata.sorting_key.definition_ast)) + command.ignore = true; + } } prepared = true; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 60a6e099b22..6344a4a6b00 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1549,3 +1549,19 @@ def test_all_groups_cluster(started_cluster): assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" ) + +def test_alter_modify_order_by(started_cluster): + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + + main_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');") + main_node.query("CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);") + main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") + snapshotting_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');") + + query = "show create table alter_modify_order_by.t1;" + expected = main_node.query(query) + assert_eq_with_retry(snapshotting_node, query, expected) + + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") From ec759699f3360df53bf21fa6149bfec1e3a08a8c Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 30 Jul 2024 21:07:59 +0800 Subject: [PATCH 017/154] format the tests/integration/test_replicated_database/test.py --- tests/integration/test_replicated_database/test.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 6344a4a6b00..72d45e45841 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1550,14 +1550,21 @@ def test_all_groups_cluster(started_cluster): "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" ) + def test_alter_modify_order_by(started_cluster): main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - main_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');") - main_node.query("CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);") + main_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');" + ) + main_node.query( + "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" + ) main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") - snapshotting_node.query("CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');") + snapshotting_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');" + ) query = "show create table alter_modify_order_by.t1;" expected = main_node.query(query) From 5ebb2c54b049ff740e30e8c2da6d5fe591e3e493 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Fri, 2 Aug 2024 14:07:44 +0800 Subject: [PATCH 018/154] Re-trigger ci --- tests/integration/test_replicated_database/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 72d45e45841..3a640897530 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1552,8 +1552,8 @@ def test_all_groups_cluster(started_cluster): def test_alter_modify_order_by(started_cluster): - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") main_node.query( "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');" @@ -1566,9 +1566,9 @@ def test_alter_modify_order_by(started_cluster): "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');" ) - query = "show create table alter_modify_order_by.t1;" + query = "show create table alter_modify_order_by.t1" expected = main_node.query(query) assert_eq_with_retry(snapshotting_node, query, expected) - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") From 3bff7ddcf8891d091bc5be2b827172029fb8b76f Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 2 Aug 2024 13:19:00 +0000 Subject: [PATCH 019/154] fix data race: delay reset of data hooks until the next sendRequest() --- src/Common/HTTPConnectionPool.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index acddcc8530d..68c13838c04 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -394,8 +394,12 @@ private: // Reset data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIORead(), log, request.getMethod(), request.getURI())); + else + Session::setReceiveDataHooks(); if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIOWrite(), log, request.getMethod(), request.getURI())); + else + Session::setSendDataHooks(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); @@ -453,8 +457,6 @@ private: } } response_stream = nullptr; - Session::setSendDataHooks(); - Session::setReceiveDataHooks(); group->atConnectionDestroy(); From 4a0300736db6ec2898ffdf82fa574b3466e5f44e Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 6 Aug 2024 12:46:22 +0800 Subject: [PATCH 020/154] empty commit From 222079e0323378ee56da7f42dca53e9fc5b3c2f1 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 6 Aug 2024 12:48:52 +0800 Subject: [PATCH 021/154] empty commit From 7d42101593e4f30b5d423ee38d169a76352c2b0d Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Wed, 7 Aug 2024 14:19:40 +0800 Subject: [PATCH 022/154] Re-trigger ci --- .../test_replicated_database/test.py | 23 ------ .../__init__.py | 0 .../configs/config.xml | 10 +++ .../configs/settings.xml | 15 ++++ .../test.py | 70 +++++++++++++++++++ 5 files changed, 95 insertions(+), 23 deletions(-) create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/__init__.py create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml create mode 100644 tests/integration/test_replicated_database_alter_modify_order_by/test.py diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 3a640897530..60a6e099b22 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -1549,26 +1549,3 @@ def test_all_groups_cluster(started_cluster): assert "bad_settings_node\ndummy_node\n" == bad_settings_node.query( "select host_name from system.clusters where name='all_groups.db_cluster' order by host_name" ) - - -def test_alter_modify_order_by(started_cluster): - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") - - main_node.query( - "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard1', 'replica1');" - ) - main_node.query( - "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" - ) - main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") - snapshotting_node.query( - "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', 'shard2', 'replica1');" - ) - - query = "show create table alter_modify_order_by.t1" - expected = main_node.query(query) - assert_eq_with_retry(snapshotting_node, query, expected) - - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC") diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/__init__.py b/tests/integration/test_replicated_database_alter_modify_order_by/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml b/tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml new file mode 100644 index 00000000000..706628cf93b --- /dev/null +++ b/tests/integration/test_replicated_database_alter_modify_order_by/configs/config.xml @@ -0,0 +1,10 @@ + + 10 + 1 + + 10 + + 50 + 42 + false + diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml new file mode 100644 index 00000000000..c637fe8eead --- /dev/null +++ b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml @@ -0,0 +1,15 @@ + + + + 1 + 1 + 0 + 0 + + + + + default + + + diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/test.py b/tests/integration/test_replicated_database_alter_modify_order_by/test.py new file mode 100644 index 00000000000..1ea126ace10 --- /dev/null +++ b/tests/integration/test_replicated_database_alter_modify_order_by/test.py @@ -0,0 +1,70 @@ +import re +import pytest + + +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry, assert_logs_contain + + +cluster = ClickHouseCluster(__file__) + +main_node = cluster.add_instance( + "main_node", + main_configs=["configs/config.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + stay_alive=True, + macros={"shard": 1, "replica": 1}, +) + +snapshotting_node = cluster.add_instance( + "snapshotting_node", + main_configs=["configs/config.xml"], + user_configs=["configs/settings.xml"], + with_zookeeper=True, + macros={"shard": 2, "replica": 1}, +) + + +all_nodes = [ + main_node, + snapshotting_node, +] + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_alter_modify_order_by(started_cluster): + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + + main_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" + ) + main_node.query( + "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" + ) + main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") + snapshotting_node.query( + "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" + ) + + query = ( + "select count() from system.tables where database = 'alter_modify_order_by';" + ) + expected = main_node.query(query) + assert_eq_with_retry(snapshotting_node, query, expected) + + query = "show create table alter_modify_order_by.t1;" + assert main_node.query(query) == snapshotting_node.query(query) + + main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") From 3a89b7034fc9f6b6012a8eb7c277bca72b7b17c2 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Wed, 7 Aug 2024 19:16:22 +0800 Subject: [PATCH 023/154] modify integration test --- .../configs/settings.xml | 4 -- .../test.py | 38 +++++++++---------- 2 files changed, 19 insertions(+), 23 deletions(-) diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml index c637fe8eead..16caee9ba20 100644 --- a/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml +++ b/tests/integration/test_replicated_database_alter_modify_order_by/configs/settings.xml @@ -1,10 +1,6 @@ - 1 - 1 - 0 - 0 diff --git a/tests/integration/test_replicated_database_alter_modify_order_by/test.py b/tests/integration/test_replicated_database_alter_modify_order_by/test.py index 1ea126ace10..22355817ee6 100644 --- a/tests/integration/test_replicated_database_alter_modify_order_by/test.py +++ b/tests/integration/test_replicated_database_alter_modify_order_by/test.py @@ -1,15 +1,14 @@ -import re import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry, assert_logs_contain +from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) -main_node = cluster.add_instance( - "main_node", +shard1_node = cluster.add_instance( + "shard1_node", main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, @@ -17,18 +16,19 @@ main_node = cluster.add_instance( macros={"shard": 1, "replica": 1}, ) -snapshotting_node = cluster.add_instance( - "snapshotting_node", +shard2_node = cluster.add_instance( + "shard2_node", main_configs=["configs/config.xml"], user_configs=["configs/settings.xml"], with_zookeeper=True, + stay_alive=True, macros={"shard": 2, "replica": 1}, ) all_nodes = [ - main_node, - snapshotting_node, + shard1_node, + shard2_node, ] @@ -43,28 +43,28 @@ def started_cluster(): def test_alter_modify_order_by(started_cluster): - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard1_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard2_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - main_node.query( + shard1_node.query( "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" ) - main_node.query( + shard1_node.query( "CREATE TABLE alter_modify_order_by.t1 (id Int64, score Int64) ENGINE = ReplicatedMergeTree('/test/tables/{uuid}/{shard}', '{replica}') ORDER BY (id);" ) - main_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") - snapshotting_node.query( + shard1_node.query("ALTER TABLE alter_modify_order_by.t1 modify order by (id);") + shard2_node.query( "CREATE DATABASE alter_modify_order_by ENGINE = Replicated('/test/database/alter_modify_order_by', '{shard}', '{replica}');" ) query = ( "select count() from system.tables where database = 'alter_modify_order_by';" ) - expected = main_node.query(query) - assert_eq_with_retry(snapshotting_node, query, expected) + expected = shard1_node.query(query) + assert_eq_with_retry(shard2_node, query, expected) query = "show create table alter_modify_order_by.t1;" - assert main_node.query(query) == snapshotting_node.query(query) + assert shard1_node.query(query) == shard2_node.query(query) - main_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") - snapshotting_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard1_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") + shard2_node.query("DROP DATABASE IF EXISTS alter_modify_order_by SYNC;") From eee3811e89ed229ffedb26e97200b80edf399dcd Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Thu, 8 Aug 2024 00:43:37 +0800 Subject: [PATCH 024/154] empty commit From 7a4bd49c4285ef3489d90aa163fb6b1858a43562 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 14 Aug 2024 17:16:36 +0000 Subject: [PATCH 025/154] More improvements in integration test --- .../test_alter_settings_on_cluster/test.py | 5 +++ .../test_always_fetch_merged/test.py | 3 ++ .../test.py | 6 +-- .../test_async_load_databases/test.py | 14 +++++-- .../test.py | 42 +++++++++---------- 5 files changed, 42 insertions(+), 28 deletions(-) diff --git a/tests/integration/test_alter_settings_on_cluster/test.py b/tests/integration/test_alter_settings_on_cluster/test.py index 32f7f2efa30..7a552c383c9 100644 --- a/tests/integration/test_alter_settings_on_cluster/test.py +++ b/tests/integration/test_alter_settings_on_cluster/test.py @@ -73,3 +73,8 @@ def test_default_database_on_cluster(started_cluster): database="test_default_database", sql="SHOW CREATE test_local_table FORMAT TSV", ).endswith("old_parts_lifetime = 100\n") + + ch1.query( + database="test_default_database", + sql="DROP TABLE test_local_table ON CLUSTER 'cluster' SYNC", + ) diff --git a/tests/integration/test_always_fetch_merged/test.py b/tests/integration/test_always_fetch_merged/test.py index ca8e775fb97..3bbfc8867f8 100644 --- a/tests/integration/test_always_fetch_merged/test.py +++ b/tests/integration/test_always_fetch_merged/test.py @@ -80,3 +80,6 @@ def test_replica_always_download(started_cluster): assert int(node1_parts) < 10 assert int(node2_parts) < 10 + + node1.query_with_retry("DROP TABLE test_table SYNC") + node2.query_with_retry("DROP TABLE test_table SYNC") diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 93319a56d0f..0ea076b1468 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -157,7 +157,7 @@ def test_with_replicated_merge_tree(): array_size_range=[10, 50], ) - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE {} SYNC".format(table_name)) def test_with_replicated_merge_tree_multithread(): @@ -185,7 +185,7 @@ def test_with_replicated_merge_tree_multithread(): array_size_range=[10, 15], ) - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE {} SYNC".format(table_name)) # Ensure that the combined duration of inserts with adaptive timeouts is less than @@ -369,4 +369,4 @@ def test_change_queries_frequency(): for line in res.splitlines(): assert int(line) == min_ms - node.query("DROP TABLE IF EXISTS {}".format(table_name)) + node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index d06897b1045..f36cff76ea2 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -28,9 +28,6 @@ def started_cluster(): """ CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; CREATE DATABASE IF NOT EXISTS test; - DROP TABLE IF EXISTS test.elements; - CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log; - INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7); """ ) @@ -49,6 +46,13 @@ def get_status(dictionary_name): def test_dict_get_data(started_cluster): query = instance.query + query( + "CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log;" + ) + query( + "INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7);" + ) + # dictionaries_lazy_load == false, so these dictionary are not loaded. assert get_status("dep_x") == "NOT_LOADED" assert get_status("dep_y") == "NOT_LOADED" @@ -97,6 +101,8 @@ def test_dict_get_data(started_cluster): assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "XX\n" assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ZZ\n" + query("DROP TABLE IF EXISTS test.elements;") + instance.restart_clickhouse() def dependent_tables_assert(): @@ -175,3 +181,5 @@ def test_multiple_tables(started_cluster): random.shuffle(order) for i in order: assert query(f"select count() from test.table_{i}") == "100\n" + for i in range(tables_count): + query(f"drop table test.table_{i}") diff --git a/tests/integration/test_asynchronous_metric_log_table/test.py b/tests/integration/test_asynchronous_metric_log_table/test.py index 622620e232a..efdd96004d2 100644 --- a/tests/integration/test_asynchronous_metric_log_table/test.py +++ b/tests/integration/test_asynchronous_metric_log_table/test.py @@ -26,26 +26,24 @@ def started_cluster(): # asynchronous_metric_update_period_s is being set to 2s so that the metrics are populated faster and # are available for querying during the test. def test_event_time_microseconds_field(started_cluster): - try: - cluster.start() - node1.query("SET log_queries = 1;") - node1.query("CREATE DATABASE replica;") - query_create = """CREATE TABLE replica.test - ( - id Int64, - event_time DateTime - ) - Engine=MergeTree() - PARTITION BY toYYYYMMDD(event_time) - ORDER BY id;""" - time.sleep(2) - node1.query(query_create) - node1.query("""INSERT INTO replica.test VALUES (1, now())""") - node1.query("SYSTEM FLUSH LOGS;") + node1.query("SET log_queries = 1;") + node1.query("CREATE DATABASE replica;") + query_create = """CREATE TABLE replica.test + ( + id Int64, + event_time DateTime + ) + Engine=MergeTree() + PARTITION BY toYYYYMMDD(event_time) + ORDER BY id;""" + time.sleep(2) + node1.query(query_create) + node1.query("""INSERT INTO replica.test VALUES (1, now())""") + node1.query("SYSTEM FLUSH LOGS;") - test_query = ( - "SELECT count() > 0 ? 'ok' : 'fail' FROM system.asynchronous_metric_log" - ) - assert "ok\n" in node1.query(test_query) - finally: - cluster.shutdown() + test_query = ( + "SELECT count() > 0 ? 'ok' : 'fail' FROM system.asynchronous_metric_log" + ) + assert "ok\n" in node1.query(test_query) + node1.query("DROP TABLE replica.test") + node1.query("DROP DATABASE replica") From d31b36ca50d8905b8eaac945c197c425d5ddb208 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 07:16:01 +0000 Subject: [PATCH 026/154] Rework integration flaky check as it was timeouting too fast as run as a single group --- tests/ci/integration_tests_runner.py | 98 +++++++++++++++++----------- 1 file changed, 60 insertions(+), 38 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 2b348be8b51..1513a8bee4f 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -784,40 +784,65 @@ class ClickhouseIntegrationTestsRunner: logging.info("Starting check with retries") final_retry = 0 logs = [] - tires_num = 1 if should_fail else FLAKY_TRIES_COUNT - for i in range(tires_num): - final_retry += 1 - logging.info("Running tests for the %s time", i) - counters, tests_times, log_paths = self.try_run_test_group( - repo_path, - "bugfix" if should_fail else "flaky", - tests_to_run, - 1, - 1, - FLAKY_REPEAT_COUNT, - ) - logs += log_paths - if counters["FAILED"]: - logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) - description_prefix = "Failed tests found: " - result_state = "failure" - if not should_fail: + counters = { + "ERROR": [], + "PASSED": [], + "FAILED": [], + "SKIPPED": [], + "BROKEN": [], + "NOT_FAILED": [], + } # type: Dict + tests_times = defaultdict(float) # type: Dict + tests_log_paths = defaultdict(list) + + for test_to_run in tests_to_run: + tries_num = 1 if should_fail else FLAKY_TRIES_COUNT + for i in range(tries_num): + final_retry += 1 + logging.info("Running tests for the %s time", i) + group_counters, group_test_times, log_paths = self.try_run_test_group( + repo_path, + "bugfix" if should_fail else "flaky", + [test_to_run], + 1, + 1, + FLAKY_REPEAT_COUNT, + ) + for counter, value in group_counters.items(): + logging.info( + "Tests from group %s stats, %s count %s", + test_to_run, + counter, + len(value), + ) + counters[counter] += value + + for test_name, test_time in group_test_times.items(): + tests_times[test_name] = test_time + tests_log_paths[test_name] = log_paths + if not should_fail and ( + group_counters["FAILED"] or group_counters["ERROR"] + ): + logging.info( + "Unexpected failure in group %s. Fail fast for current group", + test_to_run, + ) break - if counters["ERROR"]: - description_prefix = "Failed tests found: " - logging.info("Found error tests: %s", " ".join(counters["ERROR"])) - # NOTE "error" result state will restart the whole test task, - # so we use "failure" here - result_state = "failure" - if not should_fail: - break - logging.info("Try is OK, all tests passed, going to clear env") - clear_ip_tables_and_restart_daemons() - logging.info("And going to sleep for some time") - if time.time() - start > MAX_TIME_SECONDS: - logging.info("Timeout reached, going to finish flaky check") - break - time.sleep(5) + + if group_counters["FAILED"]: + logging.info("Found failed tests: %s", " ".join(counters["FAILED"])) + description_prefix = "Failed tests found: " + result_state = "failure" + if group_counters["ERROR"]: + description_prefix = "Failed tests found: " + logging.info("Found error tests: %s", " ".join(counters["ERROR"])) + # NOTE "error" result state will restart the whole test task, + # so we use "failure" here + result_state = "failure" + logging.info("Try is OK, all tests passed, going to clear env") + clear_ip_tables_and_restart_daemons() + logging.info("And going to sleep for some time") + time.sleep(5) test_result = [] for state in ("ERROR", "FAILED", "PASSED", "SKIPPED"): @@ -828,13 +853,10 @@ class ClickhouseIntegrationTestsRunner: else: text_state = state test_result += [ - ( - c + " (✕" + str(final_retry) + ")", - text_state, - f"{tests_times[c]:.2f}", - ) + (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] + status_text = description_prefix + ", ".join( [ str(n).lower().replace("failed", "fail") + ": " + str(len(c)) From c48b6d25f763d9c38b4e7ed35dec9a98d913e5c7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 08:58:36 +0000 Subject: [PATCH 027/154] more tests --- .../test_filesystem_layout/test.py | 4 ++++ .../integration/test_grant_and_revoke/test.py | 2 ++ .../test_parallel_replicas_failover/test.py | 19 ++++++++++--------- .../test.py | 4 ++-- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 4e719aa0fe9..31d6c830a2f 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -79,3 +79,7 @@ def test_file_path_escaping(started_cluster): "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin", ] ) + node.query("DROP TABLE test.`T.a_b,l-e!` SYNC") + node.query("DROP TABLE `test 2`.`T.a_b,l-e!` SYNC") + node.query("DROP DATABASE test") + node.query("DROP DATABASE `test 2`") diff --git a/tests/integration/test_grant_and_revoke/test.py b/tests/integration/test_grant_and_revoke/test.py index e533cced1e4..81cba966cae 100644 --- a/tests/integration/test_grant_and_revoke/test.py +++ b/tests/integration/test_grant_and_revoke/test.py @@ -359,6 +359,8 @@ def test_implicit_create_view_grant(): instance.query("GRANT CREATE VIEW ON test.* TO B", user="A") instance.query("CREATE VIEW test.view_2 AS SELECT 1", user="B") assert instance.query("SELECT * FROM test.view_2") == "1\n" + instance.query("DROP USER A") + instance.query("DROP VIEW test.view_2") def test_implicit_create_temporary_table_grant(): diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index bf25136bff7..2da26ee03c9 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -1,5 +1,5 @@ import pytest - +import uuid from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -25,19 +25,15 @@ def start_cluster(): def create_tables(cluster, table_name, skip_last_replica): - node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node2.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node2.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" ) if not skip_last_replica: node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data @@ -67,7 +63,7 @@ def test_skip_replicas_without_table(start_cluster): for i in range(4): expected_result += f"{i}\t1000\n" - log_comment = "5230b069-9574-407d-9b80-891b5a175f41" + log_comment = uuid.uuid4() assert ( node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", @@ -88,6 +84,8 @@ def test_skip_replicas_without_table(start_cluster): ) == "1\t1\n" ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") def test_skip_unresponsive_replicas(start_cluster): @@ -112,3 +110,6 @@ def test_skip_unresponsive_replicas(start_cluster): ) == expected_result ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") + node3.query(f"DROP TABLE {table_name} SYNC") diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index cab3fb46fe9..7093e3b3292 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -35,11 +35,10 @@ def start_cluster(): def _create_tables(table_name, table_size, index_granularity): - nodes[0].query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER {cluster_name}") nodes[0].query( f""" - CREATE TABLE IF NOT EXISTS {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) + CREATE TABLE {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}/', '{{replica}}') ORDER BY (key) SETTINGS index_granularity = {index_granularity}, max_bytes_to_merge_at_max_space_in_pool = 0, max_bytes_to_merge_at_max_space_in_pool = 1 @@ -128,3 +127,4 @@ def test_reading_with_invisible_parts( ) == f"{expected}\n" ) + nodes[0].query(f"DROP TABLE {table_name} ON CLUSTER {cluster_name} SYNC") From 7c691cbb797594d0c6649b36be13ff99c3c4664b Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Fri, 16 Aug 2024 10:44:11 +0200 Subject: [PATCH 028/154] style --- tests/ci/integration_tests_runner.py | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index f63afc297e6..006958abe6d 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -774,10 +774,8 @@ class ClickhouseIntegrationTestsRunner: logging.info("Found '%s' tests to run", " ".join(tests_to_run)) result_state = "success" description_prefix = "No flaky tests: " - start = time.time() logging.info("Starting check with retries") final_retry = 0 - logs = [] counters = { "ERROR": [], "PASSED": [], @@ -858,7 +856,7 @@ class ClickhouseIntegrationTestsRunner: ] ) - return result_state, status_text, test_result, logs + return result_state, status_text, test_result, tests_log_paths def run_impl(self, repo_path, build_path): if self.flaky_check or self.bugfix_validate_check: From 32f4b1f89143a293047beb83169d83c3e20c5480 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 17:49:38 +0000 Subject: [PATCH 029/154] Add aggregate functions distinctDynamicTypes/distinctJSONPaths/distinctJSONPathsAndTypes --- .../reference/distinctdynamictypes.md | 44 +++ .../reference/distinctjsonpaths.md | 84 +++++ docs/en/sql-reference/data-types/newjson.md | 125 ++++++- .../AggregateFunctionDistinctDynamicTypes.cpp | 155 ++++++++ .../AggregateFunctionDistinctJSONPaths.cpp | 331 ++++++++++++++++++ .../registerAggregateFunctions.cpp | 4 + src/Columns/ColumnDynamic.cpp | 35 ++ src/Columns/ColumnDynamic.h | 3 + ...istinct_dynamic_types_json_paths.reference | 121 +++++++ ...3227_distinct_dynamic_types_json_paths.sql | 50 +++ 10 files changed, 951 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md create mode 100644 docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md create mode 100644 src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp create mode 100644 tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference create mode 100644 tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md b/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md new file mode 100644 index 00000000000..970209252fa --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctdynamictypes.md @@ -0,0 +1,44 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/distinctdynamictypes +sidebar_position: 215 +--- + +# distinctDynamicTypes + +Calculates the list of distinct data types stored in [Dynamic](../../data-types/dynamic.md) column. + +**Syntax** + +```sql +distinctDynamicTypes(dynamic) +``` + +**Arguments** + +- `dynamic` — [Dynamic](../../data-types/dynamic.md) column. + +**Returned Value** + +- The sorted list of data type names [Array(String)](../../data-types/array.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_dynamic; +CREATE TABLE test_dynamic(d Dynamic) ENGINE = Memory; +INSERT INTO test_dynamic VALUES (42), (NULL), ('Hello'), ([1, 2, 3]), ('2020-01-01'), (map(1, 2)), (43), ([4, 5]), (NULL), ('World'), (map(3, 4)) +``` + +```sql +SELECT distinctDynamicTypes(d) FROM test_dynamic; +``` + +Result: + +```reference +┌─distinctDynamicTypes(d)──────────────────────────────────────┐ +│ ['Array(Int64)','Date','Int64','Map(UInt8, UInt8)','String'] │ +└──────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md new file mode 100644 index 00000000000..f916734ca44 --- /dev/null +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md @@ -0,0 +1,84 @@ +--- +slug: /en/sql-reference/aggregate-functions/reference/distinctjsonpaths +sidebar_position: 216 +--- + +# distinctJSONPaths + +Calculates the list of distinct paths stored in [JSON](../../data-types/newjson.md) column. + +**Syntax** + +```sql +distinctJSONPaths(json) +``` + +**Arguments** + +- `json` — [JSON](../../data-types/newjson.md) column. + +**Returned Value** + +- The sorted list of paths [Array(String)](../../data-types/array.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"a" : 42, "b" : "Hello"}'), ('{"b" : [1, 2, 3], "c" : {"d" : {"e" : "2020-01-01"}}}'), ('{"a" : 43, "c" : {"d" : {"f" : [{"g" : 42}]}}}') +``` + +```sql +SELECT distinctJSONPaths(json) FROM test_json; +``` + +Result: + +```reference +┌─distinctJSONPaths(json)───┐ +│ ['a','b','c.d.e','c.d.f'] │ +└───────────────────────────┘ +``` + +# distinctJSONPathsAndTypes + +Calculates the list of distinct paths and their types stored in [JSON](../../data-types/newjson.md) column. + +**Syntax** + +```sql +distinctJSONPathsAndTypes(json) +``` + +**Arguments** + +- `json` — [JSON](../../data-types/newjson.md) column. + +**Returned Value** + +- The sorted map of paths and types [Map(String, Array(String))](../../data-types/map.md). + +**Example** + +Query: + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"a" : 42, "b" : "Hello"}'), ('{"b" : [1, 2, 3], "c" : {"d" : {"e" : "2020-01-01"}}}'), ('{"a" : 43, "c" : {"d" : {"f" : [{"g" : 42}]}}}') +``` + +```sql +SELECT distinctJSONPathsAndTypes(json) FROM test_json; +``` + +Result: + +```reference +┌─distinctJSONPathsAndTypes(json)───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ +│ {'a':['Int64'],'b':['Array(Nullable(Int64))','String'],'c.d.e':['Date'],'c.d.f':['Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))']} │ +└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ +``` diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 9e43216df6c..16cc324a168 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -505,7 +505,130 @@ As we can see, ClickHouse kept the most frequent paths `a`, `b` and `c` and move ## Introspection functions -There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes). +There are several functions that can help to inspect the content of the JSON column: [JSONAllPaths](../functions/json-functions.md#jsonallpaths), [JSONAllPathsWithTypes](../functions/json-functions.md#jsonallpathswithtypes), [JSONDynamicPaths](../functions/json-functions.md#jsondynamicpaths), [JSONDynamicPathsWithTypes](../functions/json-functions.md#jsondynamicpathswithtypes), [JSONSharedDataPaths](../functions/json-functions.md#jsonshareddatapaths), [JSONSharedDataPathsWithTypes](../functions/json-functions.md#jsonshareddatapathswithtypes), [distinctDynamicTypes](../aggregate-functions/reference/distinctdynamictypes.md), [distinctJSONPaths and distinctJSONPathsAndTypes](../aggregate-functions/reference/distinctjsonpaths.md) + +**Examples** + +Let's investigate the content of [GH Archive](https://www.gharchive.org/) dataset for `2020-01-01` date: + +```sql +SELECT arrayJoin(distinctJSONPaths(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +``` + +```text +┌─arrayJoin(distinctJSONPaths(json))─────────────────────────┐ +│ actor.avatar_url │ +│ actor.display_login │ +│ actor.gravatar_id │ +│ actor.id │ +│ actor.login │ +│ actor.url │ +│ created_at │ +│ id │ +│ org.avatar_url │ +│ org.gravatar_id │ +│ org.id │ +│ org.login │ +│ org.url │ +│ payload.action │ +│ payload.before │ +│ payload.comment._links.html.href │ +│ payload.comment._links.pull_request.href │ +│ payload.comment._links.self.href │ +│ payload.comment.author_association │ +│ payload.comment.body │ +│ payload.comment.commit_id │ +│ payload.comment.created_at │ +│ payload.comment.diff_hunk │ +│ payload.comment.html_url │ +│ payload.comment.id │ +│ payload.comment.in_reply_to_id │ +│ payload.comment.issue_url │ +│ payload.comment.line │ +│ payload.comment.node_id │ +│ payload.comment.original_commit_id │ +│ payload.comment.original_position │ +│ payload.comment.path │ +│ payload.comment.position │ +│ payload.comment.pull_request_review_id │ +... +│ payload.release.node_id │ +│ payload.release.prerelease │ +│ payload.release.published_at │ +│ payload.release.tag_name │ +│ payload.release.tarball_url │ +│ payload.release.target_commitish │ +│ payload.release.upload_url │ +│ payload.release.url │ +│ payload.release.zipball_url │ +│ payload.size │ +│ public │ +│ repo.id │ +│ repo.name │ +│ repo.url │ +│ type │ +└─arrayJoin(distinctJSONPaths(json))─────────────────────────┘ +``` + +```sql +SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +``` + + +```text +┌─arrayJoin(distinctJSONPathsAndTypes(json))──────────────────┐ +│ ('actor.avatar_url',['String']) │ +│ ('actor.display_login',['String']) │ +│ ('actor.gravatar_id',['String']) │ +│ ('actor.id',['Int64']) │ +│ ('actor.login',['String']) │ +│ ('actor.url',['String']) │ +│ ('created_at',['String']) │ +│ ('id',['String']) │ +│ ('org.avatar_url',['String']) │ +│ ('org.gravatar_id',['String']) │ +│ ('org.id',['Int64']) │ +│ ('org.login',['String']) │ +│ ('org.url',['String']) │ +│ ('payload.action',['String']) │ +│ ('payload.before',['String']) │ +│ ('payload.comment._links.html.href',['String']) │ +│ ('payload.comment._links.pull_request.href',['String']) │ +│ ('payload.comment._links.self.href',['String']) │ +│ ('payload.comment.author_association',['String']) │ +│ ('payload.comment.body',['String']) │ +│ ('payload.comment.commit_id',['String']) │ +│ ('payload.comment.created_at',['String']) │ +│ ('payload.comment.diff_hunk',['String']) │ +│ ('payload.comment.html_url',['String']) │ +│ ('payload.comment.id',['Int64']) │ +│ ('payload.comment.in_reply_to_id',['Int64']) │ +│ ('payload.comment.issue_url',['String']) │ +│ ('payload.comment.line',['Int64']) │ +│ ('payload.comment.node_id',['String']) │ +│ ('payload.comment.original_commit_id',['String']) │ +│ ('payload.comment.original_position',['Int64']) │ +│ ('payload.comment.path',['String']) │ +│ ('payload.comment.position',['Int64']) │ +│ ('payload.comment.pull_request_review_id',['Int64']) │ +... +│ ('payload.release.node_id',['String']) │ +│ ('payload.release.prerelease',['Bool']) │ +│ ('payload.release.published_at',['String']) │ +│ ('payload.release.tag_name',['String']) │ +│ ('payload.release.tarball_url',['String']) │ +│ ('payload.release.target_commitish',['String']) │ +│ ('payload.release.upload_url',['String']) │ +│ ('payload.release.url',['String']) │ +│ ('payload.release.zipball_url',['String']) │ +│ ('payload.size',['Int64']) │ +│ ('public',['Bool']) │ +│ ('repo.id',['Int64']) │ +│ ('repo.name',['String']) │ +│ ('repo.url',['String']) │ +│ ('type',['String']) │ +└─arrayJoin(distinctJSONPathsAndTypes(json))──────────────────┘ +``` ## Tips for better usage of the JSON type diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp new file mode 100644 index 00000000000..6c899d6cda8 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -0,0 +1,155 @@ +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + +struct AggregateFunctionDistinctDynamicTypesData +{ + std::unordered_set data; + + void add(const String & type) + { + data.insert(type); + } + + void merge(const AggregateFunctionDistinctDynamicTypesData & other) + { + data.insert(other.data.begin(), other.data.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & type : data) + writeStringBinary(type, buf); + } + + void deserialize(ReadBuffer & buf) + { + size_t size; + readVarUInt(size, buf); + data.reserve(size); + String type; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(type, buf); + data.insert(type); + } + } + + void insertResultInto(IColumn & column) + { + /// Insert types in sorted order for better output. + auto & array_column = assert_cast(column); + auto & string_column = assert_cast(array_column.getData()); + std::vector sorted_data(data.begin(), data.end()); + std::sort(sorted_data.begin(), sorted_data.end()); + for (const auto & type : sorted_data) + string_column.insertData(type.data(), type.size()); + array_column.getOffsets().push_back(string_column.size()); + } +}; + +/// Calculates the list of distinct data types in Dynamic column. +class AggregateFunctionDistinctDynamicTypes final : public IAggregateFunctionDataHelper +{ +public: + explicit AggregateFunctionDistinctDynamicTypes(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper(argument_types_, {}, std::make_shared(std::make_shared())) + { + } + + String getName() const override { return "distinctDynamicTypes"; } + + bool allocatesMemoryInArena() const override { return false; } + + void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & dynamic_column = assert_cast(*columns[0]); + if (dynamic_column.isNullAt(row_num)) + return; + + this->data(place).add(dynamic_column.getTypeNameAt(row_num)); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) + const override + { + if (if_argument_pos >= 0 || row_begin != 0 || row_end != columns[0]->size()) + IAggregateFunctionDataHelper::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + /// Optimization for case when we add all rows from the column into single place. + /// In this case we can avoid iterating over all rows because we can get all types + /// in Dynamic column in a more efficient way. + else + assert_cast(*columns[0]).getAllTypeNames(this->data(place).data); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + /// Default value for Dynamic is NULL, so nothing to add. + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + this->data(place).insertResultInto(to); + } +}; + +AggregateFunctionPtr createAggregateFunctionDistinctDynamicTypes( + const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + if (argument_types.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function {}. Expected single argument with type Dynamic, got {} arguments", name, argument_types.size()); + + if (!isDynamic(argument_types[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}. Expected type Dynamic", argument_types[0]->getName(), name); + + return std::make_shared(argument_types); +} + +void registerAggregateFunctionDistinctDynamicTypes(AggregateFunctionFactory & factory) +{ + factory.registerFunction("distinctDynamicTypes", createAggregateFunctionDistinctDynamicTypes); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp new file mode 100644 index 00000000000..a3ede0750f2 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -0,0 +1,331 @@ +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ + +struct AggregateFunctionDistinctJSONPathsData +{ + static constexpr auto name = "distinctJSONPaths"; + + std::unordered_set data; + + void add(const ColumnObject & column, size_t row_num, const std::unordered_map &) + { + for (const auto & [path, _] : column.getTypedPaths()) + data.insert(path); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add path from dynamic paths only if it's not NULL in this row. + if (!dynamic_column->isNullAt(row_num)) + data.insert(path); + } + + /// Iterate over paths in shared data in this row. + const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column.getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(row_num) - 1]; + size_t end = shared_data_offsets[static_cast(row_num)]; + for (size_t i = start; i != end; ++i) + data.insert(shared_data_paths->getDataAt(i).toString()); + } + + void addWholeColumn(const ColumnObject & column, const std::unordered_map &) + { + for (const auto & [path, _] : column.getTypedPaths()) + data.insert(path); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add dynamic path only if it has at least one non-null value. + /// getNumberOfDefaultRows for Dynamic column is O(1). + if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) + data.insert(path); + } + + /// Iterate over all paths in shared data. + const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + data.insert(shared_data_paths->getDataAt(i).toString()); + } + + void merge(const AggregateFunctionDistinctJSONPathsData & other) + { + data.insert(other.data.begin(), other.data.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & path : data) + writeStringBinary(path, buf); + } + + void deserialize(ReadBuffer & buf) + { + size_t size; + readVarUInt(size, buf); + String path; + for (size_t i = 0; i != size; ++i) + { + readStringBinary(path, buf); + data.insert(path); + } + } + + void insertResultInto(IColumn & column) + { + /// Insert paths in sorted order for better output. + auto & array_column = assert_cast(column); + auto & string_column = assert_cast(array_column.getData()); + std::vector sorted_data(data.begin(), data.end()); + std::sort(sorted_data.begin(), sorted_data.end()); + for (const auto & path : sorted_data) + string_column.insertData(path.data(), path.size()); + array_column.getOffsets().push_back(string_column.size()); + } + + static DataTypePtr getResultType() + { + return std::make_shared(std::make_shared()); + } +}; + +struct AggregateFunctionDistinctJSONPathsAndTypesData +{ + static constexpr auto name = "distinctJSONPathsAndTypes"; + + std::unordered_map> data; + + void add(const ColumnObject & column, size_t row_num, const std::unordered_map & typed_paths_type_names) + { + for (const auto & [path, _] : column.getTypedPaths()) + data[path].insert(typed_paths_type_names.at(path)); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + if (!dynamic_column->isNullAt(row_num)) + data[path].insert(dynamic_column->getTypeNameAt(row_num)); + } + + /// Iterate over paths om shared data in this row and decode the data types. + const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); + const auto & shared_data_offsets = column.getSharedDataOffsets(); + size_t start = shared_data_offsets[static_cast(row_num) - 1]; + size_t end = shared_data_offsets[static_cast(row_num)]; + for (size_t i = start; i != end; ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + /// We should not have Nulls here but let's check just in case. + if (!isNothing(type)) + data[path].insert(type->getName()); + } + } + + void addWholeColumn(const ColumnObject & column, const std::unordered_map & typed_paths_type_names) + { + for (const auto & [path, _] : column.getTypedPaths()) + data[path].insert(typed_paths_type_names.at(path)); + for (const auto & [path, dynamic_column] : column.getDynamicPathsPtrs()) + { + /// Add dynamic path only if it has at least one non-null value. + /// getNumberOfDefaultRows for Dynamic column is O(1). + if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) + dynamic_column->getAllTypeNames(data[path]); + } + + /// Iterate over all paths in shared data and decode the data types. + const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); + for (size_t i = 0; i != shared_data_paths->size(); ++i) + { + auto path = shared_data_paths->getDataAt(i).toString(); + auto value = shared_data_values->getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + auto type = decodeDataType(buf); + /// We should not have Nulls here but let's check just in case. + if (!isNothing(type)) + data[path].insert(type->getName()); + } + } + + void merge(const AggregateFunctionDistinctJSONPathsAndTypesData & other) + { + for (const auto & [path, types] : other.data) + data[path].insert(types.begin(), types.end()); + } + + void serialize(WriteBuffer & buf) const + { + writeVarUInt(data.size(), buf); + for (const auto & [path, types] : data) + { + writeStringBinary(path, buf); + writeVarUInt(types.size(), buf); + for (const auto & type : types) + writeStringBinary(type, buf); + } + } + + void deserialize(ReadBuffer & buf) + { + size_t paths_size, types_size; + readVarUInt(paths_size, buf); + data.reserve(paths_size); + String path, type; + for (size_t i = 0; i != paths_size; ++i) + { + readStringBinary(path, buf); + readVarUInt(types_size, buf); + data[path].reserve(types_size); + for (size_t j = 0; j != types_size; ++j) + { + readStringBinary(type, buf); + data[path].insert(type); + } + } + } + + void insertResultInto(IColumn & column) + { + /// Insert sorted paths and types for better output. + auto & array_column = assert_cast(column).getNestedColumn(); + auto & tuple_column = assert_cast(array_column.getData()); + auto & key_column = assert_cast(tuple_column.getColumn(0)); + auto & value_column = assert_cast(tuple_column.getColumn(1)); + auto & value_column_data = assert_cast(value_column.getData()); + std::vector>> sorted_data; + sorted_data.reserve(data.size()); + for (const auto & [path, types] : data) + { + std::vector sorted_types(types.begin(), types.end()); + std::sort(sorted_types.begin(), sorted_types.end()); + sorted_data.emplace_back(path, std::move(sorted_types)); + } + std::sort(sorted_data.begin(), sorted_data.end()); + + for (const auto & [path, types] : sorted_data) + { + key_column.insertData(path.data(), path.size()); + for (const auto & type : types) + value_column_data.insertData(type.data(), type.size()); + value_column.getOffsets().push_back(value_column_data.size()); + } + + array_column.getOffsets().push_back(key_column.size()); + } + + static DataTypePtr getResultType() + { + return std::make_shared(std::make_shared(), std::make_shared(std::make_shared())); + } +}; + +/// Calculates the list of distinct data types in Dynamic column. +template +class AggregateFunctionDistinctJSONPathsAndTypes final : public IAggregateFunctionDataHelper> +{ +public: + explicit AggregateFunctionDistinctJSONPathsAndTypes(const DataTypes & argument_types_) + : IAggregateFunctionDataHelper>( + argument_types_, {}, Data::getResultType()) + { + const auto & typed_paths_types = assert_cast(*argument_types_[0]).getTypedPaths(); + typed_paths_type_names.reserve(typed_paths_types.size()); + for (const auto & [path, type] : typed_paths_types) + typed_paths_type_names[path] = type->getName(); + } + + String getName() const override { return Data::name; } + + bool allocatesMemoryInArena() const override { return false; } + + void ALWAYS_INLINE add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena *) const override + { + const auto & object_column = assert_cast(*columns[0]); + this->data(place).add(object_column, row_num, typed_paths_type_names); + } + + void ALWAYS_INLINE addBatchSinglePlace( + size_t row_begin, size_t row_end, AggregateDataPtr __restrict place, const IColumn ** columns, Arena * arena, ssize_t if_argument_pos) + const override + { + if (if_argument_pos >= 0 || row_begin != 0 || row_end != columns[0]->size()) + IAggregateFunctionDataHelper>::addBatchSinglePlace(row_begin, row_end, place, columns, arena, if_argument_pos); + /// Optimization for case when we add all rows from the column into single place. + /// In this case we can avoid iterating over all rows because we can get all paths + /// and types in JSON column in a more efficient way. + else + this->data(place).addWholeColumn(assert_cast(*columns[0]), typed_paths_type_names); + } + + void addManyDefaults( + AggregateDataPtr __restrict /*place*/, + const IColumn ** /*columns*/, + size_t /*length*/, + Arena * /*arena*/) const override + { + /// Default value for JSON is empty object, so nothing to add. + } + + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override + { + this->data(place).merge(this->data(rhs)); + } + + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override + { + this->data(place).serialize(buf); + } + + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override + { + this->data(place).deserialize(buf); + } + + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override + { + this->data(place).insertResultInto(to); + } + +private: + std::unordered_map typed_paths_type_names; +}; + +template +AggregateFunctionPtr createAggregateFunctionDistinctJSONPathsAndTypes( + const std::string & name, const DataTypes & argument_types, const Array & parameters, const Settings *) +{ + assertNoParameters(name, parameters); + if (argument_types.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Incorrect number of arguments for aggregate function {}. Expected single argument with type JSON, got {} arguments", name, argument_types.size()); + + if (!isObject(argument_types[0])) + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for aggregate function {}. Expected type JSON", argument_types[0]->getName(), name); + + return std::make_shared>(argument_types); +} + +void registerAggregateFunctionDistinctJSONPathsAndTypes(AggregateFunctionFactory & factory) +{ + factory.registerFunction("distinctJSONPaths", createAggregateFunctionDistinctJSONPathsAndTypes); + factory.registerFunction("distinctJSONPathsAndTypes", createAggregateFunctionDistinctJSONPathsAndTypes); +} + +} diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 4ac25e14ee6..068f1aebd87 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -89,6 +89,8 @@ void registerAggregateFunctionAnalysisOfVariance(AggregateFunctionFactory &); void registerAggregateFunctionFlameGraph(AggregateFunctionFactory &); void registerAggregateFunctionKolmogorovSmirnovTest(AggregateFunctionFactory & factory); void registerAggregateFunctionLargestTriangleThreeBuckets(AggregateFunctionFactory & factory); +void registerAggregateFunctionDistinctDynamicTypes(AggregateFunctionFactory & factory); +void registerAggregateFunctionDistinctJSONPathsAndTypes(AggregateFunctionFactory & factory); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -191,6 +193,8 @@ void registerAggregateFunctions() registerAggregateFunctionFlameGraph(factory); registerAggregateFunctionKolmogorovSmirnovTest(factory); registerAggregateFunctionLargestTriangleThreeBuckets(factory); + registerAggregateFunctionDistinctDynamicTypes(factory); + registerAggregateFunctionDistinctJSONPathsAndTypes(factory); registerWindowFunctions(factory); } diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index 1f37add9d2d..b5378e983c6 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -980,6 +980,41 @@ ColumnPtr ColumnDynamic::compress() const }); } +String ColumnDynamic::getTypeNameAt(size_t row_num) const +{ + const auto & variant_col = getVariantColumn(); + size_t discr = variant_col.globalDiscriminatorAt(row_num); + if (discr == ColumnVariant::NULL_DISCRIMINATOR) + return ""; + + if (discr == getSharedVariantDiscriminator()) + { + const auto value = getSharedVariant().getDataAt(variant_col.offsetAt(row_num)); + ReadBufferFromMemory buf(value.data, value.size); + return decodeDataType(buf)->getName(); + } + + return variant_info.variant_names[discr]; +} + +void ColumnDynamic::getAllTypeNames(std::unordered_set & names) const +{ + auto shared_variant_discr = getSharedVariantDiscriminator(); + for (size_t i = 0; i != variant_info.variant_names.size(); ++i) + { + if (i != shared_variant_discr && !variant_column_ptr->getVariantByGlobalDiscriminator(i).empty()) + names.insert(variant_info.variant_names[i]); + } + + const auto & shared_variant = getSharedVariant(); + for (size_t i = 0; i != shared_variant.size(); ++i) + { + const auto value = shared_variant.getDataAt(i); + ReadBufferFromMemory buf(value.data, value.size); + names.insert(decodeDataType(buf)->getName()); + } +} + void ColumnDynamic::prepareForSquashing(const Columns & source_columns) { if (source_columns.empty()) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index 2ae862de3af..c06c31bb8c9 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -430,6 +430,9 @@ public: const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } + String getTypeNameAt(size_t row_num) const; + void getAllTypeNames(std::unordered_set & names) const; + private: void createVariantInfo(const DataTypePtr & variant_type); diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference new file mode 100644 index 00000000000..d6538a1c4eb --- /dev/null +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference @@ -0,0 +1,121 @@ +a0 +a1 +a10 +a11 +a12 +a2 +a3 +a4 +a5 +a6 +a7 +a8 +a9 +('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a1',['String']) +('a10',['Array(Nullable(Int64))','Date','Int64','String']) +('a11',['Array(Nullable(Int64))','Date','Int64','String']) +('a12',['Array(Nullable(Int64))','Date','Int64','String']) +('a2',['Array(Nullable(Int64))','Date','Int64','String']) +('a3',['Array(Nullable(Int64))','Date','Int64','String']) +('a4',['Array(Nullable(Int64))','Date','Int64','String']) +('a5',['Array(Nullable(Int64))','Date','Int64','String']) +('a6',['Array(Nullable(Int64))','Date','Int64','String']) +('a7',['Array(Nullable(Int64))','Date','Int64','String']) +('a8',['Array(Nullable(Int64))','Date','Int64','String']) +('a9',['Array(Nullable(Int64))','Date','Int64','String']) +Array(Nullable(Int64)) +Date +Int64 +String +Array(Nullable(Int64)) +Date +Int64 +String +Filter +a1 +a2 +('a1',['String']) +('a2',['String']) +String +If +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Group by +Array(Nullable(Int64)) ['a1','a2'] +Date ['a1','a2'] +Int64 ['a1','a2'] +None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] +String ['a1','a2'] +Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} +Date {'a1':['String'],'a2':['Date']} +Int64 {'a1':['String'],'a2':['Int64']} +None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +String {'a1':['String'],'a2':['String']} +Array(Nullable(Int64)) ['Array(Nullable(Int64))'] +Date ['Date'] +Int64 ['Int64'] +None [] +String ['String'] +Remote +a0 +a1 +a10 +a11 +a12 +a2 +a3 +a4 +a5 +a6 +a7 +a8 +a9 +('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a1',['String']) +('a10',['Array(Nullable(Int64))','Date','Int64','String']) +('a11',['Array(Nullable(Int64))','Date','Int64','String']) +('a12',['Array(Nullable(Int64))','Date','Int64','String']) +('a2',['Array(Nullable(Int64))','Date','Int64','String']) +('a3',['Array(Nullable(Int64))','Date','Int64','String']) +('a4',['Array(Nullable(Int64))','Date','Int64','String']) +('a5',['Array(Nullable(Int64))','Date','Int64','String']) +('a6',['Array(Nullable(Int64))','Date','Int64','String']) +('a7',['Array(Nullable(Int64))','Date','Int64','String']) +('a8',['Array(Nullable(Int64))','Date','Int64','String']) +('a9',['Array(Nullable(Int64))','Date','Int64','String']) +Array(Nullable(Int64)) +Date +Int64 +String +Remote filter +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Remote if +a1 +a2 +('a1',['String']) +('a2',['String']) +String +Remote group by +Array(Nullable(Int64)) ['a1','a2'] +Date ['a1','a2'] +Int64 ['a1','a2'] +None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] +String ['a1','a2'] +Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} +Date {'a1':['String'],'a2':['Date']} +Int64 {'a1':['String'],'a2':['Int64']} +None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +String {'a1':['String'],'a2':['String']} +Array(Nullable(Int64)) ['Array(Nullable(Int64))'] +Date ['Date'] +Int64 ['Int64'] +None [] +String ['String'] diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql new file mode 100644 index 00000000000..4f33cc10d46 --- /dev/null +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -0,0 +1,50 @@ +set allow_experimental_dynamic_type = 1; +set allow_experimental_json_type = 1; +set allow_experimental_variant_type = 1; +set use_variant_as_common_type = 1; + +drop table if exists test_json_dynamic_aggregate_functions; +create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toDate(number)))) from numbers(200000); +select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a3)) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypes(json.a42)) from test_json_dynamic_aggregate_functions; + +select 'Filter'; +select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions where dynamicType(json.a2) == 'String'; + +select 'If'; +select arrayJoin(distinctJSONPathsIf(json, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctJSONPathsAndTypesIf(json, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; +select arrayJoin(distinctDynamicTypesIf(json.a2, dynamicType(json.a2) == 'String')) from test_json_dynamic_aggregate_functions; + +select 'Group by'; +select dynamicType(json.a2), distinctJSONPaths(json) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctDynamicTypes(json.a2) from test_json_dynamic_aggregate_functions group by dynamicType(json.a2) order by dynamicType(json.a2); + +select 'Remote'; +select arrayJoin(distinctJSONPaths(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctJSONPathsAndTypes(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctDynamicTypes(json.a2)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); + +select 'Remote filter'; +select arrayJoin(distinctJSONPaths(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctJSONPathsAndTypes(json)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; +select arrayJoin(distinctDynamicTypes(json.a2)) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) where dynamicType(json.a2) == 'String'; + +select 'Remote if'; +select arrayJoin(distinctJSONPathsIf(json, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctJSONPathsAndTypesIf(json, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); +select arrayJoin(distinctDynamicTypesIf(json.a2, dynamicType(json.a2) == 'String')) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions); + +select 'Remote group by'; +select dynamicType(json.a2), distinctJSONPaths(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select dynamicType(json.a2), distinctDynamicTypes(json.a2) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); + +drop table test_json_dynamic_aggregate_functions; From 85a813bbedeec62fdf4704041a62092bbf6a9789 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 17:55:57 +0000 Subject: [PATCH 030/154] Better example --- docs/en/sql-reference/data-types/newjson.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index 16cc324a168..99156b11a0b 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -571,7 +571,7 @@ SELECT arrayJoin(distinctJSONPaths(json)) FROM s3('s3://clickhouse-public-datase ``` ```sql -SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) +SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-public-datasets/gharchive/original/2020-01-01-*.json.gz', JSONAsObject) SETTINGS date_time_input_format='best_effort' ``` @@ -583,7 +583,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi │ ('actor.id',['Int64']) │ │ ('actor.login',['String']) │ │ ('actor.url',['String']) │ -│ ('created_at',['String']) │ +│ ('created_at',['DateTime']) │ │ ('id',['String']) │ │ ('org.avatar_url',['String']) │ │ ('org.gravatar_id',['String']) │ @@ -598,7 +598,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi │ ('payload.comment.author_association',['String']) │ │ ('payload.comment.body',['String']) │ │ ('payload.comment.commit_id',['String']) │ -│ ('payload.comment.created_at',['String']) │ +│ ('payload.comment.created_at',['DateTime']) │ │ ('payload.comment.diff_hunk',['String']) │ │ ('payload.comment.html_url',['String']) │ │ ('payload.comment.id',['Int64']) │ @@ -614,7 +614,7 @@ SELECT arrayJoin(distinctJSONPathsAndTypes(json)) FROM s3('s3://clickhouse-publi ... │ ('payload.release.node_id',['String']) │ │ ('payload.release.prerelease',['Bool']) │ -│ ('payload.release.published_at',['String']) │ +│ ('payload.release.published_at',['DateTime']) │ │ ('payload.release.tag_name',['String']) │ │ ('payload.release.tarball_url',['String']) │ │ ('payload.release.target_commitish',['String']) │ From f21e982f03488634cd09ec7ab570fec07d78a0e1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 18:01:35 +0000 Subject: [PATCH 031/154] Fix style, update tests --- .../AggregateFunctionDistinctJSONPaths.cpp | 18 ++++++++++++------ ...03227_distinct_dynamic_types_json_paths.sql | 10 ++++++++++ .../aspell-ignore/en/aspell-dict.txt | 5 +++++ 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index a3ede0750f2..58ce7e27ab0 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -19,6 +19,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int ILLEGAL_TYPE_OF_ARGUMENT; +} + struct AggregateFunctionDistinctJSONPathsData { static constexpr auto name = "distinctJSONPaths"; @@ -44,7 +50,7 @@ struct AggregateFunctionDistinctJSONPathsData for (size_t i = start; i != end; ++i) data.insert(shared_data_paths->getDataAt(i).toString()); } - + void addWholeColumn(const ColumnObject & column, const std::unordered_map &) { for (const auto & [path, _] : column.getTypedPaths()) @@ -98,7 +104,7 @@ struct AggregateFunctionDistinctJSONPathsData string_column.insertData(path.data(), path.size()); array_column.getOffsets().push_back(string_column.size()); } - + static DataTypePtr getResultType() { return std::make_shared(std::make_shared()); @@ -110,7 +116,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData static constexpr auto name = "distinctJSONPathsAndTypes"; std::unordered_map> data; - + void add(const ColumnObject & column, size_t row_num, const std::unordered_map & typed_paths_type_names) { for (const auto & [path, _] : column.getTypedPaths()) @@ -137,7 +143,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData data[path].insert(type->getName()); } } - + void addWholeColumn(const ColumnObject & column, const std::unordered_map & typed_paths_type_names) { for (const auto & [path, _] : column.getTypedPaths()) @@ -229,7 +235,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData array_column.getOffsets().push_back(key_column.size()); } - + static DataTypePtr getResultType() { return std::make_shared(std::make_shared(), std::make_shared(std::make_shared())); @@ -302,7 +308,7 @@ public: { this->data(place).insertResultInto(to); } - + private: std::unordered_map typed_paths_type_names; }; diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index 4f33cc10d46..f7707b536e6 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -47,4 +47,14 @@ select dynamicType(json.a2), distinctJSONPaths(json) from remote('127.0.0.{1,2,3 select dynamicType(json.a2), distinctJSONPathsAndTypes(json) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); select dynamicType(json.a2), distinctDynamicTypes(json.a2) from remote('127.0.0.{1,2,3}', currentDatabase(), test_json_dynamic_aggregate_functions) group by dynamicType(json.a2) order by dynamicType(json.a2); +select distinctJSONPaths() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPaths(json, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPaths(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select distinctJSONPathsAndTypes() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPathsAndTypes(json, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctJSONPathsAndTypes(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} +select distinctDynamicTypes() from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctDynamicTypes(json.a2, 42) from test_json_dynamic_aggregate_functions; -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} +select distinctDynamicTypes(42) from test_json_dynamic_aggregate_functions; -- {serverError ILLEGAL_TYPE_OF_ARGUMENT} + drop table test_json_dynamic_aggregate_functions; diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index fd836d93143..a5dd1ac5f9d 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -1551,6 +1551,11 @@ disjunctions displayName displaySecretsInShowAndSelect distro +distinctdynamictypes +distinctDynamicTypes +distinctjsonpaths +distinctJSONPaths +distinctJSONPathsAndTypes divideDecimal dmesg doesnt From b29d5242be83294af8429d30fd560a83de3c08b4 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 16 Aug 2024 22:10:11 +0000 Subject: [PATCH 032/154] Fix tidy build --- .../AggregateFunctionDistinctDynamicTypes.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 6c899d6cda8..649d64a3904 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -87,7 +87,7 @@ public: if (dynamic_column.isNullAt(row_num)) return; - this->data(place).add(dynamic_column.getTypeNameAt(row_num)); + data(place).add(dynamic_column.getTypeNameAt(row_num)); } void ALWAYS_INLINE addBatchSinglePlace( @@ -100,7 +100,7 @@ public: /// In this case we can avoid iterating over all rows because we can get all types /// in Dynamic column in a more efficient way. else - assert_cast(*columns[0]).getAllTypeNames(this->data(place).data); + assert_cast(*columns[0]).getAllTypeNames(data(place).data); } void addManyDefaults( @@ -114,22 +114,22 @@ public: void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override { - this->data(place).merge(this->data(rhs)); + data(place).merge(data(rhs)); } void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional /* version */) const override { - this->data(place).serialize(buf); + data(place).serialize(buf); } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional /* version */, Arena *) const override { - this->data(place).deserialize(buf); + data(place).deserialize(buf); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { - this->data(place).insertResultInto(to); + data(place).insertResultInto(to); } }; From 41f6e06cbe07f6bd1ce3bdb48154f1f2745b3cc2 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 17 Aug 2024 13:08:30 +0200 Subject: [PATCH 033/154] Update 03227_distinct_dynamic_types_json_paths.sql --- .../0_stateless/03227_distinct_dynamic_types_json_paths.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index f7707b536e6..c9c86277bce 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -1,3 +1,5 @@ +-- Tags: long + set allow_experimental_dynamic_type = 1; set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; From 9d183b890529d686a673d5f6ecd2d057cf534478 Mon Sep 17 00:00:00 2001 From: avogar Date: Sun, 18 Aug 2024 20:35:30 +0000 Subject: [PATCH 034/154] Add check for loo large array size --- .../AggregateFunctionDistinctDynamicTypes.cpp | 6 ++++++ .../AggregateFunctionDistinctJSONPaths.cpp | 13 +++++++++++++ src/Core/Settings.h | 1 - 3 files changed, 19 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 649d64a3904..17e32b20a99 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -19,10 +19,13 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; } struct AggregateFunctionDistinctDynamicTypesData { + constexpr static size_t MAX_ARRAY_SIZE = 0xFFFFFF; + std::unordered_set data; void add(const String & type) @@ -46,6 +49,9 @@ struct AggregateFunctionDistinctDynamicTypesData { size_t size; readVarUInt(size, buf); + if (size > MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", MAX_ARRAY_SIZE, size); + data.reserve(size); String type; for (size_t i = 0; i != size; ++i) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 58ce7e27ab0..6100bd57515 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -23,8 +23,12 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int TOO_LARGE_ARRAY_SIZE; } +constexpr static size_t DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE = 0xFFFFFF; + + struct AggregateFunctionDistinctJSONPathsData { static constexpr auto name = "distinctJSONPaths"; @@ -85,6 +89,9 @@ struct AggregateFunctionDistinctJSONPathsData { size_t size; readVarUInt(size, buf); + if (size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, size); + String path; for (size_t i = 0; i != size; ++i) { @@ -192,12 +199,18 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData { size_t paths_size, types_size; readVarUInt(paths_size, buf); + if (paths_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); + data.reserve(paths_size); String path, type; for (size_t i = 0; i != paths_size; ++i) { readStringBinary(path, buf); readVarUInt(types_size, buf); + if (types_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); + data[path].reserve(types_size); for (size_t j = 0; j != types_size; ++j) { diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 0d84ad9022a..dfcff052740 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,6 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From 228cdfde63100267ce1b8c80ddfeb689979487ba Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 15 Aug 2024 08:58:36 +0000 Subject: [PATCH 035/154] more tests --- .../test_filesystem_layout/test.py | 4 ++++ .../test_with_table_engine_grant.py | 2 ++ .../test_parallel_replicas_failover/test.py | 19 ++++++++++--------- .../test.py | 4 ++-- 4 files changed, 18 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_filesystem_layout/test.py b/tests/integration/test_filesystem_layout/test.py index 4e719aa0fe9..31d6c830a2f 100644 --- a/tests/integration/test_filesystem_layout/test.py +++ b/tests/integration/test_filesystem_layout/test.py @@ -79,3 +79,7 @@ def test_file_path_escaping(started_cluster): "test -f /var/lib/clickhouse/shadow/2/store/123/12345678-1000-4000-8000-000000000001/1_1_1_0/%7EId.bin", ] ) + node.query("DROP TABLE test.`T.a_b,l-e!` SYNC") + node.query("DROP TABLE `test 2`.`T.a_b,l-e!` SYNC") + node.query("DROP DATABASE test") + node.query("DROP DATABASE `test 2`") diff --git a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index 25ca7913e4e..5fc8f67b75b 100644 --- a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -359,6 +359,8 @@ def test_implicit_create_view_grant(): instance.query("GRANT CREATE VIEW ON test.* TO B", user="A") instance.query("CREATE VIEW test.view_2 AS SELECT 1", user="B") assert instance.query("SELECT * FROM test.view_2") == "1\n" + instance.query("DROP USER A") + instance.query("DROP VIEW test.view_2") def test_implicit_create_temporary_table_grant(): diff --git a/tests/integration/test_parallel_replicas_failover/test.py b/tests/integration/test_parallel_replicas_failover/test.py index bf25136bff7..2da26ee03c9 100644 --- a/tests/integration/test_parallel_replicas_failover/test.py +++ b/tests/integration/test_parallel_replicas_failover/test.py @@ -1,5 +1,5 @@ import pytest - +import uuid from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -25,19 +25,15 @@ def start_cluster(): def create_tables(cluster, table_name, skip_last_replica): - node1.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node2.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node3.query(f"DROP TABLE IF EXISTS {table_name} SYNC") - node1.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r1') ORDER BY (key)" ) node2.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r2') ORDER BY (key)" ) if not skip_last_replica: node3.query( - f"CREATE TABLE IF NOT EXISTS {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" + f"CREATE TABLE {table_name} (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard1/{table_name}', 'r3') ORDER BY (key)" ) # populate data @@ -67,7 +63,7 @@ def test_skip_replicas_without_table(start_cluster): for i in range(4): expected_result += f"{i}\t1000\n" - log_comment = "5230b069-9574-407d-9b80-891b5a175f41" + log_comment = uuid.uuid4() assert ( node1.query( f"SELECT key, count() FROM {table_name} GROUP BY key ORDER BY key", @@ -88,6 +84,8 @@ def test_skip_replicas_without_table(start_cluster): ) == "1\t1\n" ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") def test_skip_unresponsive_replicas(start_cluster): @@ -112,3 +110,6 @@ def test_skip_unresponsive_replicas(start_cluster): ) == expected_result ) + node1.query(f"DROP TABLE {table_name} SYNC") + node2.query(f"DROP TABLE {table_name} SYNC") + node3.query(f"DROP TABLE {table_name} SYNC") diff --git a/tests/integration/test_parallel_replicas_invisible_parts/test.py b/tests/integration/test_parallel_replicas_invisible_parts/test.py index cab3fb46fe9..7093e3b3292 100644 --- a/tests/integration/test_parallel_replicas_invisible_parts/test.py +++ b/tests/integration/test_parallel_replicas_invisible_parts/test.py @@ -35,11 +35,10 @@ def start_cluster(): def _create_tables(table_name, table_size, index_granularity): - nodes[0].query(f"DROP TABLE IF EXISTS {table_name} ON CLUSTER {cluster_name}") nodes[0].query( f""" - CREATE TABLE IF NOT EXISTS {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) + CREATE TABLE {table_name} ON CLUSTER '{cluster_name}' (key Int64, value String) Engine=ReplicatedMergeTree('/test_parallel_replicas/shard/{table_name}/', '{{replica}}') ORDER BY (key) SETTINGS index_granularity = {index_granularity}, max_bytes_to_merge_at_max_space_in_pool = 0, max_bytes_to_merge_at_max_space_in_pool = 1 @@ -128,3 +127,4 @@ def test_reading_with_invisible_parts( ) == f"{expected}\n" ) + nodes[0].query(f"DROP TABLE {table_name} ON CLUSTER {cluster_name} SYNC") From 90ad110a7dddbd1c9430b34b0016dc22eaa3d646 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 19 Aug 2024 13:58:44 +0000 Subject: [PATCH 036/154] more --- .../test.py | 28 +++++++++++-------- 1 file changed, 17 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 5e04c9e4d12..fb7aab366be 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -1,5 +1,6 @@ import pytest +import uuid import time import psycopg2 import os.path as p @@ -59,8 +60,6 @@ instance2 = cluster.add_instance( pg_manager = PostgresManager() pg_manager2 = PostgresManager() pg_manager_instance2 = PostgresManager() -pg_manager3 = PostgresManager() - @pytest.fixture(scope="module") def started_cluster(): @@ -82,12 +81,6 @@ def started_cluster(): pg_manager2.init( instance2, cluster.postgres_ip, cluster.postgres_port, "postgres_database2" ) - pg_manager3.init( - instance, - cluster.postgres_ip, - cluster.postgres_port, - default_database="postgres-postgres", - ) yield cluster @@ -924,16 +917,27 @@ def test_failed_load_from_snapshot(started_cluster): def test_symbols_in_publication_name(started_cluster): - table = "test_symbols_in_publication_name" + id = uuid.uuid4() + db = f'test_{id}' + table = f"test_symbols_in_publication_name" + + pg_manager3 = PostgresManager() + pg_manager3.init( + instance, + cluster.postgres_ip, + cluster.postgres_port, + default_database=db, + ) pg_manager3.create_postgres_table(table) instance.query( - f"INSERT INTO `{pg_manager3.get_default_database()}`.`{table}` SELECT number, number from numbers(0, 50)" + f"INSERT INTO `{db}`.`{table}` SELECT number, number from numbers(0, 50)" ) pg_manager3.create_materialized_db( ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, + materialized_database=db, settings=[ f"materialized_postgresql_tables_list = '{table}'", "materialized_postgresql_backoff_min_ms = 100", @@ -941,8 +945,10 @@ def test_symbols_in_publication_name(started_cluster): ], ) check_tables_are_synchronized( - instance, table, postgres_database=pg_manager3.get_default_database() + instance, table, materialized_database=db, postgres_database=db ) + pg_manager3.drop_materialized_db(db) + pg_manager3.execute(f'drop table "{table}"') def test_generated_columns(started_cluster): From 0793585acde8f469bf534f0639850a91efc254f1 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 16:59:30 +0200 Subject: [PATCH 037/154] Fix bad conflict resolution --- src/Core/Settings.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index dfcff052740..215d333f6c4 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,6 +616,7 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ + M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From 64bce7afa115f13ff72e2b4f0f6e1b4eacb881e7 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Mon, 19 Aug 2024 17:22:17 +0200 Subject: [PATCH 038/154] Fix spaces --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 215d333f6c4..0d84ad9022a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -616,7 +616,7 @@ class IColumn; M(Bool, throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert, true, "Throw exception on INSERT query when the setting `deduplicate_blocks_in_dependent_materialized_views` is enabled along with `async_insert`. It guarantees correctness, because these features can't work together.", 0) \ M(Bool, materialized_views_ignore_errors, false, "Allows to ignore errors for MATERIALIZED VIEW, and deliver original block to the table regardless of MVs", 0) \ M(Bool, ignore_materialized_views_with_dropped_target_table, false, "Ignore MVs with dropped target table during pushing to views", 0) \ - M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ + M(Bool, allow_materialized_view_with_bad_select, true, "Allow CREATE MATERIALIZED VIEW with SELECT query that references nonexistent tables or columns. It must still be syntactically valid. Doesn't apply to refreshable MVs. Doesn't apply if the MV schema needs to be inferred from the SELECT query (i.e. if the CREATE has no column list and no TO table). Can be used for creating MV before its source table.", 0) \ M(Bool, use_compact_format_in_distributed_parts_names, true, "Changes format of directories names for distributed table insert parts.", 0) \ M(Bool, validate_polygons, true, "Throw exception if polygon is invalid in function pointInPolygon (e.g. self-tangent, self-intersecting). If the setting is false, the function will accept invalid polygons but may silently return wrong result.", 0) \ M(UInt64, max_parser_depth, DBMS_DEFAULT_MAX_PARSER_DEPTH, "Maximum parser depth (recursion depth of recursive descend parser).", 0) \ From a2b08e93e43c357f8adbdd604a54ccb75056e4ec Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 19 Aug 2024 20:16:50 +0000 Subject: [PATCH 039/154] Change test --- ...istinct_dynamic_types_json_paths.reference | 70 +++++++++---------- ...3227_distinct_dynamic_types_json_paths.sql | 2 +- 2 files changed, 36 insertions(+), 36 deletions(-) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference index d6538a1c4eb..891982fdc81 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.reference @@ -11,25 +11,25 @@ a6 a7 a8 a9 -('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a0',['Array(Nullable(Int64))','Bool','Int64','String']) ('a1',['String']) -('a10',['Array(Nullable(Int64))','Date','Int64','String']) -('a11',['Array(Nullable(Int64))','Date','Int64','String']) -('a12',['Array(Nullable(Int64))','Date','Int64','String']) -('a2',['Array(Nullable(Int64))','Date','Int64','String']) -('a3',['Array(Nullable(Int64))','Date','Int64','String']) -('a4',['Array(Nullable(Int64))','Date','Int64','String']) -('a5',['Array(Nullable(Int64))','Date','Int64','String']) -('a6',['Array(Nullable(Int64))','Date','Int64','String']) -('a7',['Array(Nullable(Int64))','Date','Int64','String']) -('a8',['Array(Nullable(Int64))','Date','Int64','String']) -('a9',['Array(Nullable(Int64))','Date','Int64','String']) +('a10',['Array(Nullable(Int64))','Bool','Int64','String']) +('a11',['Array(Nullable(Int64))','Bool','Int64','String']) +('a12',['Array(Nullable(Int64))','Bool','Int64','String']) +('a2',['Array(Nullable(Int64))','Bool','Int64','String']) +('a3',['Array(Nullable(Int64))','Bool','Int64','String']) +('a4',['Array(Nullable(Int64))','Bool','Int64','String']) +('a5',['Array(Nullable(Int64))','Bool','Int64','String']) +('a6',['Array(Nullable(Int64))','Bool','Int64','String']) +('a7',['Array(Nullable(Int64))','Bool','Int64','String']) +('a8',['Array(Nullable(Int64))','Bool','Int64','String']) +('a9',['Array(Nullable(Int64))','Bool','Int64','String']) Array(Nullable(Int64)) -Date +Bool Int64 String Array(Nullable(Int64)) -Date +Bool Int64 String Filter @@ -46,17 +46,17 @@ a2 String Group by Array(Nullable(Int64)) ['a1','a2'] -Date ['a1','a2'] +Bool ['a1','a2'] Int64 ['a1','a2'] None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] String ['a1','a2'] Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} -Date {'a1':['String'],'a2':['Date']} +Bool {'a1':['String'],'a2':['Bool']} Int64 {'a1':['String'],'a2':['Int64']} -None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +None {'a0':['Array(Nullable(Int64))','Bool','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Bool','Int64','String'],'a11':['Array(Nullable(Int64))','Bool','Int64','String'],'a12':['Array(Nullable(Int64))','Bool','Int64','String'],'a3':['Array(Nullable(Int64))','Bool','Int64','String'],'a4':['Array(Nullable(Int64))','Bool','Int64','String'],'a5':['Array(Nullable(Int64))','Bool','Int64','String'],'a6':['Array(Nullable(Int64))','Bool','Int64','String'],'a7':['Array(Nullable(Int64))','Bool','Int64','String'],'a8':['Array(Nullable(Int64))','Bool','Int64','String'],'a9':['Array(Nullable(Int64))','Bool','Int64','String']} String {'a1':['String'],'a2':['String']} Array(Nullable(Int64)) ['Array(Nullable(Int64))'] -Date ['Date'] +Bool ['Bool'] Int64 ['Int64'] None [] String ['String'] @@ -74,21 +74,21 @@ a6 a7 a8 a9 -('a0',['Array(Nullable(Int64))','Date','Int64','String']) +('a0',['Array(Nullable(Int64))','Bool','Int64','String']) ('a1',['String']) -('a10',['Array(Nullable(Int64))','Date','Int64','String']) -('a11',['Array(Nullable(Int64))','Date','Int64','String']) -('a12',['Array(Nullable(Int64))','Date','Int64','String']) -('a2',['Array(Nullable(Int64))','Date','Int64','String']) -('a3',['Array(Nullable(Int64))','Date','Int64','String']) -('a4',['Array(Nullable(Int64))','Date','Int64','String']) -('a5',['Array(Nullable(Int64))','Date','Int64','String']) -('a6',['Array(Nullable(Int64))','Date','Int64','String']) -('a7',['Array(Nullable(Int64))','Date','Int64','String']) -('a8',['Array(Nullable(Int64))','Date','Int64','String']) -('a9',['Array(Nullable(Int64))','Date','Int64','String']) +('a10',['Array(Nullable(Int64))','Bool','Int64','String']) +('a11',['Array(Nullable(Int64))','Bool','Int64','String']) +('a12',['Array(Nullable(Int64))','Bool','Int64','String']) +('a2',['Array(Nullable(Int64))','Bool','Int64','String']) +('a3',['Array(Nullable(Int64))','Bool','Int64','String']) +('a4',['Array(Nullable(Int64))','Bool','Int64','String']) +('a5',['Array(Nullable(Int64))','Bool','Int64','String']) +('a6',['Array(Nullable(Int64))','Bool','Int64','String']) +('a7',['Array(Nullable(Int64))','Bool','Int64','String']) +('a8',['Array(Nullable(Int64))','Bool','Int64','String']) +('a9',['Array(Nullable(Int64))','Bool','Int64','String']) Array(Nullable(Int64)) -Date +Bool Int64 String Remote filter @@ -105,17 +105,17 @@ a2 String Remote group by Array(Nullable(Int64)) ['a1','a2'] -Date ['a1','a2'] +Bool ['a1','a2'] Int64 ['a1','a2'] None ['a0','a1','a10','a11','a12','a3','a4','a5','a6','a7','a8','a9'] String ['a1','a2'] Array(Nullable(Int64)) {'a1':['String'],'a2':['Array(Nullable(Int64))']} -Date {'a1':['String'],'a2':['Date']} +Bool {'a1':['String'],'a2':['Bool']} Int64 {'a1':['String'],'a2':['Int64']} -None {'a0':['Array(Nullable(Int64))','Date','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Date','Int64','String'],'a11':['Array(Nullable(Int64))','Date','Int64','String'],'a12':['Array(Nullable(Int64))','Date','Int64','String'],'a3':['Array(Nullable(Int64))','Date','Int64','String'],'a4':['Array(Nullable(Int64))','Date','Int64','String'],'a5':['Array(Nullable(Int64))','Date','Int64','String'],'a6':['Array(Nullable(Int64))','Date','Int64','String'],'a7':['Array(Nullable(Int64))','Date','Int64','String'],'a8':['Array(Nullable(Int64))','Date','Int64','String'],'a9':['Array(Nullable(Int64))','Date','Int64','String']} +None {'a0':['Array(Nullable(Int64))','Bool','Int64','String'],'a1':['String'],'a10':['Array(Nullable(Int64))','Bool','Int64','String'],'a11':['Array(Nullable(Int64))','Bool','Int64','String'],'a12':['Array(Nullable(Int64))','Bool','Int64','String'],'a3':['Array(Nullable(Int64))','Bool','Int64','String'],'a4':['Array(Nullable(Int64))','Bool','Int64','String'],'a5':['Array(Nullable(Int64))','Bool','Int64','String'],'a6':['Array(Nullable(Int64))','Bool','Int64','String'],'a7':['Array(Nullable(Int64))','Bool','Int64','String'],'a8':['Array(Nullable(Int64))','Bool','Int64','String'],'a9':['Array(Nullable(Int64))','Bool','Int64','String']} String {'a1':['String'],'a2':['String']} Array(Nullable(Int64)) ['Array(Nullable(Int64))'] -Date ['Date'] +Bool ['Bool'] Int64 ['Int64'] None [] String ['String'] diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index c9c86277bce..6930f5a3d44 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -7,7 +7,7 @@ set use_variant_as_common_type = 1; drop table if exists test_json_dynamic_aggregate_functions; create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; -insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toDate(number)))) from numbers(200000); +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(200000); select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; From 1dfe559db45a5e03438e5d857e3a5bf5806975be Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Tue, 20 Aug 2024 19:52:33 +0800 Subject: [PATCH 040/154] emptry commit From 1f2f3c69b9bcc4d7f19962cbc2562c6b3ce404dd Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 26 Jan 2024 23:26:04 +0100 Subject: [PATCH 041/154] New metrics for ThreadPool - Introduced performance metrics for better monitoring and troubleshooting of ThreadPool. --- src/Common/ProfileEvents.cpp | 12 +++++++ src/Common/ThreadPool.cpp | 69 ++++++++++++++++++++++++++++++++++-- 2 files changed, 79 insertions(+), 2 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index d43d9fdcea8..6bcdf9d8e91 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -86,6 +86,18 @@ M(NetworkReceiveBytes, "Total number of bytes received from network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ M(NetworkSendBytes, "Total number of bytes send to network. Only ClickHouse-related network interaction is included, not by 3rd party libraries.") \ \ + M(GlobalThreadPoolExpansions, "Counts the total number of times new threads have been added to the global thread pool. This metric indicates the frequency of expansions in the global thread pool to accommodate increased processing demands.") \ + M(GlobalThreadPoolShrinks, "Counts the total number of times the global thread pool has shrunk by removing threads. This occurs when the number of idle threads exceeds max_thread_pool_free_size, indicating adjustments in the global thread pool size in response to decreased thread utilization.") \ + M(GlobalThreadPoolThreadCreationMicroseconds, "Total time spent waiting for new threads to start.") \ + M(GlobalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the global thread pool.") \ + M(GlobalThreadPoolJobs, "Counts the number of jobs that have been pushed to the global thread pool.") \ + M(LocalThreadPoolExpansions, "Counts the total number of times threads have been borrowed from the global thread pool to expand local thread pools.") \ + M(LocalThreadPoolShrinks, "Counts the total number of times threads have been returned to the global thread pool from local thread pools.") \ + M(LocalThreadPoolThreadCreationMicroseconds, "Total time local thread pools have spent waiting to borrow a thread from the global pool.") \ + M(LocalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the local thread pools.") \ + M(LocalThreadPoolJobs, "Counts the number of jobs that have been pushed to the local thread pools.") \ + M(LocalThreadPoolBusyMicroseconds, "Total time threads have spent executing the actual work.") \ + \ M(DiskS3GetRequestThrottlerCount, "Number of DiskS3 GET and SELECT requests passed through throttler.") \ M(DiskS3GetRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform DiskS3 GET and SELECT request throttling.") \ M(DiskS3PutRequestThrottlerCount, "Number of DiskS3 PUT, COPY, POST and LIST requests passed through throttler.") \ diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index c8f1ae99969..d93859e1abc 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -27,6 +28,22 @@ namespace CurrentMetrics extern const Metric GlobalThreadScheduled; } +namespace ProfileEvents +{ + extern const Event GlobalThreadPoolExpansions; + extern const Event GlobalThreadPoolShrinks; + extern const Event GlobalThreadPoolThreadCreationMicroseconds; + extern const Event GlobalThreadPoolLockWaitMicroseconds; + extern const Event GlobalThreadPoolJobs; + + extern const Event LocalThreadPoolExpansions; + extern const Event LocalThreadPoolShrinks; + extern const Event LocalThreadPoolThreadCreationMicroseconds; + extern const Event LocalThreadPoolLockWaitMicroseconds; + extern const Event LocalThreadPoolJobs; + extern const Event LocalThreadPoolBusyMicroseconds; +} + class JobWithPriority { public: @@ -180,14 +197,18 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: }; { + Stopwatch watch; std::unique_lock lock(mutex); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolLockWaitMicroseconds : ProfileEvents::LocalThreadPoolLockWaitMicroseconds, + watch.elapsedMicroseconds()); if (CannotAllocateThreadFaultInjector::injectFault()) return on_error("fault injected"); auto pred = [this] { return !queue_size || scheduled_jobs < queue_size || shutdown; }; - if (wait_microseconds) /// Check for optional. Condition is true if the optional is set and the value is zero. + if (wait_microseconds) /// Check for optional. Condition is true if the optional is set. Even if the value is zero. { if (!job_finished.wait_for(lock, std::chrono::microseconds(*wait_microseconds), pred)) return on_error(fmt::format("no free thread (timeout={})", *wait_microseconds)); @@ -216,7 +237,13 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: try { + Stopwatch watch2; threads.front() = Thread([this, it = threads.begin()] { worker(it); }); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolThreadCreationMicroseconds : ProfileEvents::LocalThreadPoolThreadCreationMicroseconds, + watch2.elapsedMicroseconds()); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolExpansions : ProfileEvents::LocalThreadPoolExpansions); } catch (...) { @@ -239,6 +266,8 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: /// Wake up a free thread to run the new job. new_job_or_shutdown.notify_one(); + ProfileEvents::increment( std::is_same_v ? ProfileEvents::GlobalThreadPoolJobs : ProfileEvents::LocalThreadPoolJobs); + return static_cast(true); } @@ -262,7 +291,14 @@ void ThreadPoolImpl::startNewThreadsNoLock() try { + Stopwatch watch; threads.front() = Thread([this, it = threads.begin()] { worker(it); }); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolThreadCreationMicroseconds : ProfileEvents::LocalThreadPoolThreadCreationMicroseconds, + watch.elapsedMicroseconds()); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolExpansions : ProfileEvents::LocalThreadPoolExpansions); + } catch (...) { @@ -293,7 +329,11 @@ void ThreadPoolImpl::scheduleOrThrow(Job job, Priority priority, uint64_ template void ThreadPoolImpl::wait() { + Stopwatch watch; std::unique_lock lock(mutex); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolLockWaitMicroseconds : ProfileEvents::LocalThreadPoolLockWaitMicroseconds, + watch.elapsedMicroseconds()); /// Signal here just in case. /// If threads are waiting on condition variables, but there are some jobs in the queue /// then it will prevent us from deadlock. @@ -334,7 +374,11 @@ void ThreadPoolImpl::finalize() /// Wait for all currently running jobs to finish (we don't wait for all scheduled jobs here like the function wait() does). for (auto & thread : threads) + { thread.join(); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolShrinks : ProfileEvents::LocalThreadPoolShrinks); + } threads.clear(); } @@ -391,7 +435,11 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ std::optional job_data; { + Stopwatch watch; std::unique_lock lock(mutex); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolLockWaitMicroseconds : ProfileEvents::LocalThreadPoolLockWaitMicroseconds, + watch.elapsedMicroseconds()); // Finish with previous job if any if (job_is_done) @@ -424,6 +472,8 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ { thread_it->detach(); threads.erase(thread_it); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolShrinks : ProfileEvents::LocalThreadPoolShrinks); } return; } @@ -459,7 +509,22 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ CurrentMetrics::Increment metric_active_pool_threads(metric_active_threads); - job_data->job(); + if constexpr (!std::is_same_v) + { + Stopwatch watch; + job_data->job(); + // This metric is less relevant for the global thread pool, as it would show large values (time while + // a thread was used by local pools) and increment only when local pools are destroyed. + // + // In cases where global pool threads are used directly (without a local thread pool), distinguishing + // them is difficult. + ProfileEvents::increment(ProfileEvents::LocalThreadPoolBusyMicroseconds, watch.elapsedMicroseconds()); + } + else + { + job_data->job(); + } + if (thread_trace_context.root_span.isTraceEnabled()) { From 843d8696316fb6b648200585e411d3bce1b46f98 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 9 Feb 2024 08:14:05 +0100 Subject: [PATCH 042/154] add one more metric --- src/Common/ProfileEvents.cpp | 2 ++ src/Common/ThreadPool.cpp | 15 +++++++++++++++ 2 files changed, 17 insertions(+) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 6bcdf9d8e91..c09599e7172 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -91,12 +91,14 @@ M(GlobalThreadPoolThreadCreationMicroseconds, "Total time spent waiting for new threads to start.") \ M(GlobalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the global thread pool.") \ M(GlobalThreadPoolJobs, "Counts the number of jobs that have been pushed to the global thread pool.") \ + M(GlobalThreadPoolJobWaitTimeMicroseconds, "Measures the elapsed time from when a job is scheduled in the thread pool to when it is picked up for execution by a worker thread. This metric helps identify delays in job processing, indicating the responsiveness of the thread pool to new tasks.") \ M(LocalThreadPoolExpansions, "Counts the total number of times threads have been borrowed from the global thread pool to expand local thread pools.") \ M(LocalThreadPoolShrinks, "Counts the total number of times threads have been returned to the global thread pool from local thread pools.") \ M(LocalThreadPoolThreadCreationMicroseconds, "Total time local thread pools have spent waiting to borrow a thread from the global pool.") \ M(LocalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the local thread pools.") \ M(LocalThreadPoolJobs, "Counts the number of jobs that have been pushed to the local thread pools.") \ M(LocalThreadPoolBusyMicroseconds, "Total time threads have spent executing the actual work.") \ + M(LocalThreadPoolJobWaitTimeMicroseconds, "Measures the elapsed time from when a job is scheduled in the thread pool to when it is picked up for execution by a worker thread. This metric helps identify delays in job processing, indicating the responsiveness of the thread pool to new tasks.") \ \ M(DiskS3GetRequestThrottlerCount, "Number of DiskS3 GET and SELECT requests passed through throttler.") \ M(DiskS3GetRequestThrottlerSleepMicroseconds, "Total time a query was sleeping to conform DiskS3 GET and SELECT request throttling.") \ diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index d93859e1abc..0b28b7567a7 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -35,6 +35,7 @@ namespace ProfileEvents extern const Event GlobalThreadPoolThreadCreationMicroseconds; extern const Event GlobalThreadPoolLockWaitMicroseconds; extern const Event GlobalThreadPoolJobs; + extern const Event GlobalThreadPoolJobWaitTimeMicroseconds; extern const Event LocalThreadPoolExpansions; extern const Event LocalThreadPoolShrinks; @@ -42,6 +43,8 @@ namespace ProfileEvents extern const Event LocalThreadPoolLockWaitMicroseconds; extern const Event LocalThreadPoolJobs; extern const Event LocalThreadPoolBusyMicroseconds; + extern const Event LocalThreadPoolJobWaitTimeMicroseconds; + } class JobWithPriority @@ -57,6 +60,7 @@ public: /// Call stacks of all jobs' schedulings leading to this one std::vector frame_pointers; bool enable_job_stack_trace = false; + Stopwatch job_create_time; JobWithPriority( Job job_, Priority priority_, CurrentMetrics::Metric metric, @@ -76,6 +80,13 @@ public: { return priority > rhs.priority; // Reversed for `priority_queue` max-heap to yield minimum value (i.e. highest priority) first } + + UInt64 elapsedMicroseconds() const + { + return job_create_time.elapsedMicroseconds(); + } + + }; static constexpr auto DEFAULT_THREAD_NAME = "ThreadPool"; @@ -483,6 +494,10 @@ void ThreadPoolImpl::worker(typename std::list::iterator thread_ job_data = std::move(const_cast(jobs.top())); jobs.pop(); + ProfileEvents::increment( + std::is_same_v ? ProfileEvents::GlobalThreadPoolJobWaitTimeMicroseconds : ProfileEvents::LocalThreadPoolJobWaitTimeMicroseconds, + job_data->elapsedMicroseconds()); + /// We don't run jobs after `shutdown` is set, but we have to properly dequeue all jobs and finish them. if (shutdown) { From 06b3ea2a30c8ea58be7603228b05e56a40853415 Mon Sep 17 00:00:00 2001 From: iceFireser <18734827554@163.com> Date: Wed, 21 Aug 2024 15:52:44 +0800 Subject: [PATCH 043/154] emptry commit From d5aec83ce79127fc68f08c0e545d5d063d79f7cc Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 20 Aug 2024 22:35:32 +0200 Subject: [PATCH 044/154] Mark LocalThread metrics obsolete due to https://github.com/ClickHouse/ClickHouse/pull/47880 --- src/Common/CurrentMetrics.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index 67890568941..204e09e92b2 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -75,9 +75,9 @@ M(GlobalThread, "Number of threads in global thread pool.") \ M(GlobalThreadActive, "Number of threads in global thread pool running a task.") \ M(GlobalThreadScheduled, "Number of queued or active jobs in global thread pool.") \ - M(LocalThread, "Number of threads in local thread pools. The threads in local thread pools are taken from the global thread pool.") \ - M(LocalThreadActive, "Number of threads in local thread pools running a task.") \ - M(LocalThreadScheduled, "Number of queued or active jobs in local thread pools.") \ + M(LocalThread, "Obsolete. Number of threads in local thread pools. The threads in local thread pools are taken from the global thread pool.") \ + M(LocalThreadActive, "Obsolete. Number of threads in local thread pools running a task.") \ + M(LocalThreadScheduled, "Obsolete. Number of queued or active jobs in local thread pools.") \ M(MergeTreeDataSelectExecutorThreads, "Number of threads in the MergeTreeDataSelectExecutor thread pool.") \ M(MergeTreeDataSelectExecutorThreadsActive, "Number of threads in the MergeTreeDataSelectExecutor thread pool running a task.") \ M(MergeTreeDataSelectExecutorThreadsScheduled, "Number of queued or active jobs in the MergeTreeDataSelectExecutor thread pool.") \ From a66bd99d2fbc21ac00c979f2b9b301b7c2fd9d44 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Tue, 20 Aug 2024 21:39:36 +0200 Subject: [PATCH 045/154] make jobs queue in the ThreadPool stable (i.e. FIFO for the same priority), otherwise some jobs can stay in queue untaken for a long time --- src/Common/ThreadPool.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index a31d793264e..fd9149bda04 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -131,7 +131,7 @@ private: bool threads_remove_themselves = true; const bool shutdown_on_exception = true; - boost::heap::priority_queue jobs; + boost::heap::priority_queue> jobs; std::list threads; std::exception_ptr first_exception; std::stack on_destroy_callbacks; From 64d4ef002df41697247c8ed59586e02d7a4bfaf3 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 21 Aug 2024 18:20:05 +0200 Subject: [PATCH 046/154] fix style --- src/Common/ProfileEvents.cpp | 4 ++-- src/Common/ThreadPool.cpp | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c09599e7172..044f787aee9 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -89,11 +89,11 @@ M(GlobalThreadPoolExpansions, "Counts the total number of times new threads have been added to the global thread pool. This metric indicates the frequency of expansions in the global thread pool to accommodate increased processing demands.") \ M(GlobalThreadPoolShrinks, "Counts the total number of times the global thread pool has shrunk by removing threads. This occurs when the number of idle threads exceeds max_thread_pool_free_size, indicating adjustments in the global thread pool size in response to decreased thread utilization.") \ M(GlobalThreadPoolThreadCreationMicroseconds, "Total time spent waiting for new threads to start.") \ - M(GlobalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the global thread pool.") \ + M(GlobalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the global thread pool.") \ M(GlobalThreadPoolJobs, "Counts the number of jobs that have been pushed to the global thread pool.") \ M(GlobalThreadPoolJobWaitTimeMicroseconds, "Measures the elapsed time from when a job is scheduled in the thread pool to when it is picked up for execution by a worker thread. This metric helps identify delays in job processing, indicating the responsiveness of the thread pool to new tasks.") \ M(LocalThreadPoolExpansions, "Counts the total number of times threads have been borrowed from the global thread pool to expand local thread pools.") \ - M(LocalThreadPoolShrinks, "Counts the total number of times threads have been returned to the global thread pool from local thread pools.") \ + M(LocalThreadPoolShrinks, "Counts the total number of times threads have been returned to the global thread pool from local thread pools.") \ M(LocalThreadPoolThreadCreationMicroseconds, "Total time local thread pools have spent waiting to borrow a thread from the global pool.") \ M(LocalThreadPoolLockWaitMicroseconds, "Total time threads have spent waiting for locks in the local thread pools.") \ M(LocalThreadPoolJobs, "Counts the number of jobs that have been pushed to the local thread pools.") \ diff --git a/src/Common/ThreadPool.cpp b/src/Common/ThreadPool.cpp index 0b28b7567a7..8685533e2d1 100644 --- a/src/Common/ThreadPool.cpp +++ b/src/Common/ThreadPool.cpp @@ -277,7 +277,7 @@ ReturnType ThreadPoolImpl::scheduleImpl(Job job, Priority priority, std: /// Wake up a free thread to run the new job. new_job_or_shutdown.notify_one(); - ProfileEvents::increment( std::is_same_v ? ProfileEvents::GlobalThreadPoolJobs : ProfileEvents::LocalThreadPoolJobs); + ProfileEvents::increment(std::is_same_v ? ProfileEvents::GlobalThreadPoolJobs : ProfileEvents::LocalThreadPoolJobs); return static_cast(true); } From b4a0a9b00cafc86399c62c244b8ada6e0f8d2db4 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 22 Aug 2024 16:58:19 +0000 Subject: [PATCH 047/154] rework test_async_insert_adaptive_busy_timeout --- .../test.py | 55 ++++++++++--------- .../test_with_table_engine_grant.py | 1 + 2 files changed, 31 insertions(+), 25 deletions(-) diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 0ea076b1468..5599786026f 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -104,7 +104,7 @@ def test_with_merge_tree(): _insert_queries_sequentially( table_name, _query_settings, - iterations=100, + iterations=10, max_values_size=1000, array_size_range=[10, 50], ) @@ -125,7 +125,7 @@ def test_with_merge_tree_multithread(): table_name, _query_settings, thread_num=15, - tasks=1000, + tasks=100, max_values_size=1000, array_size_range=[10, 15], ) @@ -152,7 +152,7 @@ def test_with_replicated_merge_tree(): _insert_queries_sequentially( table_name, settings, - iterations=100, + iterations=10, max_values_size=1000, array_size_range=[10, 50], ) @@ -180,7 +180,7 @@ def test_with_replicated_merge_tree_multithread(): table_name, _query_settings, thread_num=15, - tasks=1000, + tasks=100, max_values_size=1000, array_size_range=[10, 15], ) @@ -200,13 +200,13 @@ def test_compare_sequential_inserts_durations_for_adaptive_and_fixed_async_timeo fixed_tm_settings = copy.copy(_query_settings) fixed_tm_settings["async_insert_use_adaptive_busy_timeout"] = 0 - fixed_tm_settings["async_insert_busy_timeout_ms"] = 200 + fixed_tm_settings["async_insert_busy_timeout_ms"] = 100 fixed_tm_run_duration = timeit.timeit( lambda: _insert_queries_sequentially( fixed_tm_table_name, fixed_tm_settings, - iterations=100, + iterations=50, max_values_size=1000, array_size_range=[10, 50], ), @@ -231,13 +231,13 @@ def test_compare_sequential_inserts_durations_for_adaptive_and_fixed_async_timeo adaptive_tm_settings = copy.copy(_query_settings) adaptive_tm_settings["async_insert_busy_timeout_min_ms"] = 10 - adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 1000 + adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 500 adaptive_tm_run_duration = timeit.timeit( lambda: _insert_queries_sequentially( adaptive_tm_table_name, adaptive_tm_settings, - iterations=100, + iterations=50, max_values_size=1000, array_size_range=[10, 50], ), @@ -268,14 +268,14 @@ def test_compare_parallel_inserts_durations_for_adaptive_and_fixed_async_timeout fixed_tm_settings = copy.copy(_query_settings) fixed_tm_settings["async_insert_use_adaptive_busy_timeout"] = 0 - fixed_tm_settings["async_insert_busy_timeout_ms"] = 200 + fixed_tm_settings["async_insert_busy_timeout_ms"] = 500 fixed_tm_run_duration = timeit.timeit( lambda: _insert_queries_in_parallel( fixed_tm_table_name, fixed_tm_settings, thread_num=15, - tasks=1000, + tasks=150, max_values_size=1000, array_size_range=[10, 50], ), @@ -300,14 +300,14 @@ def test_compare_parallel_inserts_durations_for_adaptive_and_fixed_async_timeout adaptive_tm_settings = copy.copy(_query_settings) adaptive_tm_settings["async_insert_busy_timeout_min_ms"] = 10 - adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 200 + adaptive_tm_settings["async_insert_busy_timeout_max_ms"] = 500 adaptive_tm_run_duration = timeit.timeit( lambda: _insert_queries_in_parallel( adaptive_tm_table_name, adaptive_tm_settings, thread_num=15, - tasks=100, + tasks=150, max_values_size=1000, array_size_range=[10, 50], ), @@ -344,29 +344,34 @@ def test_change_queries_frequency(): settings = copy.copy(_query_settings) min_ms = 50 + max_ms = 200 + settings["async_insert_busy_timeout_min_ms"] = min_ms - settings["async_insert_busy_timeout_max_ms"] = 2000 + settings["async_insert_busy_timeout_max_ms"] = max_ms + + _insert_queries_sequentially( + table_name, + settings, + iterations=50, + max_values_size=1000, + array_size_range=[10, 50], + ) + + select_log_query = f"SELECT countIf(timeout_milliseconds - {min_ms} < 25) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" + res = node.query(select_log_query) + assert int(res) >= 5 _insert_queries_in_parallel( table_name, settings, - thread_num=15, + thread_num=20, tasks=2000, max_values_size=1000, array_size_range=[10, 15], ) - _insert_queries_sequentially( - table_name, - settings, - iterations=200, - max_values_size=1000, - array_size_range=[10, 50], - ) - - select_log_query = "SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 50" + select_log_query = f"SELECT countIf({max_ms} - timeout_milliseconds < 100) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) - for line in res.splitlines(): - assert int(line) == min_ms + assert int(res) >= 5 node.query("DROP TABLE IF EXISTS {} SYNC".format(table_name)) diff --git a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py index 5fc8f67b75b..ad3d35d8bbd 100644 --- a/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py +++ b/tests/integration/test_grant_and_revoke/test_with_table_engine_grant.py @@ -532,6 +532,7 @@ def test_current_database(): assert "Not enough privileges" in instance.query_and_get_error( "SELECT * FROM table", user="A" ) + instance.query("DROP TABLE default.table SYNC") def test_grant_with_replace_option(): From e6b18f2b300c7aec883d60619f432f7e4b4cca3a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 22 Aug 2024 19:05:07 +0000 Subject: [PATCH 048/154] black --- .../test_postgresql_replica_database_engine_2/test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_postgresql_replica_database_engine_2/test.py b/tests/integration/test_postgresql_replica_database_engine_2/test.py index 428e9ecdb96..0a364d7802b 100644 --- a/tests/integration/test_postgresql_replica_database_engine_2/test.py +++ b/tests/integration/test_postgresql_replica_database_engine_2/test.py @@ -61,6 +61,7 @@ pg_manager = PostgresManager() pg_manager2 = PostgresManager() pg_manager_instance2 = PostgresManager() + @pytest.fixture(scope="module") def started_cluster(): try: @@ -918,7 +919,7 @@ def test_failed_load_from_snapshot(started_cluster): def test_symbols_in_publication_name(started_cluster): id = uuid.uuid4() - db = f'test_{id}' + db = f"test_{id}" table = f"test_symbols_in_publication_name" pg_manager3 = PostgresManager() From 1aaf9a08c4e0433dcf2afc97770b5a595bd108ee Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 22 Aug 2024 23:25:38 +0000 Subject: [PATCH 049/154] fix async inserts with alter --- src/Interpreters/AsynchronousInsertQueue.cpp | 62 +++++++++++--------- 1 file changed, 34 insertions(+), 28 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 62777524c2a..52f8a8dd42a 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -33,6 +33,8 @@ #include #include #include +#include +#include namespace CurrentMetrics { @@ -308,6 +310,7 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const /* no_squash */ false, /* no_destination */ false, /* async_insert */ false); + auto table = interpreter.getTable(insert_query); auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context); @@ -318,6 +321,10 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const /// InterpreterInsertQuery::getTable() -> ITableFunction::execute(). if (insert_query.table_id) query_context->checkAccess(AccessType::INSERT, insert_query.table_id, sample_block.getNames()); + + insert_query.columns = std::make_shared(); + for (const auto & column : sample_block) + insert_query.columns->children.push_back(std::make_shared(column.name)); } AsynchronousInsertQueue::PushResult @@ -872,36 +879,35 @@ try } }; - Chunk chunk; - auto header = pipeline.getHeader(); - - if (key.data_kind == DataKind::Parsed) - chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); - else - chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); - - ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); - - auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) - { - LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); - queue_shard_flush_time_history.updateWithCurrentTime(); - - bool pulling_pipeline = false; - logQueryFinish( - query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); - }; - - - if (chunk.getNumRows() == 0) - { - finish_entries(); - log_and_add_finish_to_query_log(0, 0); - return; - } - try { + Chunk chunk; + auto header = pipeline.getHeader(); + + if (key.data_kind == DataKind::Parsed) + chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); + else + chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); + + ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); + + auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) + { + LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); + queue_shard_flush_time_history.updateWithCurrentTime(); + + bool pulling_pipeline = false; + logQueryFinish( + query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); + }; + + if (chunk.getNumRows() == 0) + { + finish_entries(); + log_and_add_finish_to_query_log(0, 0); + return; + } + size_t num_rows = chunk.getNumRows(); size_t num_bytes = chunk.bytes(); From 7517ef4cc9f990e1ac949366a1ec8688aba0420f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 16:08:04 +0000 Subject: [PATCH 050/154] fix async insert with alter modify column --- src/Interpreters/AsynchronousInsertQueue.cpp | 178 ++++++++++-------- src/Interpreters/AsynchronousInsertQueue.h | 5 + .../03229_async_insert_alter.reference | 8 + .../0_stateless/03229_async_insert_alter.sql | 46 +++++ .../03229_async_insert_alter_http.reference | 8 + .../03229_async_insert_alter_http.sh | 55 ++++++ 6 files changed, 226 insertions(+), 74 deletions(-) create mode 100644 tests/queries/0_stateless/03229_async_insert_alter.reference create mode 100644 tests/queries/0_stateless/03229_async_insert_alter.sql create mode 100644 tests/queries/0_stateless/03229_async_insert_alter_http.reference create mode 100755 tests/queries/0_stateless/03229_async_insert_alter_http.sh diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 52f8a8dd42a..5ed0d6d6257 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -699,6 +699,17 @@ catch (...) tryLogCurrentException("AsynchronousInsertQueue", "Failed to add elements to AsynchronousInsertLog"); } +void convertBlockToHeader(Block & block, const Block & header) +{ + auto converting_dag = ActionsDAG::makeConvertingActions( + block.getColumnsWithTypeAndName(), + header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + auto converting_actions = std::make_shared(std::move(converting_dag)); + converting_actions->execute(block); +} + String serializeQuery(const IAST & query, size_t max_length) { return query.hasSecretParts() @@ -794,6 +805,48 @@ try if (async_insert_log) log_elements.reserve(data->entries.size()); + auto add_entry_to_asynchronous_insert_log = [&]( + const InsertData::EntryPtr & entry, + const NameToNameMap & query_by_format, + const String & parsing_exception, + size_t num_rows, + size_t num_bytes) + { + if (!async_insert_log) + return; + + AsynchronousInsertLogElement elem; + elem.event_time = timeInSeconds(entry->create_time); + elem.event_time_microseconds = timeInMicroseconds(entry->create_time); + elem.database = query_database; + elem.table = query_table; + elem.format = entry->format; + elem.query_id = entry->query_id; + elem.bytes = num_bytes; + elem.rows = num_rows; + elem.exception = parsing_exception; + elem.data_kind = entry->chunk.getDataKind(); + elem.timeout_milliseconds = data->timeout_ms.count(); + elem.flush_query_id = insert_query_id; + + auto it = query_by_format.find(entry->format); + elem.query_for_logging = it != query_by_format.end() ? it->second : key.query_str; + + /// If there was a parsing error, + /// the entry won't be flushed anyway, + /// so add the log element immediately. + if (!elem.exception.empty()) + { + elem.status = AsynchronousInsertLogElement::ParsingError; + async_insert_log->add(std::move(elem)); + } + else + { + elem.status = AsynchronousInsertLogElement::Ok; + log_elements.push_back(elem); + } + }; + try { interpreter = std::make_unique( @@ -822,49 +875,21 @@ try catch (...) { logExceptionBeforeStart(query_for_logging, insert_context, key.query, query_span, start_watch.elapsedMilliseconds()); + + if (async_insert_log) + { + auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); + for (const auto & entry : data->entries) + add_entry_to_asynchronous_insert_log(entry, query_by_format, "", 0, entry->chunk.byteSize()); + + auto exception = getCurrentExceptionMessage(false); + auto flush_time = std::chrono::system_clock::now(); + appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, exception); + } throw; } - auto add_entry_to_asynchronous_insert_log = [&](const auto & entry, - const auto & entry_query_for_logging, - const auto & exception, - size_t num_rows, - size_t num_bytes, - Milliseconds timeout_ms) - { - if (!async_insert_log) - return; - - AsynchronousInsertLogElement elem; - elem.event_time = timeInSeconds(entry->create_time); - elem.event_time_microseconds = timeInMicroseconds(entry->create_time); - elem.query_for_logging = entry_query_for_logging; - elem.database = query_database; - elem.table = query_table; - elem.format = entry->format; - elem.query_id = entry->query_id; - elem.bytes = num_bytes; - elem.rows = num_rows; - elem.exception = exception; - elem.data_kind = entry->chunk.getDataKind(); - elem.timeout_milliseconds = timeout_ms.count(); - elem.flush_query_id = insert_query_id; - - /// If there was a parsing error, - /// the entry won't be flushed anyway, - /// so add the log element immediately. - if (!elem.exception.empty()) - { - elem.status = AsynchronousInsertLogElement::ParsingError; - async_insert_log->add(std::move(elem)); - } - else - { - log_elements.push_back(elem); - } - }; - - auto finish_entries = [&] + auto finish_entries = [&](size_t num_rows, size_t num_bytes) { for (const auto & entry : data->entries) { @@ -877,6 +902,13 @@ try auto flush_time = std::chrono::system_clock::now(); appendElementsToLogSafe(*async_insert_log, std::move(log_elements), flush_time, ""); } + + LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); + queue_shard_flush_time_history.updateWithCurrentTime(); + + bool pulling_pipeline = false; + logQueryFinish( + query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); }; try @@ -891,20 +923,9 @@ try ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); - auto log_and_add_finish_to_query_log = [&](size_t num_rows, size_t num_bytes) - { - LOG_DEBUG(log, "Flushed {} rows, {} bytes for query '{}'", num_rows, num_bytes, key.query_str); - queue_shard_flush_time_history.updateWithCurrentTime(); - - bool pulling_pipeline = false; - logQueryFinish( - query_log_elem, insert_context, key.query, pipeline, pulling_pipeline, query_span, QueryCache::Usage::None, internal); - }; - if (chunk.getNumRows() == 0) { - finish_entries(); - log_and_add_finish_to_query_log(0, 0); + finish_entries(0, 0); return; } @@ -917,7 +938,7 @@ try CompletedPipelineExecutor completed_executor(pipeline); completed_executor.execute(); - log_and_add_finish_to_query_log(num_rows, num_bytes); + finish_entries(num_rows, num_bytes); } catch (...) { @@ -931,8 +952,6 @@ try } throw; } - - finish_entries(); } catch (const Exception & e) { @@ -1013,7 +1032,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, query_for_logging, current_exception, num_rows, num_bytes, data->timeout_ms); + add_to_async_insert_log(entry, {}, current_exception, num_rows, num_bytes); current_exception.clear(); entry->resetChunk(); @@ -1036,19 +1055,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( auto chunk_info = std::make_shared(); auto result_columns = header.cloneEmptyColumns(); - std::unordered_map format_to_query; - - auto get_query_by_format = [&](const String & format) -> const String & - { - auto [it, inserted] = format_to_query.try_emplace(format); - if (!inserted) - return it->second; - - auto query = key.query->clone(); - assert_cast(*query).format = format; - it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); - return it->second; - }; + auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); for (const auto & entry : data->entries) { @@ -1057,17 +1064,19 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected entry with data kind Preprocessed. Got: {}", entry->chunk.getDataKind()); - auto columns = block->getColumns(); + Block block_to_insert = *block; + if (!isCompatibleHeader(block_to_insert, header)) + convertBlockToHeader(block_to_insert, header); + + auto columns = block_to_insert.getColumns(); for (size_t i = 0, s = columns.size(); i < s; ++i) result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); - total_rows += block->rows(); + total_rows += block_to_insert.rows(); chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - const auto & query_for_logging = get_query_by_format(entry->format); - add_to_async_insert_log(entry, query_for_logging, "", block->rows(), block->bytes(), data->timeout_ms); - + add_to_async_insert_log(entry, query_by_format, "", block_to_insert.rows(), block_to_insert.bytes()); entry->resetChunk(); } @@ -1076,6 +1085,27 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( return chunk; } +NameToNameMap AsynchronousInsertQueue::getQueriesByFormat( + const ASTPtr & query, + const std::list & entries, + const ContextPtr & insert_context) +{ + std::unordered_map format_to_query; + auto query_copy = query->clone(); + + for (const auto & entry : entries) + { + auto [it, inserted] = format_to_query.try_emplace(entry->format); + if (!inserted) + continue; + + assert_cast(*query_copy).format = entry->format; + it->second = serializeQuery(*query_copy, insert_context->getSettingsRef().log_queries_cut_to_length); + } + + return format_to_query; +} + template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index da14b43d276..9a84fe8bb12 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -293,6 +293,11 @@ private: const ContextPtr & insert_context, LogFunc && add_to_async_insert_log); + static NameToNameMap getQueriesByFormat( + const ASTPtr & query, + const std::list & entries, + const ContextPtr & insert_context); + template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); diff --git a/tests/queries/0_stateless/03229_async_insert_alter.reference b/tests/queries/0_stateless/03229_async_insert_alter.reference new file mode 100644 index 00000000000..f66021d0bfe --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter.reference @@ -0,0 +1,8 @@ +42 24 0 +42 24 0 +43 34 55 +42 24 +43 34 +INSERT INTO default.t_async_insert_alter (id, v1) FORMAT Values Preprocessed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Preprocessed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Preprocessed FlushError diff --git a/tests/queries/0_stateless/03229_async_insert_alter.sql b/tests/queries/0_stateless/03229_async_insert_alter.sql new file mode 100644 index 00000000000..a95bbc6e55f --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter.sql @@ -0,0 +1,46 @@ +-- Tags: no-parallel + +SET wait_for_async_insert = 0; +SET async_insert_busy_timeout_max_ms = 300000; +SET async_insert_busy_timeout_min_ms = 300000; +SET async_insert_use_adaptive_busy_timeout = 0; + +DROP TABLE IF EXISTS t_async_insert_alter; + +CREATE TABLE t_async_insert_alter (id Int64, v1 Int64) ENGINE = MergeTree ORDER BY id SETTINGS async_insert = 1; + +-- ADD COLUMN + +INSERT INTO t_async_insert_alter VALUES (42, 24); + +ALTER TABLE t_async_insert_alter ADD COLUMN value2 Int64; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; + +-- MODIFY COLUMN + +INSERT INTO t_async_insert_alter VALUES (43, 34, 55); + +ALTER TABLE t_async_insert_alter MODIFY COLUMN value2 String; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; + +-- DROP COLUMN + +INSERT INTO t_async_insert_alter VALUES ('100', '200', '300'); + +ALTER TABLE t_async_insert_alter DROP COLUMN value2; + +SYSTEM FLUSH ASYNC INSERT QUEUE; +SYSTEM FLUSH LOGS; + +SELECT * FROM t_async_insert_alter ORDER BY id; +SELECT query, data_kind, status FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_alter' ORDER BY event_time_microseconds; + +DROP TABLE t_async_insert_alter; diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.reference b/tests/queries/0_stateless/03229_async_insert_alter_http.reference new file mode 100644 index 00000000000..195701d2b82 --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.reference @@ -0,0 +1,8 @@ +42 24 0 +42 24 0 +43 34 55 +42 24 +43 34 +INSERT INTO default.t_async_insert_alter (id, v1) FORMAT Values Parsed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Parsed Ok +INSERT INTO default.t_async_insert_alter (id, v1, value2) FORMAT Values Parsed FlushError diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.sh b/tests/queries/0_stateless/03229_async_insert_alter_http.sh new file mode 100755 index 00000000000..18e68f51285 --- /dev/null +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.sh @@ -0,0 +1,55 @@ +#!/usr/bin/env bash +# Tags: no-parallel + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT -q " + DROP TABLE IF EXISTS t_async_insert_alter; + CREATE TABLE t_async_insert_alter (id Int64, v1 Int64) ENGINE = MergeTree ORDER BY id SETTINGS async_insert = 1; +" + +url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0" + +# ADD COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES (42, 24)" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter ADD COLUMN value2 Int64; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; +" + +# MODIFY COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES (43, 34, 55)" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter MODIFY COLUMN value2 String; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; +" + +## DROP COLUMN + +${CLICKHOUSE_CURL} -sS "$url" -d "INSERT INTO t_async_insert_alter VALUES ('100', '200', '300')" + +$CLICKHOUSE_CLIENT -q " + ALTER TABLE t_async_insert_alter DROP COLUMN value2; + + SYSTEM FLUSH ASYNC INSERT QUEUE; + SYSTEM FLUSH LOGS; + + SELECT * FROM t_async_insert_alter ORDER BY id; + SELECT query, data_kind, status FROM system.asynchronous_insert_log WHERE database = currentDatabase() AND table = 't_async_insert_alter' ORDER BY event_time_microseconds; + + DROP TABLE t_async_insert_alter; +" From b39c5cdb4021fd2b52fde4d320e421abcd868c77 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 23 Aug 2024 16:17:14 +0000 Subject: [PATCH 051/154] fix tests --- .../02790_async_queries_in_query_log.reference | 12 ++++++------ ...03148_async_queries_in_query_log_errors.reference | 6 +++--- .../queries/0_stateless/03229_async_insert_alter.sql | 1 + .../0_stateless/03229_async_insert_alter_http.sh | 1 + 4 files changed, 11 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference index 567e8d4f4b5..84254cc8735 100644 --- a/tests/queries/0_stateless/02790_async_queries_in_query_log.reference +++ b/tests/queries/0_stateless/02790_async_queries_in_query_log.reference @@ -9,7 +9,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -26,7 +26,7 @@ written_rows: 4 written_bytes: 16 result_rows: 4 result_bytes: 16 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -54,7 +54,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -71,7 +71,7 @@ written_rows: 6 written_bytes: 24 result_rows: 6 result_bytes: 24 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -118,7 +118,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] @@ -135,7 +135,7 @@ written_rows: 3 written_bytes: 12 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 1, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing','default.async_insert_target'] diff --git a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference index 685d28268f6..7cc2efd50ec 100644 --- a/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference +++ b/tests/queries/0_stateless/03148_async_queries_in_query_log_errors.reference @@ -4,7 +4,7 @@ Row 1: ────── database: default table: async_insert_landing -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values format: Values error: DB::Exc*****on: Cannot parse string 'Invalid' as UInt32: populated_flush_query_id: 1 @@ -18,7 +18,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] @@ -35,7 +35,7 @@ written_rows: 0 written_bytes: 0 result_rows: 0 result_bytes: 0 -query: INSERT INTO default.async_insert_landing SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values +query: INSERT INTO default.async_insert_landing (id) SETTINGS wait_for_async_insert = 0, async_insert = 1 FORMAT Values query_kind: AsyncInsertFlush databases: ['default'] tables: ['default.async_insert_landing'] diff --git a/tests/queries/0_stateless/03229_async_insert_alter.sql b/tests/queries/0_stateless/03229_async_insert_alter.sql index a95bbc6e55f..022e386bef0 100644 --- a/tests/queries/0_stateless/03229_async_insert_alter.sql +++ b/tests/queries/0_stateless/03229_async_insert_alter.sql @@ -1,4 +1,5 @@ -- Tags: no-parallel +-- no-parallel because the test uses FLUSH ASYNC INSERT QUEUE SET wait_for_async_insert = 0; SET async_insert_busy_timeout_max_ms = 300000; diff --git a/tests/queries/0_stateless/03229_async_insert_alter_http.sh b/tests/queries/0_stateless/03229_async_insert_alter_http.sh index 18e68f51285..fe72ed3299a 100755 --- a/tests/queries/0_stateless/03229_async_insert_alter_http.sh +++ b/tests/queries/0_stateless/03229_async_insert_alter_http.sh @@ -1,5 +1,6 @@ #!/usr/bin/env bash # Tags: no-parallel +# no-parallel because the test uses FLUSH ASYNC INSERT QUEUE CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 66be44e7aa719681b7f13e31a91f2c105e705a4f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 23 Aug 2024 19:09:09 +0000 Subject: [PATCH 052/154] better --- tests/ci/integration_tests_runner.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index 074c947fe02..deddcb95cf4 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -33,7 +33,7 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 10 # run whole pytest several times +FLAKY_TRIES_COUNT = 5 # run whole pytest several times FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 @@ -794,7 +794,7 @@ class ClickhouseIntegrationTestsRunner: } # type: Dict tests_times = defaultdict(float) # type: Dict tests_log_paths = defaultdict(list) - + id_counter = 0 for test_to_run in tests_to_run: tries_num = 1 if should_fail else FLAKY_TRIES_COUNT for i in range(tries_num): @@ -805,12 +805,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Running tests for the %s time", i) group_counters, group_test_times, log_paths = self.try_run_test_group( repo_path, - "bugfix" if should_fail else "flaky", + f"bugfix_{id_counter}" if should_fail else f"flaky{id_counter}", [test_to_run], 1, 1, FLAKY_REPEAT_COUNT, ) + id_counter = id_counter + 1 for counter, value in group_counters.items(): logging.info( "Tests from group %s stats, %s count %s", From 9339c0593b354dbb54b418425c6cfe1ac37e556c Mon Sep 17 00:00:00 2001 From: Max Kainov Date: Mon, 26 Aug 2024 13:37:37 +0200 Subject: [PATCH 053/154] handle timeout in flaky check --- tests/ci/integration_tests_runner.py | 67 ++++++++++++++-------------- 1 file changed, 33 insertions(+), 34 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index deddcb95cf4..a7d5a8c4cf6 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -873,21 +873,45 @@ class ClickhouseIntegrationTestsRunner: def run_impl(self, repo_path, build_path): stopwatch = Stopwatch() if self.flaky_check or self.bugfix_validate_check: - return self.run_flaky_check( - repo_path, build_path, should_fail=self.bugfix_validate_check + result_state, status_text, test_result, tests_log_paths = ( + self.run_flaky_check( + repo_path, build_path, should_fail=self.bugfix_validate_check + ) + ) + else: + result_state, status_text, test_result, tests_log_paths = ( + self.run_normal_check(build_path, repo_path) ) - self._install_clickhouse(build_path) + if self.soft_deadline_time < time.time(): + status_text = "Timeout, " + status_text + result_state = "failure" + if timeout_expired: + logging.error( + "Job killed by external timeout signal - setting status to failure!" + ) + status_text = "Job timeout expired, " + status_text + result_state = "failure" + # add mock test case to make timeout visible in job report and in ci db + test_result.insert( + 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") + ) + + if "(memory)" in self.params["context_name"]: + result_state = "success" + + return result_state, status_text, test_result, tests_log_paths + + def run_normal_check(self, build_path, repo_path): + self._install_clickhouse(build_path) logging.info("Pulling images") self._pre_pull_images(repo_path) - logging.info( "Dump iptables before run %s", subprocess.check_output("sudo iptables -nvL", shell=True), ) all_tests = self._get_all_tests(repo_path) - if self.run_by_hash_total != 0: grouped_tests = self.group_test_by_file(all_tests) all_filtered_by_hash_tests = [] @@ -895,7 +919,6 @@ class ClickhouseIntegrationTestsRunner: if stringhash(group) % self.run_by_hash_total == self.run_by_hash_num: all_filtered_by_hash_tests += tests_in_group all_tests = all_filtered_by_hash_tests - parallel_skip_tests = self._get_parallel_tests_skip_list(repo_path) logging.info( "Found %s tests first 3 %s", len(all_tests), " ".join(all_tests[:3]) @@ -927,14 +950,12 @@ class ClickhouseIntegrationTestsRunner: len(not_found_tests), " ".join(not_found_tests[:3]), ) - grouped_tests = self.group_test_by_file(filtered_sequential_tests) i = 0 for par_group in chunks(filtered_parallel_tests, PARALLEL_GROUP_SIZE): grouped_tests[f"parallel{i}"] = par_group i += 1 logging.info("Found %s tests groups", len(grouped_tests)) - counters = { "ERROR": [], "PASSED": [], @@ -945,14 +966,11 @@ class ClickhouseIntegrationTestsRunner: } # type: Dict tests_times = defaultdict(float) tests_log_paths = defaultdict(list) - items_to_run = list(grouped_tests.items()) - logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): logging.info("Shuffling test groups") random.shuffle(items_to_run) - for group, tests in items_to_run: if timeout_expired: print("Timeout expired - break tests execution") @@ -980,7 +998,6 @@ class ClickhouseIntegrationTestsRunner: if len(counters["FAILED"]) + len(counters["ERROR"]) >= 20: logging.info("Collected more than 20 failed/error tests, stopping") break - if counters["FAILED"] or counters["ERROR"]: logging.info( "Overall status failure, because we have tests in FAILED or ERROR state" @@ -989,7 +1006,6 @@ class ClickhouseIntegrationTestsRunner: else: logging.info("Overall success!") result_state = "success" - test_result = [] for state in ( "ERROR", @@ -1009,33 +1025,14 @@ class ClickhouseIntegrationTestsRunner: (c, text_state, f"{tests_times[c]:.2f}", tests_log_paths[c]) for c in counters[state] ] - failed_sum = len(counters["FAILED"]) + len(counters["ERROR"]) status_text = f"fail: {failed_sum}, passed: {len(counters['PASSED'])}" - if self.soft_deadline_time < time.time(): - status_text = "Timeout, " + status_text - result_state = "failure" - - if timeout_expired: - logging.error( - "Job killed by external timeout signal - setting status to failure!" - ) - status_text = "Job timeout expired, " + status_text - result_state = "failure" - # add mock test case to make timeout visible in job report and in ci db - test_result.insert( - 0, (JOB_TIMEOUT_TEST_NAME, "FAIL", f"{stopwatch.duration_seconds}", "") - ) - if not counters or sum(len(counter) for counter in counters.values()) == 0: status_text = "No tests found for some reason! It's a bug" result_state = "failure" - if "(memory)" in self.params["context_name"]: - result_state = "success" - - return result_state, status_text, test_result, [] + return result_state, status_text, test_result, tests_log_paths def write_results(results_file, status_file, results, status): @@ -1068,7 +1065,9 @@ def run(): logging.info("Clearing dmesg before run") subprocess.check_call("sudo -E dmesg --clear", shell=True) - state, description, test_results, _ = runner.run_impl(repo_path, build_path) + state, description, test_results, _test_log_paths = runner.run_impl( + repo_path, build_path + ) logging.info("Tests finished") if IS_CI: From 5bad631d468a8cddbd1af40dc43c2d5db2931b46 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 054/154] Add config for outout_format_identifers_quoting_style --- src/Core/Settings.h | 2 ++ src/Core/SettingsChangesHistory.cpp | 5 +++++ src/Core/SettingsEnums.cpp | 6 ++++++ src/Core/SettingsEnums.h | 3 +++ src/Interpreters/formatWithPossiblyHidingSecrets.h | 2 +- src/Parsers/IAST.cpp | 5 ++++- src/Parsers/IAST.h | 2 +- 7 files changed, 22 insertions(+), 3 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 23dc2a8fdc5..c33562e25ed 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1296,6 +1296,8 @@ class IColumn; M(Bool, precise_float_parsing, false, "Prefer more precise (but slower) float parsing algorithm", 0) \ M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \ + \ + M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \ // End of FORMAT_FACTORY_SETTINGS diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 5e831c6301c..76ec440aaf2 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,6 +57,11 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { + {"24.12", + { + {"output_format_identifier_quoting_style", 1, 1, "New setting."} + } + }, {"24.12", { } diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index 30e60f246f4..0c43b5256d6 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -244,4 +244,10 @@ IMPLEMENT_SETTING_ENUM( GroupArrayActionWhenLimitReached, ErrorCodes::BAD_ARGUMENTS, {{"throw", GroupArrayActionWhenLimitReached::THROW}, {"discard", GroupArrayActionWhenLimitReached::DISCARD}}) + +IMPLEMENT_SETTING_ENUM(IdentifierQuotingStyle, ErrorCodes::BAD_ARGUMENTS, + {{"None", IdentifierQuotingStyle::None}, + {"Backticks", IdentifierQuotingStyle::Backticks}, + {"DoubleQuotes", IdentifierQuotingStyle::DoubleQuotes}, + {"BackticksMySQL", IdentifierQuotingStyle::BackticksMySQL}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 5c2d8825e76..d358a2d44a0 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -351,6 +352,8 @@ DECLARE_SETTING_ENUM_WITH_RENAME(DateTimeOverflowBehavior, FormatSettings::DateT DECLARE_SETTING_ENUM(SQLSecurityType) +DECLARE_SETTING_ENUM(IdentifierQuotingStyle) + enum class GroupArrayActionWhenLimitReached : uint8_t { THROW, diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 14e84f1d1a4..8ec40d3dcc3 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -26,7 +26,7 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); return settings.query.formatWithPossiblyHidingSensitiveData( - settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names); + settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names, settings.ctx->getSettingsRef().output_format_identifier_quoting_style); } } diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 5bd2c92c60a..90cdd330ee8 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,12 +165,15 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const +String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style) const { + WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; settings.print_pretty_type_names = print_pretty_type_names; + settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + settings.identifier_quoting_style = identifier_quoting_style; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); } diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 2293d50b0ec..777265aaa7c 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -278,7 +278,7 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names) const; + String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about From d6ca589ebed7a16f498fbbe4365d9ac07686af08 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 055/154] Treat PROJECTION, DICTIONARY, COLUMN, INDEX names as identifiers when formating --- src/Core/SettingsChangesHistory.cpp | 2 +- src/Parsers/ASTColumnDeclaration.cpp | 3 +-- src/Parsers/ASTDictionaryAttributeDeclaration.cpp | 2 +- src/Parsers/ASTIndexDeclaration.cpp | 2 +- src/Parsers/ASTProjectionDeclaration.cpp | 2 +- 5 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 76ec440aaf2..46f6590b724 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,7 +59,7 @@ static std::initializer_listformatImpl(s, state, frame); } diff --git a/src/Parsers/ASTProjectionDeclaration.cpp b/src/Parsers/ASTProjectionDeclaration.cpp index 60050986161..af79745a88e 100644 --- a/src/Parsers/ASTProjectionDeclaration.cpp +++ b/src/Parsers/ASTProjectionDeclaration.cpp @@ -17,7 +17,7 @@ ASTPtr ASTProjectionDeclaration::clone() const void ASTProjectionDeclaration::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - settings.ostr << backQuoteIfNeed(name); + settings.writeIdentifier(name); std::string indent_str = settings.one_line ? "" : std::string(4u * frame.indent, ' '); std::string nl_or_nothing = settings.one_line ? "" : "\n"; settings.ostr << settings.nl_or_ws << indent_str << "(" << nl_or_nothing; From e1d66b9e262466695bff2a30cccec047a953e269 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 056/154] Add output_format_always_quote_identifiers into the settings --- src/Core/ExternalTable.cpp | 12 ++++++++--- src/Core/Settings.h | 1 + src/Core/SettingsChangesHistory.cpp | 1 + src/Interpreters/executeQuery.cpp | 4 ++-- .../formatWithPossiblyHidingSecrets.h | 7 ++++++- src/Parsers/ASTColumnDeclaration.cpp | 3 ++- src/Parsers/IAST.cpp | 20 ++++++++++++------- src/Parsers/IAST.h | 15 ++++++++++---- 8 files changed, 45 insertions(+), 18 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 4ff0d7092d8..1f8e63db72e 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -17,11 +17,12 @@ #include #include -#include #include #include #include #include +#include +#include "Parsers/IdentifierQuotingStyle.h" namespace DB @@ -85,7 +86,9 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg /// We use `formatWithPossiblyHidingSensitiveData` instead of `getColumnNameWithoutAlias` because `column->type` is an ASTFunction. /// `getColumnNameWithoutAlias` will return name of the function with `(arguments)` even if arguments is empty. if (column) - structure.emplace_back(column->name, column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false)); + structure.emplace_back( + column->name, + column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -102,7 +105,10 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: {}", error); for (size_t i = 0; i < type_list_raw->children.size(); ++i) - structure.emplace_back("_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData(0, true, true, false)); + structure.emplace_back( + "_" + toString(i + 1), + type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData( + 0, true, true, false, false, IdentifierQuotingStyle::Backticks)); } void BaseExternalTable::initSampleBlock() diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c33562e25ed..0e58562c745 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -1297,6 +1297,7 @@ class IColumn; M(DateTimeOverflowBehavior, date_time_overflow_behavior, "ignore", "Overflow mode for Date, Date32, DateTime, DateTime64 types. Possible values: 'ignore', 'throw', 'saturate'.", 0) \ M(Bool, validate_experimental_and_suspicious_types_inside_nested_types, true, "Validate usage of experimental and suspicious types inside nested types like Array/Map/Tuple", 0) \ \ + M(Bool, output_format_always_quote_identifiers, false, "Always quote identifiers", 0) \ M(IdentifierQuotingStyle, output_format_identifier_quoting_style, IdentifierQuotingStyle::Backticks, "Set the quoting style for identifiers", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 46f6590b724..ac7100d3706 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -59,6 +59,7 @@ static std::initializer_list executeQueryImpl( /// Verify that AST formatting is consistent: /// If you format AST, parse it back, and format it again, you get the same string. - String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false); + String formatted1 = ast->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks); /// The query can become more verbose after formatting, so: size_t new_max_query_size = max_query_size > 0 ? (1000 + 2 * max_query_size) : 0; @@ -811,7 +811,7 @@ static std::tuple executeQueryImpl( chassert(ast2); - String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false); + String formatted2 = ast2->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks); if (formatted1 != formatted2) throw Exception(ErrorCodes::LOGICAL_ERROR, diff --git a/src/Interpreters/formatWithPossiblyHidingSecrets.h b/src/Interpreters/formatWithPossiblyHidingSecrets.h index 8ec40d3dcc3..61f689fb821 100644 --- a/src/Interpreters/formatWithPossiblyHidingSecrets.h +++ b/src/Interpreters/formatWithPossiblyHidingSecrets.h @@ -26,7 +26,12 @@ inline String format(const SecretHidingFormatSettings & settings) && settings.ctx->getAccess()->isGranted(AccessType::displaySecretsInShowAndSelect); return settings.query.formatWithPossiblyHidingSensitiveData( - settings.max_length, settings.one_line, show_secrets, settings.ctx->getSettingsRef().print_pretty_type_names, settings.ctx->getSettingsRef().output_format_identifier_quoting_style); + settings.max_length, + settings.one_line, + show_secrets, + settings.ctx->getSettingsRef().print_pretty_type_names, + settings.ctx->getSettingsRef().output_format_always_quote_identifiers, + settings.ctx->getSettingsRef().output_format_identifier_quoting_style); } } diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 39b1da68a86..7b5ea49b63d 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -66,7 +66,8 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo { frame.need_parens = false; - format_settings.writeIdentifier(name); + /// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query. + format_settings.writeIdentifier(name, true); if (type) { diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 90cdd330ee8..858630cef2e 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -165,14 +165,20 @@ size_t IAST::checkDepthImpl(size_t max_depth) const return res; } -String IAST::formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style) const +String IAST::formatWithPossiblyHidingSensitiveData( + size_t max_length, + bool one_line, + bool show_secrets, + bool print_pretty_type_names, + bool always_quote_identifiers, + IdentifierQuotingStyle identifier_quoting_style) const { WriteBufferFromOwnString buf; FormatSettings settings(buf, one_line); settings.show_secrets = show_secrets; settings.print_pretty_type_names = print_pretty_type_names; - settings.always_quote_identifiers = identifier_quoting_style != IdentifierQuotingStyle::None; + settings.always_quote_identifiers = always_quote_identifiers; settings.identifier_quoting_style = identifier_quoting_style; format(settings); return wipeSensitiveDataAndCutToLength(buf.str(), max_length); @@ -211,13 +217,13 @@ String IAST::getColumnNameWithoutAlias() const } -void IAST::FormatSettings::writeIdentifier(const String & name) const +void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoting) const { switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible arguments: always_quote_identifiers = true && " "identifier_quoting_style == IdentifierQuotingStyle::None"); @@ -226,7 +232,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::Backticks: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeBackQuotedString(name, ostr); else writeProbablyBackQuotedString(name, ostr); @@ -234,7 +240,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::DoubleQuotes: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeDoubleQuotedString(name, ostr); else writeProbablyDoubleQuotedString(name, ostr); @@ -242,7 +248,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name) const } case IdentifierQuotingStyle::BackticksMySQL: { - if (always_quote_identifiers) + if (force_quoting || always_quote_identifiers) writeBackQuotedStringMySQL(name, ostr); else writeProbablyBackQuotedStringMySQL(name, ostr); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index 777265aaa7c..c8fb93d3bf6 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -237,7 +237,8 @@ public: { } - void writeIdentifier(const String & name) const; + // If `force_quoting` is true, `name` is always quoted regardless of `always_quote_identifiers` + void writeIdentifier(const String & name, bool force_quoting = false) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. @@ -278,7 +279,13 @@ public: /// Secrets are displayed regarding show_secrets, then SensitiveDataMasker is applied. /// You can use Interpreters/formatWithPossiblyHidingSecrets.h for convenience. - String formatWithPossiblyHidingSensitiveData(size_t max_length, bool one_line, bool show_secrets, bool print_pretty_type_names, IdentifierQuotingStyle identifier_quoting_style = IdentifierQuotingStyle::Backticks) const; + String formatWithPossiblyHidingSensitiveData( + size_t max_length, + bool one_line, + bool show_secrets, + bool print_pretty_type_names, + bool always_quote_identifiers, + IdentifierQuotingStyle identifier_quoting_style) const; /** formatForLogging and formatForErrorMessage always hide secrets. This inconsistent * behaviour is due to the fact such functions are called from Client which knows nothing about @@ -287,12 +294,12 @@ public: */ String formatForLogging(size_t max_length = 0) const { - return formatWithPossiblyHidingSensitiveData(max_length, true, false, false); + return formatWithPossiblyHidingSensitiveData(max_length, true, false, false, false, IdentifierQuotingStyle::Backticks); } String formatForErrorMessage() const { - return formatWithPossiblyHidingSensitiveData(0, true, false, false); + return formatWithPossiblyHidingSensitiveData(0, true, false, false, false, IdentifierQuotingStyle::Backticks); } virtual bool hasSecretParts() const { return childrenHaveSecretParts(); } From 32e995acbb1ac613217190561fbdda976c4386b5 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 057/154] Update settings change history version --- src/Core/SettingsChangesHistory.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index ac7100d3706..982cd0389a9 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,7 +57,7 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.12", + {"24.13", { {"output_format_always_quote_identifiers", false, false, "New setting."}, {"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."} From 8e6de7e767e3db62efab622a7cf05e81da5d7866 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Mon, 26 Aug 2024 15:26:21 +0000 Subject: [PATCH 058/154] 1) Fix: move new setting to 24.9 key. 2) Fix: fource quoting dictionary attribute names --- src/Core/SettingsChangesHistory.cpp | 8 ++------ src/Parsers/ASTDictionaryAttributeDeclaration.cpp | 2 +- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 982cd0389a9..258065dcfd4 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,12 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.13", - { - {"output_format_always_quote_identifiers", false, false, "New setting."}, - {"output_format_identifier_quoting_style", "Backticks", "Backticks", "New setting."} - } - }, {"24.12", { } @@ -81,6 +75,8 @@ static std::initializer_list Date: Mon, 26 Aug 2024 18:43:21 +0200 Subject: [PATCH 059/154] increase timeout for flaky check --- tests/ci/ci_config.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5453bffd9c6..5de572bc0cb 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,6 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, + timeout=4*3600 # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From 34d13bd7467610daec7004640e1ab07da8a03f81 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 26 Aug 2024 19:33:04 +0200 Subject: [PATCH 060/154] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 5de572bc0cb..0d5f29d705f 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,7 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, - timeout=4*3600 # to be able to process many updated tests + timeout=4 * 3600 # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From f9f960a5ed65b479257fff70ce2ef7234c61406d Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 26 Aug 2024 22:04:54 +0200 Subject: [PATCH 061/154] Update ci_config.py --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 0d5f29d705f..e594168ece3 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -424,7 +424,7 @@ class CI: pr_only=True, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.INTEGRATION_TEST_TSAN, - timeout=4 * 3600 # to be able to process many updated tests + timeout=4 * 3600, # to be able to process many updated tests ), JobNames.COMPATIBILITY_TEST: CommonJobConfigs.COMPATIBILITY_TEST.with_properties( required_builds=[BuildNames.PACKAGE_RELEASE], From 69cf54192c3e4a74db645f48afad90f52dd2da1e Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 27 Aug 2024 09:33:45 +0000 Subject: [PATCH 062/154] 1) Add tests for the identifier quoting style setting. 2) Update uni-test for table overrides --- src/Parsers/ASTColumnDeclaration.cpp | 2 +- .../ASTDictionaryAttributeDeclaration.cpp | 2 +- src/Parsers/IAST.cpp | 38 +- src/Parsers/IAST.h | 6 +- src/Parsers/tests/gtest_Parser.cpp | 2 +- ..._format_identifier_quoting_style.reference | 40 +++ ...output_format_identifier_quoting_style.sql | 328 ++++++++++++++++++ 7 files changed, 408 insertions(+), 10 deletions(-) create mode 100644 tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference create mode 100644 tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql diff --git a/src/Parsers/ASTColumnDeclaration.cpp b/src/Parsers/ASTColumnDeclaration.cpp index 7b5ea49b63d..d7728462df3 100644 --- a/src/Parsers/ASTColumnDeclaration.cpp +++ b/src/Parsers/ASTColumnDeclaration.cpp @@ -67,7 +67,7 @@ void ASTColumnDeclaration::formatImpl(const FormatSettings & format_settings, Fo frame.need_parens = false; /// We have to always quote column names to avoid ambiguity with INDEX and other declarations in CREATE query. - format_settings.writeIdentifier(name, true); + format_settings.quoteIdentifier(name); if (type) { diff --git a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp index 070cc6f222a..a600987dc45 100644 --- a/src/Parsers/ASTDictionaryAttributeDeclaration.cpp +++ b/src/Parsers/ASTDictionaryAttributeDeclaration.cpp @@ -35,7 +35,7 @@ void ASTDictionaryAttributeDeclaration::formatImpl(const FormatSettings & settin { frame.need_parens = false; - settings.writeIdentifier(name, true); + settings.quoteIdentifier(name); if (type) { diff --git a/src/Parsers/IAST.cpp b/src/Parsers/IAST.cpp index 858630cef2e..54b11f2888e 100644 --- a/src/Parsers/IAST.cpp +++ b/src/Parsers/IAST.cpp @@ -217,13 +217,13 @@ String IAST::getColumnNameWithoutAlias() const } -void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoting) const +void IAST::FormatSettings::writeIdentifier(const String & name) const { switch (identifier_quoting_style) { case IdentifierQuotingStyle::None: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Incompatible arguments: always_quote_identifiers = true && " "identifier_quoting_style == IdentifierQuotingStyle::None"); @@ -232,7 +232,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::Backticks: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeBackQuotedString(name, ostr); else writeProbablyBackQuotedString(name, ostr); @@ -240,7 +240,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::DoubleQuotes: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeDoubleQuotedString(name, ostr); else writeProbablyDoubleQuotedString(name, ostr); @@ -248,7 +248,7 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } case IdentifierQuotingStyle::BackticksMySQL: { - if (force_quoting || always_quote_identifiers) + if (always_quote_identifiers) writeBackQuotedStringMySQL(name, ostr); else writeProbablyBackQuotedStringMySQL(name, ostr); @@ -257,6 +257,34 @@ void IAST::FormatSettings::writeIdentifier(const String & name, bool force_quoti } } + +void IAST::FormatSettings::quoteIdentifier(const String & name) const +{ + switch (identifier_quoting_style) + { + case IdentifierQuotingStyle::None: + { + writeBackQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::Backticks: + { + writeBackQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::DoubleQuotes: + { + writeDoubleQuotedString(name, ostr); + break; + } + case IdentifierQuotingStyle::BackticksMySQL: + { + writeBackQuotedStringMySQL(name, ostr); + break; + } + } +} + void IAST::dumpTree(WriteBuffer & ostr, size_t indent) const { String indent_str(indent, '-'); diff --git a/src/Parsers/IAST.h b/src/Parsers/IAST.h index c8fb93d3bf6..dfb6a6cbeba 100644 --- a/src/Parsers/IAST.h +++ b/src/Parsers/IAST.h @@ -237,8 +237,10 @@ public: { } - // If `force_quoting` is true, `name` is always quoted regardless of `always_quote_identifiers` - void writeIdentifier(const String & name, bool force_quoting = false) const; + void writeIdentifier(const String & name) const; + // Quote identifier `name` even when `always_quote_identifiers` is false. + // If `identifier_quoting_style` is `IdentifierQuotingStyle::None`, quote it with `IdentifierQuotingStyle::Backticks` + void quoteIdentifier(const String & name) const; }; /// State. For example, a set of nodes can be remembered, which we already walk through. diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index f0abc68f966..98cd9682c9c 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -243,7 +243,7 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, }, { "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX foo `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" + "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX `foo` `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" }, { "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", diff --git a/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference new file mode 100644 index 00000000000..c563617a01c --- /dev/null +++ b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.reference @@ -0,0 +1,40 @@ +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid` +CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month` +CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE "default"."uk_price_paid"\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX "county_index" "county" TYPE set(10) GRANULARITY 1,\n PROJECTION "town_date_projection"\n (\n SELECT \n "town",\n "date",\n "price"\n ORDER BY \n "town",\n "date"\n ),\n PROJECTION "handy_aggs_projection"\n (\n SELECT \n avg("price"),\n max("price"),\n sum("price")\n GROUP BY "town"\n )\n)\nENGINE = MergeTree\nORDER BY ("postcode1", "postcode2", "date")\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW "default"."prices_by_year_view" TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n "price",\n "date",\n "addr1",\n "addr2",\n "street",\n "town",\n "district",\n "county"\nFROM "default"."uk_price_paid" +CREATE TABLE "default"."uk_prices_aggs_dest"\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY "month"\nORDER BY "month"\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW "default"."uk_prices_aggs_view" TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction("min", UInt32),\n "max_price" SimpleAggregateFunction("max", UInt32),\n "volume" AggregateFunction("count", UInt32),\n "avg_price" AggregateFunction("avg", UInt32)\n)\nAS WITH toStartOfMonth("date") AS "month"\nSELECT\n "month",\n minSimpleState("price") AS "min_price",\n maxSimpleState("price") AS "max_price",\n countState("price") AS "volume",\n avgState("price") AS "avg_price"\nFROM "default"."uk_price_paid"\nGROUP BY "month" +CREATE DICTIONARY "default"."uk_mortgage_rates_dict"\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY "date"\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n "price" UInt32,\n "date" Date,\n "postcode1" LowCardinality(String),\n "postcode2" LowCardinality(String),\n "type" Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n "is_new" UInt8,\n "duration" Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "locality" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n "price" UInt32,\n "date" Date,\n "addr1" String,\n "addr2" String,\n "street" LowCardinality(String),\n "town" LowCardinality(String),\n "district" LowCardinality(String),\n "county" LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n "month" Date,\n "min_price" SimpleAggregateFunction(min, UInt32),\n "max_price" SimpleAggregateFunction(max, UInt32),\n "volume" AggregateFunction(count, UInt32),\n "avg_price" AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n "date" DateTime64,\n "variable" Decimal32(2),\n "fixed" Decimal32(2),\n "bank" Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE `default`.`uk_price_paid`\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX `county_index` `county` TYPE set(10) GRANULARITY 1,\n PROJECTION `town_date_projection`\n (\n SELECT \n `town`,\n `date`,\n `price`\n ORDER BY \n `town`,\n `date`\n ),\n PROJECTION `handy_aggs_projection`\n (\n SELECT \n avg(`price`),\n max(`price`),\n sum(`price`)\n GROUP BY `town`\n )\n)\nENGINE = MergeTree\nORDER BY (`postcode1`, `postcode2`, `date`)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`prices_by_year_view` TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n `price`,\n `date`,\n `addr1`,\n `addr2`,\n `street`,\n `town`,\n `district`,\n `county`\nFROM `default`.`uk_price_paid` +CREATE TABLE `default`.`uk_prices_aggs_dest`\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY `month`\nORDER BY `month`\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW `default`.`uk_prices_aggs_view` TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(`min`, UInt32),\n `max_price` SimpleAggregateFunction(`max`, UInt32),\n `volume` AggregateFunction(`count`, UInt32),\n `avg_price` AggregateFunction(`avg`, UInt32)\n)\nAS WITH toStartOfMonth(`date`) AS `month`\nSELECT\n `month`,\n minSimpleState(`price`) AS `min_price`,\n maxSimpleState(`price`) AS `max_price`,\n countState(`price`) AS `volume`,\n avgState(`price`) AS `avg_price`\nFROM `default`.`uk_price_paid`\nGROUP BY `month` +CREATE DICTIONARY `default`.`uk_mortgage_rates_dict`\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY `date`\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) +CREATE TABLE default.uk_price_paid\n(\n `price` UInt32,\n `date` Date,\n `postcode1` LowCardinality(String),\n `postcode2` LowCardinality(String),\n `type` Enum8(\'other\' = 0, \'terraced\' = 1, \'semi-detached\' = 2, \'detached\' = 3, \'flat\' = 4),\n `is_new` UInt8,\n `duration` Enum8(\'unknown\' = 0, \'freehold\' = 1, \'leasehold\' = 2),\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `locality` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String),\n INDEX county_index county TYPE set(10) GRANULARITY 1,\n PROJECTION town_date_projection\n (\n SELECT \n town,\n date,\n price\n ORDER BY \n town,\n date\n ),\n PROJECTION handy_aggs_projection\n (\n SELECT \n avg(price),\n max(price),\n sum(price)\n GROUP BY town\n )\n)\nENGINE = MergeTree\nORDER BY (postcode1, postcode2, date)\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.prices_by_year_view TO default.prices_by_year_dest\n(\n `price` UInt32,\n `date` Date,\n `addr1` String,\n `addr2` String,\n `street` LowCardinality(String),\n `town` LowCardinality(String),\n `district` LowCardinality(String),\n `county` LowCardinality(String)\n)\nAS SELECT\n price,\n date,\n addr1,\n addr2,\n street,\n town,\n district,\n county\nFROM default.uk_price_paid +CREATE TABLE default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nENGINE = AggregatingMergeTree\nPRIMARY KEY month\nORDER BY month\nSETTINGS index_granularity = 8192 +CREATE MATERIALIZED VIEW default.uk_prices_aggs_view TO default.uk_prices_aggs_dest\n(\n `month` Date,\n `min_price` SimpleAggregateFunction(min, UInt32),\n `max_price` SimpleAggregateFunction(max, UInt32),\n `volume` AggregateFunction(count, UInt32),\n `avg_price` AggregateFunction(avg, UInt32)\n)\nAS WITH toStartOfMonth(date) AS month\nSELECT\n month,\n minSimpleState(price) AS min_price,\n maxSimpleState(price) AS max_price,\n countState(price) AS volume,\n avgState(price) AS avg_price\nFROM default.uk_price_paid\nGROUP BY month +CREATE DICTIONARY default.uk_mortgage_rates_dict\n(\n `date` DateTime64,\n `variable` Decimal32(2),\n `fixed` Decimal32(2),\n `bank` Decimal32(2)\n)\nPRIMARY KEY date\nSOURCE(CLICKHOUSE(TABLE \'uk_mortgage_rates\'))\nLIFETIME(MIN 0 MAX 2628000000)\nLAYOUT(COMPLEX_KEY_HASHED()) diff --git a/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql new file mode 100644 index 00000000000..c500dd4e4c6 --- /dev/null +++ b/tests/queries/0_stateless/03230_output_format_identifier_quoting_style.sql @@ -0,0 +1,328 @@ +DROP DICTIONARY IF EXISTS uk_mortgage_rates_dict; +DROP TABLE IF EXISTS uk_mortgage_rates; +DROP VIEW IF EXISTS uk_prices_aggs_view; +DROP TABLE IF EXISTS uk_prices_aggs_dest; +DROP VIEW IF EXISTS prices_by_year_view; +DROP TABLE IF EXISTS prices_by_year_dest; +DROP TABLE IF EXISTS uk_price_paid; + +-- Create tables, views, dictionaries + +CREATE TABLE uk_price_paid +( + price UInt32, + date Date, + postcode1 LowCardinality(String), + postcode2 LowCardinality(String), + type Enum('terraced' = 1, 'semi-detached' = 2, 'detached' = 3, 'flat' = 4, 'other' = 0), + is_new UInt8, + duration Enum('freehold' = 1, 'leasehold' = 2, 'unknown' = 0), + addr1 String, + addr2 String, + street LowCardinality(String), + locality LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String), + INDEX county_index county TYPE set(10) GRANULARITY 1, + PROJECTION town_date_projection + ( + SELECT + town, + date, + price + ORDER BY + town, + date + ), + PROJECTION handy_aggs_projection + ( + SELECT + avg(price), + max(price), + sum(price) + GROUP BY town + ) +) +ENGINE = MergeTree +ORDER BY (postcode1, postcode2, date); + +CREATE TABLE prices_by_year_dest ( + price UInt32, + date Date, + addr1 String, + addr2 String, + street LowCardinality(String), + town LowCardinality(String), + district LowCardinality(String), + county LowCardinality(String) +) +ENGINE = MergeTree +PRIMARY KEY (town, date) +PARTITION BY toYear(date); + +CREATE MATERIALIZED VIEW prices_by_year_view +TO prices_by_year_dest +AS + SELECT + price, + date, + addr1, + addr2, + street, + town, + district, + county + FROM uk_price_paid; + +CREATE TABLE uk_prices_aggs_dest ( + month Date, + min_price SimpleAggregateFunction(min, UInt32), + max_price SimpleAggregateFunction(max, UInt32), + volume AggregateFunction(count, UInt32), + avg_price AggregateFunction(avg, UInt32) +) +ENGINE = AggregatingMergeTree +PRIMARY KEY month; + +CREATE MATERIALIZED VIEW uk_prices_aggs_view +TO uk_prices_aggs_dest +AS + WITH + toStartOfMonth(date) AS month + SELECT + month, + minSimpleState(price) AS min_price, + maxSimpleState(price) AS max_price, + countState(price) AS volume, + avgState(price) AS avg_price + FROM uk_price_paid + GROUP BY month; + +CREATE TABLE uk_mortgage_rates ( + date DateTime64, + variable Decimal32(2), + fixed Decimal32(2), + bank Decimal32(2) +) +ENGINE Memory(); + +INSERT INTO uk_mortgage_rates VALUES ('2004-02-29', 5.02, 4.9, 4); +INSERT INTO uk_mortgage_rates VALUES ('2004-03-31', 5.11, 4.91, 4); + +CREATE DICTIONARY uk_mortgage_rates_dict ( + date DateTime64, + variable Decimal32(2), + fixed Decimal32(2), + bank Decimal32(2) +) +PRIMARY KEY date +SOURCE( + CLICKHOUSE(TABLE 'uk_mortgage_rates') +) +LAYOUT(COMPLEX_KEY_HASHED()) +LIFETIME(2628000000); + + +-- Show tables, views, dictionaries with default settings +SHOW CREATE TABLE uk_price_paid; + +SHOW CREATE VIEW prices_by_year_view; + +SHOW CREATE uk_prices_aggs_dest; + +SHOW CREATE VIEW uk_prices_aggs_view; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict; + + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='None' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='None'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='Backticks' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='Backticks'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='Backticks' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='Backticks'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='DoubleQuotes' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='DoubleQuotes'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='DoubleQuotes' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='DoubleQuotes'; + + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=true, output_format_identifier_quoting_style='BackticksMySQL' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=true, + output_format_identifier_quoting_style='BackticksMySQL'; + +-- Show tables, views, dictionaries with output_format_always_quote_identifiers=false, output_format_identifier_quoting_style='BackticksMySQL' +SHOW CREATE TABLE uk_price_paid +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW prices_by_year_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE uk_prices_aggs_dest +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE VIEW uk_prices_aggs_view +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +SHOW CREATE DICTIONARY uk_mortgage_rates_dict +SETTINGS + output_format_always_quote_identifiers=false, + output_format_identifier_quoting_style='BackticksMySQL'; + +DROP DICTIONARY uk_mortgage_rates_dict; +DROP TABLE uk_mortgage_rates; +DROP VIEW uk_prices_aggs_view; +DROP TABLE uk_prices_aggs_dest; +DROP VIEW prices_by_year_view; +DROP TABLE prices_by_year_dest; +DROP TABLE uk_price_paid; From c44cc71310f982bd9d4101ee02001f5590971f17 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Aug 2024 11:32:05 +0000 Subject: [PATCH 063/154] forgot several files --- tests/integration/helpers/postgres_utility.py | 14 +++++++------- .../test.py | 8 ++++---- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 468c3b3bb63..4bb68284314 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -245,9 +245,9 @@ class PostgresManager: ): postgres_database = self.database_or_default(postgres_database) self.created_materialized_postgres_db_list.add(materialized_database) - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database}") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}`") - create_query = f"CREATE DATABASE {materialized_database} ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" + create_query = f"CREATE DATABASE `{materialized_database}` ENGINE = MaterializedPostgreSQL('{ip}:{port}', '{postgres_database}', '{user}', '{password}')" if len(settings) > 0: create_query += " SETTINGS " for i in range(len(settings)): @@ -259,7 +259,7 @@ class PostgresManager: assert materialized_database in self.instance.query("SHOW DATABASES") def drop_materialized_db(self, materialized_database="test_database"): - self.instance.query(f"DROP DATABASE IF EXISTS {materialized_database} SYNC") + self.instance.query(f"DROP DATABASE IF EXISTS `{materialized_database}` SYNC") if materialized_database in self.created_materialized_postgres_db_list: self.created_materialized_postgres_db_list.remove(materialized_database) @@ -329,11 +329,11 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM {materialized_database}") + database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") assert table in database_tables @@ -366,9 +366,9 @@ def check_tables_are_synchronized( table_path = "" if len(schema_name) == 0: - table_path = f"{materialized_database}.{table_name}" + table_path = f"`{materialized_database}`.`{table_name}`" else: - table_path = f"{materialized_database}.`{schema_name}.{table_name}`" + table_path = f"`{materialized_database}`.`{schema_name}.{table_name}`" print(f"Checking table is synchronized: {table_path}") result_query = f"select * from {table_path} order by {order_by};" diff --git a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py index 5599786026f..280985b55b3 100644 --- a/tests/integration/test_async_insert_adaptive_busy_timeout/test.py +++ b/tests/integration/test_async_insert_adaptive_busy_timeout/test.py @@ -356,7 +356,7 @@ def test_change_queries_frequency(): max_values_size=1000, array_size_range=[10, 50], ) - + node.query("SYSTEM FLUSH LOGS") select_log_query = f"SELECT countIf(timeout_milliseconds - {min_ms} < 25) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) assert int(res) >= 5 @@ -364,12 +364,12 @@ def test_change_queries_frequency(): _insert_queries_in_parallel( table_name, settings, - thread_num=20, - tasks=2000, + thread_num=10, + tasks=1000, max_values_size=1000, array_size_range=[10, 15], ) - + node.query("SYSTEM FLUSH LOGS") select_log_query = f"SELECT countIf({max_ms} - timeout_milliseconds < 100) FROM (SELECT timeout_milliseconds FROM system.asynchronous_insert_log ORDER BY event_time DESC LIMIT 10)" res = node.query(select_log_query) assert int(res) >= 5 From c1a83f47341032ae90f43ba84a54c0ec0e60810c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Tue, 27 Aug 2024 13:44:17 +0200 Subject: [PATCH 064/154] Fix possible wrong result during anyHeavy state merge --- src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp | 3 +++ tests/queries/0_stateless/03230_anyHeavy_merge.reference | 1 + tests/queries/0_stateless/03230_anyHeavy_merge.sql | 4 ++++ 3 files changed, 8 insertions(+) create mode 100644 tests/queries/0_stateless/03230_anyHeavy_merge.reference create mode 100644 tests/queries/0_stateless/03230_anyHeavy_merge.sql diff --git a/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp b/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp index ffddd46f2e3..dbc5f9be72f 100644 --- a/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp +++ b/src/AggregateFunctions/AggregateFunctionAnyHeavy.cpp @@ -68,7 +68,10 @@ public: if (data().isEqualTo(to.data())) counter += to.counter; else if (!data().has() || counter < to.counter) + { data().set(to.data(), arena); + counter = to.counter - counter; + } else counter -= to.counter; } diff --git a/tests/queries/0_stateless/03230_anyHeavy_merge.reference b/tests/queries/0_stateless/03230_anyHeavy_merge.reference new file mode 100644 index 00000000000..78981922613 --- /dev/null +++ b/tests/queries/0_stateless/03230_anyHeavy_merge.reference @@ -0,0 +1 @@ +a diff --git a/tests/queries/0_stateless/03230_anyHeavy_merge.sql b/tests/queries/0_stateless/03230_anyHeavy_merge.sql new file mode 100644 index 00000000000..5d4c0e55d0f --- /dev/null +++ b/tests/queries/0_stateless/03230_anyHeavy_merge.sql @@ -0,0 +1,4 @@ +DROP TABLE IF EXISTS t; +CREATE TABLE t (letter String) ENGINE=MergeTree order by () partition by letter; +INSERT INTO t VALUES ('a'), ('a'), ('a'), ('a'), ('b'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('a'), ('c'); +SELECT anyHeavy(if(letter != 'b', letter, NULL)) FROM t; From cc701c92b4d870a4a2028cccbc46c2cfda8c405d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Aug 2024 15:05:05 +0200 Subject: [PATCH 065/154] Add plan_step_name and plan_step_description columns to system.processors_profile_log --- src/Interpreters/ProcessorsProfileLog.cpp | 4 ++++ src/Interpreters/ProcessorsProfileLog.h | 2 ++ src/Processors/IProcessor.cpp | 12 ++++++++++++ src/Processors/IProcessor.h | 10 +++++----- 4 files changed, 23 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ProcessorsProfileLog.cpp b/src/Interpreters/ProcessorsProfileLog.cpp index 7dec2a3163a..8a646b5d0e7 100644 --- a/src/Interpreters/ProcessorsProfileLog.cpp +++ b/src/Interpreters/ProcessorsProfileLog.cpp @@ -30,6 +30,8 @@ ColumnsDescription ProcessorProfileLogElement::getColumnsDescription() {"id", std::make_shared(), "ID of processor."}, {"parent_ids", std::make_shared(std::make_shared()), "Parent processors IDs."}, {"plan_step", std::make_shared(), "ID of the query plan step which created this processor. The value is zero if the processor was not added from any step."}, + {"plan_step_name", std::make_shared(), "Name of the query plan step which created this processor. The value is empty if the processor was not added from any step."}, + {"plan_step_description", std::make_shared(), "Description of the query plan step which created this processor. The value is empty if the processor was not added from any step."}, {"plan_group", std::make_shared(), "Group of the processor if it was created by query plan step. A group is a logical partitioning of processors added from the same query plan step. Group is used only for beautifying the result of EXPLAIN PIPELINE result."}, {"initial_query_id", std::make_shared(), "ID of the initial query (for distributed query execution)."}, @@ -64,6 +66,8 @@ void ProcessorProfileLogElement::appendToBlock(MutableColumns & columns) const } columns[i++]->insert(plan_step); + columns[i++]->insert(plan_step_name); + columns[i++]->insert(plan_step_description); columns[i++]->insert(plan_group); columns[i++]->insertData(initial_query_id.data(), initial_query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size()); diff --git a/src/Interpreters/ProcessorsProfileLog.h b/src/Interpreters/ProcessorsProfileLog.h index abece2604f2..fbf52f45f56 100644 --- a/src/Interpreters/ProcessorsProfileLog.h +++ b/src/Interpreters/ProcessorsProfileLog.h @@ -19,6 +19,8 @@ struct ProcessorProfileLogElement UInt64 plan_step{}; UInt64 plan_group{}; + String plan_step_name; + String plan_step_description; String initial_query_id; String query_id; diff --git a/src/Processors/IProcessor.cpp b/src/Processors/IProcessor.cpp index fc595a7b565..d9bd5074c09 100644 --- a/src/Processors/IProcessor.cpp +++ b/src/Processors/IProcessor.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -9,6 +10,17 @@ namespace DB { +void IProcessor::setQueryPlanStep(IQueryPlanStep * step, size_t group) +{ + query_plan_step = step; + query_plan_step_group = group; + if (step) + { + plan_step_name = step->getName(); + plan_step_description = step->getStepDescription(); + } +} + void IProcessor::cancel() noexcept { diff --git a/src/Processors/IProcessor.h b/src/Processors/IProcessor.h index 02b8a3daa28..fd75eb530aa 100644 --- a/src/Processors/IProcessor.h +++ b/src/Processors/IProcessor.h @@ -311,14 +311,12 @@ public: constexpr static size_t NO_STREAM = std::numeric_limits::max(); /// Step of QueryPlan from which processor was created. - void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0) - { - query_plan_step = step; - query_plan_step_group = group; - } + void setQueryPlanStep(IQueryPlanStep * step, size_t group = 0); IQueryPlanStep * getQueryPlanStep() const { return query_plan_step; } size_t getQueryPlanStepGroup() const { return query_plan_step_group; } + const String & getPlanStepName() const { return plan_step_name; } + const String & getPlanStepDescription() const { return plan_step_description; } uint64_t getElapsedNs() const { return elapsed_ns; } uint64_t getInputWaitElapsedNs() const { return input_wait_elapsed_ns; } @@ -410,6 +408,8 @@ private: IQueryPlanStep * query_plan_step = nullptr; size_t query_plan_step_group = 0; + String plan_step_name; + String plan_step_description; }; From 2a68ad8704d62b179a123a8fe3355d58450f2b99 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 27 Aug 2024 15:12:01 +0200 Subject: [PATCH 066/154] Fill plan_step_name and plan_step_description --- src/Interpreters/executeQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index decc16a3704..d8ceae77d13 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -471,6 +471,8 @@ void logQueryFinish( processor_elem.parent_ids = std::move(parents); processor_elem.plan_step = reinterpret_cast(processor->getQueryPlanStep()); + processor_elem.plan_step_name = processor->getPlanStepName(); + processor_elem.plan_step_description = processor->getPlanStepDescription(); processor_elem.plan_group = processor->getQueryPlanStepGroup(); processor_elem.processor_name = processor->getName(); From 9ae0d5161349bd89a9a185a8f60aaab0e9357f34 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Tue, 27 Aug 2024 15:02:02 +0000 Subject: [PATCH 067/154] Reformat the code, pass variable names in comments when calling --- src/Core/ExternalTable.cpp | 17 ++++++++++++++--- 1 file changed, 14 insertions(+), 3 deletions(-) diff --git a/src/Core/ExternalTable.cpp b/src/Core/ExternalTable.cpp index 1f8e63db72e..7fe23f844b6 100644 --- a/src/Core/ExternalTable.cpp +++ b/src/Core/ExternalTable.cpp @@ -18,11 +18,11 @@ #include #include #include +#include #include #include #include #include -#include "Parsers/IdentifierQuotingStyle.h" namespace DB @@ -88,7 +88,13 @@ void BaseExternalTable::parseStructureFromStructureField(const std::string & arg if (column) structure.emplace_back( column->name, - column->type->formatWithPossiblyHidingSensitiveData(0, true, true, false, false, IdentifierQuotingStyle::Backticks)); + column->type->formatWithPossiblyHidingSensitiveData( + /*max_length=*/0, + /*one_line=*/true, + /*show_secrets=*/true, + /*print_pretty_type_names=*/false, + /*always_quote_identifiers=*/false, + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks)); else throw Exception(ErrorCodes::BAD_ARGUMENTS, "Error while parsing table structure: expected column definition, got {}", child->formatForErrorMessage()); } @@ -108,7 +114,12 @@ void BaseExternalTable::parseStructureFromTypesField(const std::string & argumen structure.emplace_back( "_" + toString(i + 1), type_list_raw->children[i]->formatWithPossiblyHidingSensitiveData( - 0, true, true, false, false, IdentifierQuotingStyle::Backticks)); + /*max_length=*/0, + /*one_line=*/true, + /*show_secrets=*/true, + /*print_pretty_type_names=*/false, + /*always_quote_identifiers=*/false, + /*identifier_quoting_style=*/IdentifierQuotingStyle::Backticks)); } void BaseExternalTable::initSampleBlock() From aed953bf943eb57feef7ae5a0f2ebf9336543f88 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 27 Aug 2024 16:00:33 +0000 Subject: [PATCH 068/154] black --- tests/integration/helpers/postgres_utility.py | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/integration/helpers/postgres_utility.py b/tests/integration/helpers/postgres_utility.py index 4bb68284314..c61c535bd62 100644 --- a/tests/integration/helpers/postgres_utility.py +++ b/tests/integration/helpers/postgres_utility.py @@ -329,11 +329,15 @@ def assert_nested_table_is_created( table = schema_name + "." + table_name print(f"Checking table {table} exists in {materialized_database}") - database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") + database_tables = instance.query( + f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" + ) while table not in database_tables: time.sleep(0.2) - database_tables = instance.query(f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'") + database_tables = instance.query( + f"SHOW TABLES FROM `{materialized_database}` WHERE name = '{table}'" + ) assert table in database_tables From e0b9c0f14f6a7e817fddc6eedc5f6bd69b6d9f83 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 28 Aug 2024 03:47:15 +0000 Subject: [PATCH 069/154] Remove settings update in ASTTableOverride::formatImpl --- src/Parsers/ASTTableOverrides.cpp | 4 +- src/Parsers/tests/gtest_Parser.cpp | 98 +++++++++++++++--------------- 2 files changed, 50 insertions(+), 52 deletions(-) diff --git a/src/Parsers/ASTTableOverrides.cpp b/src/Parsers/ASTTableOverrides.cpp index ccb485f6c69..8352e68b156 100644 --- a/src/Parsers/ASTTableOverrides.cpp +++ b/src/Parsers/ASTTableOverrides.cpp @@ -22,10 +22,8 @@ ASTPtr ASTTableOverride::clone() const return res; } -void ASTTableOverride::formatImpl(const FormatSettings & settings_, FormatState & state, FormatStateStacked frame) const +void ASTTableOverride::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const { - FormatSettings settings = settings_; - settings.always_quote_identifiers = true; String nl_or_nothing = settings.one_line ? "" : "\n"; String nl_or_ws = settings.one_line ? " " : "\n"; String hl_keyword = settings.hilite ? hilite_keyword : ""; diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 98cd9682c9c..1b50d4480d7 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -225,55 +225,55 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", - "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE `tbl`\n(\n\n),\nTABLE OVERRIDE `a`\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE `b`\n(\n PARTITION BY rand()\n)" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n INDEX `foo` `foo` TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(`_staged` = 1, 'staging', toYYYYMM(`created`))\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", - "CREATE DATABASE db\nTABLE OVERRIDE `t1`\n(\n TTL `inserted` + toIntervalMonth(1)\n),\nTABLE OVERRIDE `t2`\n(\n TTL `inserted` + toIntervalMonth(2)\n)" - }, - { - "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE `tab3`\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n),\nTABLE OVERRIDE `tab5`\n(\n PARTITION BY `c3`\n TTL `c3` + toIntervalMinute(10)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", - "CREATE DATABASE db\nTABLE OVERRIDE `tbl`\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(`created`)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE `a`\n(\n ORDER BY (`id`, `version`)\n)\nCOMMENT 'db comment'" - } -}))); + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", + "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", + "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" + }, + { + "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" + } + }))); INSTANTIATE_TEST_SUITE_P(ParserCreateUserQuery, ParserTest, ::testing::Combine( From 2c6563ee691a7c1a0cee61af815ba037393957d2 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Wed, 28 Aug 2024 12:37:28 +0000 Subject: [PATCH 070/154] Fix test in tests/integration/test_materialized_mysql_database/materialized_with_ddl.py --- .../test_materialized_mysql_database/materialized_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py index 86000799ae4..9a99f0c9aa8 100644 --- a/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialized_with_ddl.py @@ -2353,7 +2353,7 @@ def table_overrides(clickhouse_node, mysql_node, service_name): ) check_query(clickhouse_node, "SELECT count() FROM table_overrides.t1", "1001\n") show_db = clickhouse_node.query("SHOW CREATE DATABASE table_overrides") - assert "TABLE OVERRIDE `t1`\\n(\\n\\n)" in show_db, show_db + assert "TABLE OVERRIDE t1\\n(\\n\\n)" in show_db, show_db clickhouse_node.query("DROP DATABASE IF EXISTS table_overrides") mysql_node.query("DROP DATABASE IF EXISTS table_overrides") From 7f613563e1a9a11a2c035488aca058c396c9e05a Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 28 Aug 2024 13:39:29 +0000 Subject: [PATCH 071/154] initial impl for cp/mv --- programs/keeper-client/Commands.cpp | 108 ++++++++++++++++++++++++ programs/keeper-client/Commands.h | 28 ++++++ programs/keeper-client/KeeperClient.cpp | 2 + 3 files changed, 138 insertions(+) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 7226bd82df7..b88791ad1d6 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -677,4 +677,112 @@ void GetAllChildrenNumberCommand::execute(const ASTKeeperQuery * query, KeeperCl std::cout << totalNumChildren << "\n"; } +namespace +{ + +class CPMVOperation +{ +public: + CPMVOperation(String src_, String dest_, bool remove_src_, KeeperClient * client_) + : src(std::move(src_)), dest(std::move(dest_)), remove_src(remove_src_), client(client_) + { + } + + bool perform() + { + Coordination::Stat src_stat; + String data = client->zookeeper->get(src, &src_stat); + + // allow to copy only persistent nodes + if (src_stat.ephemeralOwner) + throw std::runtime_error("TODO: it is possible to copy only persistent nodes"); + + Coordination::Requests ops{ + zkutil::makeCheckRequest(src, src_stat.version), + zkutil::makeCreateRequest(dest, data, zkutil::CreateMode::Persistent), // Do we need to copy ACLs here? + }; + + if (remove_src) + ops.push_back(zkutil::makeRemoveRequest(src, src_stat.version)); + + Coordination::Responses responses; + auto code = client->zookeeper->tryMultiNoThrow(ops, responses); + + switch (code) + { + case Coordination::Error::ZOK: + return true; + case Coordination::Error::ZBADVERSION: + return false; + case Coordination::Error::ZNODEEXISTS: + throw std::runtime_error("TODO: Destination path already exists"); + default: + zkutil::KeeperMultiException::check(code, ops, responses); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unreachable"); + } + +private: + String src; + String dest; + bool remove_src = false; + KeeperClient * client = nullptr; +}; + +} + +bool CPCommand::parse(IParser::Pos & pos, std::shared_ptr & node, [[maybe_unused]] Expected & expected) const +{ + String src_path; + if (!parseKeeperPath(pos, expected, src_path)) + return false; + node->args.push_back(std::move(src_path)); + + String to_path; + if (!parseKeeperPath(pos, expected, to_path)) + return false; + node->args.push_back(std::move(to_path)); + + return true; +} + +void CPCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto src = client->getAbsolutePath(query->args[0].safeGet()); + auto dest = client->getAbsolutePath(query->args[1].safeGet()); + + CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/false, /*client_=*/client); + + while (!operation.perform()) + ; +} + + +bool MVCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const +{ + String src_path; + if (!parseKeeperPath(pos, expected, src_path)) + return false; + node->args.push_back(std::move(src_path)); + + String to_path; + if (!parseKeeperPath(pos, expected, to_path)) + return false; + node->args.push_back(std::move(to_path)); + + return true; +} + +void MVCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) const +{ + auto src = client->getAbsolutePath(query->args[0].safeGet()); + auto dest = client->getAbsolutePath(query->args[1].safeGet()); + + CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/true, /*client_=*/client); + + while (!operation.perform()) + ; +} + } diff --git a/programs/keeper-client/Commands.h b/programs/keeper-client/Commands.h index c6dd731fb3b..686a752b6b6 100644 --- a/programs/keeper-client/Commands.h +++ b/programs/keeper-client/Commands.h @@ -266,4 +266,32 @@ class GetAllChildrenNumberCommand : public IKeeperClientCommand } }; +class CPCommand : public IKeeperClientCommand +{ + String getName() const override { return "cp"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} -- Copies 'src' node to 'dest' path."; + } +}; + +class MVCommand : public IKeeperClientCommand +{ + String getName() const override { return "mv"; } + + bool parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const override; + + void execute(const ASTKeeperQuery * query, KeeperClient * client) const override; + + String getHelpMessage() const override + { + return "{} -- Moves 'src' node to the 'dest' path."; + } +}; + } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index ad376d4b88f..97caa142124 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -212,6 +212,8 @@ void KeeperClient::initialize(Poco::Util::Application & /* self */) std::make_shared(), std::make_shared(), std::make_shared(), + std::make_shared(), + std::make_shared(), }); String home_path; From 644cfb27d66355652afec7762402a0241f283156 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 15:34:34 +0100 Subject: [PATCH 072/154] Add RealTimeMicroseconds to HTTP Header X-ClickHouse-Summary --- src/IO/Progress.cpp | 13 +++++++++++++ src/IO/Progress.h | 6 ++++++ src/Interpreters/executeQuery.cpp | 3 +++ 3 files changed, 22 insertions(+) diff --git a/src/IO/Progress.cpp b/src/IO/Progress.cpp index c5bcd0c490a..179c7f7f807 100644 --- a/src/IO/Progress.cpp +++ b/src/IO/Progress.cpp @@ -91,6 +91,8 @@ void ProgressValues::writeJSON(WriteBuffer & out) const writeText(result_bytes, out); writeCString("\",\"elapsed_ns\":\"", out); writeText(elapsed_ns, out); + writeCString("\",\"real_time_microseconds\":\"", out); + writeText(real_time_microseconds, out); writeCString("\"", out); writeCString("}", out); } @@ -110,6 +112,7 @@ bool Progress::incrementPiecewiseAtomically(const Progress & rhs) result_bytes += rhs.result_bytes; elapsed_ns += rhs.elapsed_ns; + real_time_microseconds += rhs.real_time_microseconds; return rhs.read_rows || rhs.written_rows; } @@ -129,6 +132,7 @@ void Progress::reset() result_bytes = 0; elapsed_ns = 0; + real_time_microseconds = 0; } ProgressValues Progress::getValues() const @@ -148,6 +152,7 @@ ProgressValues Progress::getValues() const res.result_bytes = result_bytes.load(std::memory_order_relaxed); res.elapsed_ns = elapsed_ns.load(std::memory_order_relaxed); + res.real_time_microseconds = real_time_microseconds.load(std::memory_order_relaxed); return res; } @@ -169,6 +174,7 @@ ProgressValues Progress::fetchValuesAndResetPiecewiseAtomically() res.result_bytes = result_bytes.fetch_and(0); res.elapsed_ns = elapsed_ns.fetch_and(0); + res.real_time_microseconds = real_time_microseconds.fetch_and(0); return res; } @@ -190,6 +196,7 @@ Progress Progress::fetchAndResetPiecewiseAtomically() res.result_bytes = result_bytes.fetch_and(0); res.elapsed_ns = elapsed_ns.fetch_and(0); + res.real_time_microseconds = real_time_microseconds.fetch_and(0); return res; } @@ -209,6 +216,7 @@ Progress & Progress::operator=(Progress && other) noexcept result_bytes = other.result_bytes.load(std::memory_order_relaxed); elapsed_ns = other.elapsed_ns.load(std::memory_order_relaxed); + real_time_microseconds = other.real_time_microseconds.load(std::memory_order_relaxed); return *this; } @@ -244,4 +252,9 @@ void Progress::incrementElapsedNs(UInt64 elapsed_ns_) elapsed_ns.fetch_add(elapsed_ns_, std::memory_order_relaxed); } +void Progress::incrementRealTimeMicroseconds(UInt64 microseconds) +{ + real_time_microseconds.fetch_add(microseconds, std::memory_order_relaxed); +} + } diff --git a/src/IO/Progress.h b/src/IO/Progress.h index d0afc9d845f..223496166ec 100644 --- a/src/IO/Progress.h +++ b/src/IO/Progress.h @@ -28,6 +28,7 @@ struct ProgressValues UInt64 result_bytes = 0; UInt64 elapsed_ns = 0; + UInt64 real_time_microseconds = 0; void read(ReadBuffer & in, UInt64 server_revision); void write(WriteBuffer & out, UInt64 client_revision) const; @@ -40,6 +41,7 @@ struct ReadProgress UInt64 read_bytes = 0; UInt64 total_rows_to_read = 0; UInt64 total_bytes_to_read = 0; + UInt64 real_time_microseconds = 0; ReadProgress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0, UInt64 total_bytes_to_read_ = 0) : read_rows(read_rows_), read_bytes(read_bytes_), total_rows_to_read(total_rows_to_read_), total_bytes_to_read(total_bytes_to_read_) {} @@ -96,6 +98,8 @@ struct Progress std::atomic elapsed_ns {0}; + std::atomic real_time_microseconds {0}; + Progress() = default; Progress(UInt64 read_rows_, UInt64 read_bytes_, UInt64 total_rows_to_read_ = 0, UInt64 total_bytes_to_read_ = 0) @@ -125,6 +129,8 @@ struct Progress void incrementElapsedNs(UInt64 elapsed_ns_); + void incrementRealTimeMicroseconds(UInt64 microseconds); + void reset(); ProgressValues getValues() const; diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index decc16a3704..fd61cd7f86f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -419,6 +419,9 @@ void logQueryFinish( { Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); + + UInt64 cpu_real_time = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; + p.incrementRealTimeMicroseconds(cpu_real_time); progress_callback(p); } From a1c9cc471d5b8afbc21a4aa6f8d44791c896eb10 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 28 Aug 2024 16:51:45 +0200 Subject: [PATCH 073/154] Apply suggestions from code review MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Co-authored-by: János Benjamin Antal --- .../AggregateFunctionDistinctJSONPaths.cpp | 20 +++++++++---------- src/Columns/ColumnDynamic.cpp | 2 +- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 6100bd57515..98996aac2f7 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -49,8 +49,8 @@ struct AggregateFunctionDistinctJSONPathsData /// Iterate over paths in shared data in this row. const auto [shared_data_paths, _] = column.getSharedDataPathsAndValues(); const auto & shared_data_offsets = column.getSharedDataOffsets(); - size_t start = shared_data_offsets[static_cast(row_num) - 1]; - size_t end = shared_data_offsets[static_cast(row_num)]; + const size_t start = shared_data_offsets[static_cast(row_num) - 1]; + const size_t end = shared_data_offsets[static_cast(row_num)]; for (size_t i = start; i != end; ++i) data.insert(shared_data_paths->getDataAt(i).toString()); } @@ -137,8 +137,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData /// Iterate over paths om shared data in this row and decode the data types. const auto [shared_data_paths, shared_data_values] = column.getSharedDataPathsAndValues(); const auto & shared_data_offsets = column.getSharedDataOffsets(); - size_t start = shared_data_offsets[static_cast(row_num) - 1]; - size_t end = shared_data_offsets[static_cast(row_num)]; + const size_t start = shared_data_offsets[static_cast(row_num) - 1]; + const size_t end = shared_data_offsets[static_cast(row_num)]; for (size_t i = start; i != end; ++i) { auto path = shared_data_paths->getDataAt(i).toString(); @@ -146,8 +146,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - if (!isNothing(type)) - data[path].insert(type->getName()); + chassert(!isNothingType(type)); + data[path].insert(type->getName()); } } @@ -172,8 +172,8 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - if (!isNothing(type)) - data[path].insert(type->getName()); + chassert(!isNothingType(type)); + data[path].insert(type->getName()); } } @@ -200,7 +200,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData size_t paths_size, types_size; readVarUInt(paths_size, buf); if (paths_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size for paths (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, paths_size); data.reserve(paths_size); String path, type; @@ -209,7 +209,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData readStringBinary(path, buf); readVarUInt(types_size, buf); if (types_size > DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE) - throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); + throw Exception(ErrorCodes::TOO_LARGE_ARRAY_SIZE, "Too large array size for types (maximum: {}): {}", DISTINCT_JSON_PATHS_MAX_ARRAY_SIZE, types_size); data[path].reserve(types_size); for (size_t j = 0; j != types_size; ++j) diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index b5378e983c6..eb85c6b0d08 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -983,7 +983,7 @@ ColumnPtr ColumnDynamic::compress() const String ColumnDynamic::getTypeNameAt(size_t row_num) const { const auto & variant_col = getVariantColumn(); - size_t discr = variant_col.globalDiscriminatorAt(row_num); + const size_t discr = variant_col.globalDiscriminatorAt(row_num); if (discr == ColumnVariant::NULL_DISCRIMINATOR) return ""; From 0aba986372504cbdc6eb28283f51a706ffdb7b2e Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 28 Aug 2024 15:06:05 +0000 Subject: [PATCH 074/154] Address review comments --- .../reference/distinctjsonpaths.md | 41 +++++++++++++++++++ .../AggregateFunctionDistinctDynamicTypes.cpp | 2 +- .../AggregateFunctionDistinctJSONPaths.cpp | 8 ++-- src/Columns/ColumnDynamic.cpp | 2 +- src/Columns/ColumnDynamic.h | 2 +- 5 files changed, 48 insertions(+), 7 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md index f916734ca44..d88b2eb024b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md +++ b/docs/en/sql-reference/aggregate-functions/reference/distinctjsonpaths.md @@ -82,3 +82,44 @@ Result: │ {'a':['Int64'],'b':['Array(Nullable(Int64))','String'],'c.d.e':['Date'],'c.d.f':['Array(JSON(max_dynamic_types=16, max_dynamic_paths=256))']} │ └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ ``` + +**Note** + +If JSON declaration contains paths with specified types, these paths will be always included in the result of `distinctJSONPaths/distinctJSONPathsAndTypes` functions even if input data didn't have values for these paths. + +```sql +DROP TABLE IF EXISTS test_json; +CREATE TABLE test_json(json JSON(a UInt32)) ENGINE = Memory; +INSERT INTO test_json VALUES ('{"b" : "Hello"}'), ('{"b" : "World", "c" : [1, 2, 3]}'); +``` + +```sql +SELECT json FROM test_json; +``` + +```text +┌─json──────────────────────────────────┐ +│ {"a":0,"b":"Hello"} │ +│ {"a":0,"b":"World","c":["1","2","3"]} │ +└───────────────────────────────────────┘ +``` + +```sql +SELECT distinctJSONPaths(json) FROM test_json; +``` + +```text +┌─distinctJSONPaths(json)─┐ +│ ['a','b','c'] │ +└─────────────────────────┘ +``` + +```sql +SELECT distinctJSONPathsAndTypes(json) FROM test_json; +``` + +```text +┌─distinctJSONPathsAndTypes(json)────────────────────────────────┐ +│ {'a':['UInt32'],'b':['String'],'c':['Array(Nullable(Int64))']} │ +└────────────────────────────────────────────────────────────────┘ +``` \ No newline at end of file diff --git a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp index 17e32b20a99..57f7aecd316 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctDynamicTypes.cpp @@ -106,7 +106,7 @@ public: /// In this case we can avoid iterating over all rows because we can get all types /// in Dynamic column in a more efficient way. else - assert_cast(*columns[0]).getAllTypeNames(data(place).data); + assert_cast(*columns[0]).getAllTypeNamesInto(data(place).data); } void addManyDefaults( diff --git a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp index 98996aac2f7..4e60e6fe60b 100644 --- a/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp +++ b/src/AggregateFunctions/AggregateFunctionDistinctJSONPaths.cpp @@ -146,7 +146,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - chassert(!isNothingType(type)); + chassert(!isNothing(type)); data[path].insert(type->getName()); } } @@ -160,7 +160,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData /// Add dynamic path only if it has at least one non-null value. /// getNumberOfDefaultRows for Dynamic column is O(1). if (dynamic_column->getNumberOfDefaultRows() != dynamic_column->size()) - dynamic_column->getAllTypeNames(data[path]); + dynamic_column->getAllTypeNamesInto(data[path]); } /// Iterate over all paths in shared data and decode the data types. @@ -172,7 +172,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData ReadBufferFromMemory buf(value.data, value.size); auto type = decodeDataType(buf); /// We should not have Nulls here but let's check just in case. - chassert(!isNothingType(type)); + chassert(!isNothing(type)); data[path].insert(type->getName()); } } @@ -255,7 +255,7 @@ struct AggregateFunctionDistinctJSONPathsAndTypesData } }; -/// Calculates the list of distinct data types in Dynamic column. +/// Calculates the list of distinct paths or pairs (path, type) in JSON column. template class AggregateFunctionDistinctJSONPathsAndTypes final : public IAggregateFunctionDataHelper> { diff --git a/src/Columns/ColumnDynamic.cpp b/src/Columns/ColumnDynamic.cpp index eb85c6b0d08..269c8455e2f 100644 --- a/src/Columns/ColumnDynamic.cpp +++ b/src/Columns/ColumnDynamic.cpp @@ -997,7 +997,7 @@ String ColumnDynamic::getTypeNameAt(size_t row_num) const return variant_info.variant_names[discr]; } -void ColumnDynamic::getAllTypeNames(std::unordered_set & names) const +void ColumnDynamic::getAllTypeNamesInto(std::unordered_set & names) const { auto shared_variant_discr = getSharedVariantDiscriminator(); for (size_t i = 0; i != variant_info.variant_names.size(); ++i) diff --git a/src/Columns/ColumnDynamic.h b/src/Columns/ColumnDynamic.h index c06c31bb8c9..5789b80a2de 100644 --- a/src/Columns/ColumnDynamic.h +++ b/src/Columns/ColumnDynamic.h @@ -431,7 +431,7 @@ public: const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); } String getTypeNameAt(size_t row_num) const; - void getAllTypeNames(std::unordered_set & names) const; + void getAllTypeNamesInto(std::unordered_set & names) const; private: void createVariantInfo(const DataTypePtr & variant_type); From 41a4dd77058df5c5a9e20af9a510610233dbd695 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 16:09:23 +0100 Subject: [PATCH 075/154] Rename variable --- src/Interpreters/executeQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index fd61cd7f86f..2a796004d10 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -420,8 +420,8 @@ void logQueryFinish( Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); - UInt64 cpu_real_time = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; - p.incrementRealTimeMicroseconds(cpu_real_time); + UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; + p.incrementRealTimeMicroseconds(real_time_microseconds); progress_callback(p); } From 4d486f1a74b265ec746bdb87d98b16d4c57e72e4 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Wed, 28 Aug 2024 14:53:40 +0000 Subject: [PATCH 076/154] update error policy --- programs/keeper-client/Commands.cpp | 7 ------- 1 file changed, 7 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index b88791ad1d6..2afd55e835e 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -693,10 +693,6 @@ public: Coordination::Stat src_stat; String data = client->zookeeper->get(src, &src_stat); - // allow to copy only persistent nodes - if (src_stat.ephemeralOwner) - throw std::runtime_error("TODO: it is possible to copy only persistent nodes"); - Coordination::Requests ops{ zkutil::makeCheckRequest(src, src_stat.version), zkutil::makeCreateRequest(dest, data, zkutil::CreateMode::Persistent), // Do we need to copy ACLs here? @@ -714,8 +710,6 @@ public: return true; case Coordination::Error::ZBADVERSION: return false; - case Coordination::Error::ZNODEEXISTS: - throw std::runtime_error("TODO: Destination path already exists"); default: zkutil::KeeperMultiException::check(code, ops, responses); } @@ -758,7 +752,6 @@ void CPCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con ; } - bool MVCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const { String src_path; From 6b08d2b6de2f22b5972a0b27e6f77ab98344a1da Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 16:51:56 +0100 Subject: [PATCH 077/154] Update docs --- docs/en/interfaces/http.md | 12 ++++++------ docs/ru/interfaces/http.md | 10 +++++----- docs/zh/interfaces/http.md | 12 ++++++------ 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index 03fdfa048c8..ffd65fce00f 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -58,7 +58,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds": "0"} 1 ``` @@ -472,7 +472,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -668,7 +668,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -708,7 +708,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact
% @@ -766,7 +766,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -785,7 +785,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index d9da51892f9..01d788e62cf 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -50,7 +50,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} 1 ``` @@ -367,7 +367,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0", "elapsed_ns":"662334", "real_time_microseconds":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -601,7 +601,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact
% @@ -659,7 +659,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -678,7 +678,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < Relative Path File * Connection #0 to host localhost left intact diff --git a/docs/zh/interfaces/http.md b/docs/zh/interfaces/http.md index f55cf41936f..4767a540d61 100644 --- a/docs/zh/interfaces/http.md +++ b/docs/zh/interfaces/http.md @@ -53,7 +53,7 @@ Connection: Close Content-Type: text/tab-separated-values; charset=UTF-8 X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f -X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} 1 ``` @@ -363,7 +363,7 @@ $ curl -v 'http://localhost:8123/predefined_query' < X-ClickHouse-Format: Template < X-ClickHouse-Timezone: Asia/Shanghai < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < # HELP "Query" "Number of executing queries" # TYPE "Query" counter @@ -524,7 +524,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/hi' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334", "real_time_microseconds":"0"} < * Connection #0 to host localhost left intact Say Hi!% @@ -564,7 +564,7 @@ $ curl -v -H 'XXX:xxx' 'http://localhost:8123/get_config_static_handler' < Content-Type: text/plain; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} < * Connection #0 to host localhost left intact
% @@ -616,7 +616,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_absolute_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} < Absolute Path File * Connection #0 to host localhost left intact @@ -635,7 +635,7 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler' < Content-Type: text/html; charset=UTF-8 < Transfer-Encoding: chunked < Keep-Alive: timeout=10 -< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334"} +< X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0","elapsed_ns":"662334","real_time_microseconds":"0"} < Relative Path File * Connection #0 to host localhost left intact From 82eae9f09f58959c2d9727dce66092132168de66 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 28 Aug 2024 16:18:57 +0000 Subject: [PATCH 078/154] tune --- tests/ci/integration_tests_runner.py | 2 +- tests/integration/test_async_load_databases/test.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/integration_tests_runner.py b/tests/ci/integration_tests_runner.py index a7d5a8c4cf6..96e1ddc9bff 100755 --- a/tests/ci/integration_tests_runner.py +++ b/tests/ci/integration_tests_runner.py @@ -33,7 +33,7 @@ CLICKHOUSE_BINARY_PATH = "usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-odbc-bridge" CLICKHOUSE_LIBRARY_BRIDGE_BINARY_PATH = "usr/bin/clickhouse-library-bridge" -FLAKY_TRIES_COUNT = 5 # run whole pytest several times +FLAKY_TRIES_COUNT = 3 # run whole pytest several times FLAKY_REPEAT_COUNT = 5 # runs test case in single module several times MAX_TIME_SECONDS = 3600 diff --git a/tests/integration/test_async_load_databases/test.py b/tests/integration/test_async_load_databases/test.py index f36cff76ea2..94aba46c713 100644 --- a/tests/integration/test_async_load_databases/test.py +++ b/tests/integration/test_async_load_databases/test.py @@ -182,4 +182,4 @@ def test_multiple_tables(started_cluster): for i in order: assert query(f"select count() from test.table_{i}") == "100\n" for i in range(tables_count): - query(f"drop table test.table_{i}") + query(f"drop table test.table_{i} sync") From 3f9f70d27d7d796ee8d4037a6498f23e812ef125 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 21:10:47 +0200 Subject: [PATCH 079/154] Revert "Fix strange trash" This reverts commit aee9612e5127f9cbe0a05c12a08480fc5072ff44. --- src/Core/SettingsFields.cpp | 2 +- src/Core/SettingsFields.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 278b1101c71..86e247722c8 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - return "auto(" + ::DB::toString(value) + ")"; + return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); } diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 533d69f3fbb..266141815e3 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -153,7 +153,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto == true`. + /// Writes "auto()" instead of simple "" if `is_auto==true`. String toString() const; void parseFromString(const String & str); From 0dacd16947742be39be53ec72460673438e2e984 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Tue, 27 Aug 2024 21:14:04 +0200 Subject: [PATCH 080/154] Update SettingsFields.h --- src/Core/SettingsFields.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 266141815e3..533d69f3fbb 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -153,7 +153,7 @@ struct SettingFieldMaxThreads operator UInt64() const { return value; } /// NOLINT explicit operator Field() const { return value; } - /// Writes "auto()" instead of simple "" if `is_auto==true`. + /// Writes "auto()" instead of simple "" if `is_auto == true`. String toString() const; void parseFromString(const String & str); From ab567bad1354edb199baa9c8dc478c1617d6521b Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 01:28:27 +0200 Subject: [PATCH 081/154] What if I just strip quotes --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 86e247722c8..ca84299ec5b 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -210,7 +210,7 @@ namespace { UInt64 stringToMaxThreads(const String & str) { - if (startsWith(str, "auto")) + if (startsWith(str, "auto") || startsWith(str, "'auto")) return 0; return parseFromString(str); } From 0c23257aa8220ebc8bd9c040088cfb3b26ff6a05 Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:09:59 +0200 Subject: [PATCH 082/154] Revert "Fix integration test" This reverts commit 0f3506bf7af20231a290ed97a59b0e233fbc7c6e. --- tests/integration/test_cgroup_limit/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_cgroup_limit/test.py b/tests/integration/test_cgroup_limit/test.py index 5d56135d9ff..e77b0f70960 100644 --- a/tests/integration/test_cgroup_limit/test.py +++ b/tests/integration/test_cgroup_limit/test.py @@ -46,7 +46,7 @@ def test_cgroup_cpu_limit(): "clickhouse local -q \"select value from system.settings where name='max_threads'\"", num_cpus, ) - expect_output = (r"auto({})".format(math.ceil(num_cpus))).encode() + expect_output = (r"\'auto({})\'".format(math.ceil(num_cpus))).encode() assert ( result.strip() == expect_output ), f"fail for cpu limit={num_cpus}, result={result.strip()}, expect={expect_output}" From 896b1726e756bbc4ff3dd9eb417b00e13263f04e Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 15:11:19 +0200 Subject: [PATCH 083/154] Add a comment --- src/Core/SettingsFields.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index ca84299ec5b..47baedfa9db 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,6 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) + /// Removing quotes this will introduce incompatibility between replicas with different version. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 0815a85e2bece8e9b5d22e4217a9a505ff9a231a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 16:15:10 +0200 Subject: [PATCH 084/154] Lint --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 47baedfa9db..5c073707b9e 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - /// Removing quotes this will introduce incompatibility between replicas with different version. + /// Removing these quotes this will introduce incompatibility between replicas with different versions. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 30eca0f4a04437cd5edc72a86864cd73bba9a17a Mon Sep 17 00:00:00 2001 From: Konstantin Bogdanov Date: Wed, 28 Aug 2024 16:16:09 +0200 Subject: [PATCH 085/154] Lint harder --- src/Core/SettingsFields.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index 5c073707b9e..930eedb8d70 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -237,7 +237,7 @@ SettingFieldMaxThreads & SettingFieldMaxThreads::operator=(const Field & f) String SettingFieldMaxThreads::toString() const { if (is_auto) - /// Removing these quotes this will introduce incompatibility between replicas with different versions. + /// Removing quotes here will introduce an incompatibility between replicas with different versions. return "'auto(" + ::DB::toString(value) + ")'"; else return ::DB::toString(value); From 100c560cb84163a025429d5816d00056bb838adb Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 20:37:49 +0100 Subject: [PATCH 086/154] Added RealTimeMicroseconds event --- src/Interpreters/executeQuery.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 2a796004d10..682d5bae36e 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -90,6 +90,7 @@ namespace ProfileEvents extern const Event SelectQueryTimeMicroseconds; extern const Event InsertQueryTimeMicroseconds; extern const Event OtherQueryTimeMicroseconds; + extern const Event RealTimeMicroseconds; } namespace DB From 2898b123c3337a30946aefb16e057bd13dee2553 Mon Sep 17 00:00:00 2001 From: Samuel Warfield Date: Wed, 28 Aug 2024 14:11:16 -0600 Subject: [PATCH 087/154] Update parametric-functions.md Fix link to pattern syntax --- .../aggregate-functions/parametric-functions.md | 5 +++-- 1 file changed, 3 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 093d88f939f..47fbb2ea31d 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -104,7 +104,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). +- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). **Returned values** @@ -114,6 +114,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord Type: `UInt8`. + **Pattern syntax** - `(?N)` — Matches the condition argument at position `N`. Conditions are numbered in the `[1, 32]` range. For example, `(?1)` matches the argument passed to the `cond1` parameter. @@ -196,7 +197,7 @@ sequenceCount(pattern)(timestamp, cond1, cond2, ...) **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequencematch). +- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). **Returned values** From 91724c29a48b894dabab5587c2f218ead7aaf87a Mon Sep 17 00:00:00 2001 From: Alejandro Date: Wed, 28 Aug 2024 23:26:13 +0100 Subject: [PATCH 088/154] Only increase if profile_counters is intialized --- src/Interpreters/executeQuery.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 682d5bae36e..72a6708bb27 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -421,8 +421,12 @@ void logQueryFinish( Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); - UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; - p.incrementRealTimeMicroseconds(real_time_microseconds); + + if (info.profile_counters) + { + UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; + p.incrementRealTimeMicroseconds(real_time_microseconds); + } progress_callback(p); } From 57ba0f0b32ac8d6a092ce7a56b825265fa34dbc1 Mon Sep 17 00:00:00 2001 From: Tuan Pham Anh Date: Thu, 29 Aug 2024 00:31:33 +0000 Subject: [PATCH 089/154] format gtest_Parser.cpp --- src/Parsers/tests/gtest_Parser.cpp | 98 +++++++++++++++--------------- 1 file changed, 49 insertions(+), 49 deletions(-) diff --git a/src/Parsers/tests/gtest_Parser.cpp b/src/Parsers/tests/gtest_Parser.cpp index 1b50d4480d7..47f7a54389b 100644 --- a/src/Parsers/tests/gtest_Parser.cpp +++ b/src/Parsers/tests/gtest_Parser.cpp @@ -225,55 +225,55 @@ INSTANTIATE_TEST_SUITE_P(ParserCreateDatabaseQuery, ParserTest, ::testing::Combine( ::testing::Values(std::make_shared()), ::testing::ValuesIn(std::initializer_list{ - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", - "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" - }, - { - "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", - "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", - "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" - }, - { - "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", - "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" - }, - { - "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", - "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" - }, - { - "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", - "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" - } - }))); + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw')", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE `tbl`\n(PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE=Foo TABLE OVERRIDE `tbl` (), TABLE OVERRIDE a (COLUMNS (_created DateTime MATERIALIZED now())), TABLE OVERRIDE b (PARTITION BY rand())", + "CREATE DATABASE db\nENGINE = Foo\nTABLE OVERRIDE tbl\n(\n\n),\nTABLE OVERRIDE a\n(\n COLUMNS\n (\n `_created` DateTime MATERIALIZED now()\n )\n),\nTABLE OVERRIDE b\n(\n PARTITION BY rand()\n)" + }, + { + "CREATE DATABASE db ENGINE=MaterializeMySQL('addr:port', 'db', 'user', 'pw') TABLE OVERRIDE tbl (COLUMNS (id UUID) PARTITION BY toYYYYMM(created))", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('addr:port', 'db', 'user', 'pw')\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `id` UUID\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (COLUMNS (INDEX foo foo TYPE minmax GRANULARITY 1) PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n INDEX foo foo TYPE minmax GRANULARITY 1\n )\n PARTITION BY if(_staged = 1, 'staging', toYYYYMM(created))\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE t1 (TTL inserted + INTERVAL 1 MONTH DELETE), TABLE OVERRIDE t2 (TTL `inserted` + INTERVAL 2 MONTH DELETE)", + "CREATE DATABASE db\nTABLE OVERRIDE t1\n(\n TTL inserted + toIntervalMonth(1)\n),\nTABLE OVERRIDE t2\n(\n TTL inserted + toIntervalMonth(2)\n)" + }, + { + "CREATE DATABASE db ENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw') SETTINGS allows_query_when_mysql_lost = 1 TABLE OVERRIDE tab3 (COLUMNS (_staged UInt8 MATERIALIZED 1) PARTITION BY (c3) TTL c3 + INTERVAL 10 minute), TABLE OVERRIDE tab5 (PARTITION BY (c3) TTL c3 + INTERVAL 10 minute)", + "CREATE DATABASE db\nENGINE = MaterializeMySQL('127.0.0.1:3306', 'db', 'root', 'pw')\nSETTINGS allows_query_when_mysql_lost = 1\nTABLE OVERRIDE tab3\n(\n COLUMNS\n (\n `_staged` UInt8 MATERIALIZED 1\n )\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n),\nTABLE OVERRIDE tab5\n(\n PARTITION BY c3\n TTL c3 + toIntervalMinute(10)\n)" + }, + { + "CREATE DATABASE db TABLE OVERRIDE tbl (PARTITION BY toYYYYMM(created) COLUMNS (created DateTime CODEC(Delta)))", + "CREATE DATABASE db\nTABLE OVERRIDE tbl\n(\n COLUMNS\n (\n `created` DateTime CODEC(Delta)\n )\n PARTITION BY toYYYYMM(created)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)" + }, + { + "CREATE DATABASE db ENGINE = Foo() SETTINGS a = 1, b = 2 COMMENT 'db comment' TABLE OVERRIDE a (ORDER BY (id, version))", + "CREATE DATABASE db\nENGINE = Foo\nSETTINGS a = 1, b = 2\nTABLE OVERRIDE a\n(\n ORDER BY (id, version)\n)\nCOMMENT 'db comment'" + } +}))); INSTANTIATE_TEST_SUITE_P(ParserCreateUserQuery, ParserTest, ::testing::Combine( From 88c99fa0636fc389f13bd6a16dbb83d2e7bfa876 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 08:43:01 +0100 Subject: [PATCH 090/154] Do not depend on the response of getInfo for increasing the RealTimeMicroseconds --- src/Interpreters/executeQuery.cpp | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index 72a6708bb27..4da2ea55066 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -399,9 +399,14 @@ void logQueryFinish( /// Update performance counters before logging to query_log CurrentThread::finalizePerformanceCounters(); - QueryStatusInfo info = process_list_elem->getInfo(true, context->getSettingsRef().log_profile_events); - elem.type = QueryLogElementType::QUERY_FINISH; + std::shared_ptr profile_counters; + QueryStatusInfo info = process_list_elem->getInfo(true, true); + if (context->getSettingsRef().log_profile_events) + profile_counters = info.profile_counters; + else + profile_counters.swap(info.profile_counters); + elem.type = QueryLogElementType::QUERY_FINISH; addStatusInfoToQueryLogElement(elem, info, query_ast, context); if (pulling_pipeline) @@ -420,13 +425,7 @@ void logQueryFinish( { Progress p; p.incrementPiecewiseAtomically(Progress{ResultProgress{elem.result_rows, elem.result_bytes}}); - - - if (info.profile_counters) - { - UInt64 real_time_microseconds = (*info.profile_counters)[ProfileEvents::RealTimeMicroseconds]; - p.incrementRealTimeMicroseconds(real_time_microseconds); - } + p.incrementRealTimeMicroseconds((*profile_counters)[ProfileEvents::RealTimeMicroseconds]); progress_callback(p); } From 920a1b2801a8395696e5ae48231e140c6af7e73c Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 11:39:29 +0100 Subject: [PATCH 091/154] Added test to validate that the value is being populated --- .../03228_url_engine_response_headers.reference | 1 + .../0_stateless/03228_url_engine_response_headers.sql | 8 ++++++++ 2 files changed, 9 insertions(+) diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.reference b/tests/queries/0_stateless/03228_url_engine_response_headers.reference index f28952972b8..199af6ed2ee 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.reference +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.reference @@ -1,2 +1,3 @@ Map(LowCardinality(String), LowCardinality(String)) 1 1 +3 100 1 \ No newline at end of file diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql index ff8e47611f4..95e8cd1865d 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.sql +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -5,3 +5,11 @@ SELECT *, mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'] FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); + +-- The real_time_microseconds is not available in the `X-ClickHouse-Progress` header (it is available in the `X-ClickHouse-Summary` header). +-- We need to wait until the query is finished to get the real_time_microseconds. +SELECT + *, + mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'], + toUInt64OrDefault(mapFromString(_headers['X-ClickHouse-Summary'])['real_time_microseconds']) >= 0 ? 1 : 0 +FROM url('http://127.0.0.1:8123/?query=SELECT%20uniq%28number%253%29%20FROM%20numbers%28100%29&user=default&wait_end_of_query=1', LineAsString, 's String'); From e9d806ea34bc18af65c15c65ff0bf5dcdf0b1e46 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 12:10:18 +0100 Subject: [PATCH 092/154] Added missing breakline --- .../0_stateless/03228_url_engine_response_headers.reference | 2 +- tests/queries/0_stateless/03228_url_engine_response_headers.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.reference b/tests/queries/0_stateless/03228_url_engine_response_headers.reference index 199af6ed2ee..84b499f5a8c 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.reference +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.reference @@ -1,3 +1,3 @@ Map(LowCardinality(String), LowCardinality(String)) 1 1 -3 100 1 \ No newline at end of file +3 100 1 diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql index 95e8cd1865d..20fcec41030 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.sql +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -11,5 +11,5 @@ FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's SELECT *, mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'], - toUInt64OrDefault(mapFromString(_headers['X-ClickHouse-Summary'])['real_time_microseconds']) >= 0 ? 1 : 0 + toUInt64OrZero(mapFromString(_headers['X-ClickHouse-Summary'])['real_time_microseconds']) >= 0 ? 1 : 0 FROM url('http://127.0.0.1:8123/?query=SELECT%20uniq%28number%253%29%20FROM%20numbers%28100%29&user=default&wait_end_of_query=1', LineAsString, 's String'); From d533c24321a8a1c8d7a937bbce6756e91a2230b8 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:14:43 +0200 Subject: [PATCH 093/154] Reduce test size --- .../0_stateless/03227_distinct_dynamic_types_json_paths.sql | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql index 6930f5a3d44..3af911e27fa 100644 --- a/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql +++ b/tests/queries/0_stateless/03227_distinct_dynamic_types_json_paths.sql @@ -4,10 +4,11 @@ set allow_experimental_dynamic_type = 1; set allow_experimental_json_type = 1; set allow_experimental_variant_type = 1; set use_variant_as_common_type = 1; +set max_block_size = 10000; drop table if exists test_json_dynamic_aggregate_functions; create table test_json_dynamic_aggregate_functions (json JSON(a1 String, max_dynamic_paths=2, max_dynamic_types=2)) engine=Memory; -insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(200000); +insert into test_json_dynamic_aggregate_functions select toJSONString(map('a' || number % 13, multiIf(number % 5 == 0, NULL, number % 5 == 1, number::UInt32, number % 5 == 2, 'str_' || number, number % 5 == 3, range(number % 5), toBool(number % 2)))) from numbers(100000); select arrayJoin(distinctJSONPaths(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctJSONPathsAndTypes(json)) from test_json_dynamic_aggregate_functions; select arrayJoin(distinctDynamicTypes(json.a2)) from test_json_dynamic_aggregate_functions; From 653b0802c0cd47a832d2b52617be7dfe0c30a70c Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:17:27 +0200 Subject: [PATCH 094/154] init --- src/Storages/StorageFile.cpp | 1 + .../0_stateless/03232_file_path_normalizing.reference | 1 + tests/queries/0_stateless/03232_file_path_normalizing.sh | 7 +++++++ 3 files changed, 9 insertions(+) create mode 100644 tests/queries/0_stateless/03232_file_path_normalizing.reference create mode 100644 tests/queries/0_stateless/03232_file_path_normalizing.sh diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 50294df32a4..036a01914cf 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -126,6 +126,7 @@ void listFilesWithRegexpMatchingImpl( /// Otherwise it will not allow to work with symlinks in `user_files_path` directory. fs::canonical(path_for_ls + for_match); fs::path absolute_path = fs::absolute(path_for_ls + for_match); + absolute_path = absolute_path.lexically_normal(); /// ensure that the resulting path is normalized (e.g., removes any redundant slashes or . and .. segments) result.push_back(absolute_path.string()); } catch (const std::exception &) // NOLINT diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference new file mode 100644 index 00000000000..fe3792e5062 --- /dev/null +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -0,0 +1 @@ +/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh new file mode 100644 index 00000000000..6c3c12a1013 --- /dev/null +++ b/tests/queries/0_stateless/03232_file_path_normalizing.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_LOCAL -q "SELECT _path FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" From 96a2685f91dc6b75c2cafd39945502e92a4a5a9f Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:24:23 +0200 Subject: [PATCH 095/154] empty commit From 34ce43904309f566e216722c2169f8ccc07dc4a0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 12:04:40 +0000 Subject: [PATCH 096/154] chmod +x on test --- tests/queries/0_stateless/03232_file_path_normalizing.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 tests/queries/0_stateless/03232_file_path_normalizing.sh diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh old mode 100644 new mode 100755 From 97be458b58a181207ffdfd52c2e99c6745a410aa Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 12:05:15 +0000 Subject: [PATCH 097/154] add no-fasttest tag --- tests/queries/0_stateless/03232_file_path_normalizing.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh index 6c3c12a1013..eeaa1f2014d 100755 --- a/tests/queries/0_stateless/03232_file_path_normalizing.sh +++ b/tests/queries/0_stateless/03232_file_path_normalizing.sh @@ -1,4 +1,5 @@ #!/usr/bin/env bash +# Tags: no-fasttest CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh From 8bbb78375e52c3ae16531de5c96b9eb6bb365b59 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 12:18:51 +0000 Subject: [PATCH 098/154] add try limit --- programs/keeper-client/Commands.cpp | 39 +++++++++++++++++++++++------ 1 file changed, 32 insertions(+), 7 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 2afd55e835e..2e946ccbf11 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -682,13 +682,25 @@ namespace class CPMVOperation { + constexpr static UInt64 kTryLimit = 1000; + public: CPMVOperation(String src_, String dest_, bool remove_src_, KeeperClient * client_) : src(std::move(src_)), dest(std::move(dest_)), remove_src(remove_src_), client(client_) { } - bool perform() + bool isTryLimitReached() const + { + return failed_tries_count >= kTryLimit; + } + + bool isCompleted() const + { + return is_completed; + } + + void perform() { Coordination::Stat src_stat; String data = client->zookeeper->get(src, &src_stat); @@ -707,9 +719,19 @@ public: switch (code) { case Coordination::Error::ZOK: - return true; + { + is_completed = true; + return; + } case Coordination::Error::ZBADVERSION: - return false; + { + ++failed_tries_count; + + if (isTryLimitReached()) + zkutil::KeeperMultiException::check(code, ops, responses); + + return; + } default: zkutil::KeeperMultiException::check(code, ops, responses); } @@ -722,6 +744,9 @@ private: String dest; bool remove_src = false; KeeperClient * client = nullptr; + + bool is_completed = false; + uint64_t failed_tries_count = 0; }; } @@ -748,8 +773,8 @@ void CPCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/false, /*client_=*/client); - while (!operation.perform()) - ; + while (!operation.isTryLimitReached() && !operation.isCompleted()) + operation.perform(); } bool MVCommand::parse(IParser::Pos & pos, std::shared_ptr & node, Expected & expected) const @@ -774,8 +799,8 @@ void MVCommand::execute(const ASTKeeperQuery * query, KeeperClient * client) con CPMVOperation operation(std::move(src), std::move(dest), /*remove_src_=*/true, /*client_=*/client); - while (!operation.perform()) - ; + while (!operation.isTryLimitReached() && !operation.isCompleted()) + operation.perform(); } } From e9497cdcbcb62612dfedbf6f10ae98e9d7d8870d Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 12:20:42 +0000 Subject: [PATCH 099/154] use tryMulti instead of NoThrow --- programs/keeper-client/Commands.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 2e946ccbf11..0025feacb4b 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -714,7 +714,7 @@ public: ops.push_back(zkutil::makeRemoveRequest(src, src_stat.version)); Coordination::Responses responses; - auto code = client->zookeeper->tryMultiNoThrow(ops, responses); + auto code = client->zookeeper->tryMulti(ops, responses); switch (code) { From f06678ee6cb1a0ddf6b8eea74702b6f7575f897d Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 13:12:37 +0000 Subject: [PATCH 100/154] add test --- .../03230_keeper_cp_mv_commands.reference | 15 ++++++++ .../03230_keeper_cp_mv_commands.sh | 38 +++++++++++++++++++ 2 files changed, 53 insertions(+) create mode 100644 tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference create mode 100755 tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference new file mode 100644 index 00000000000..15f79fdebc7 --- /dev/null +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference @@ -0,0 +1,15 @@ +initial +A C +simple copy +A C D +node-A +simple move +A C H +node-A +move node with childs -- must be error +Transaction failed (Not empty): Op #2, path: /test-keeper-client-cp-mv-commands/A +A C H +move node to existing +Transaction failed (Node exists): Op #1, path: /test-keeper-client-cp-mv-commands/A +A C H +clean up diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh new file mode 100755 index 00000000000..15e117c7ca6 --- /dev/null +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +path="/test-keeper-client-cp-mv-commands" + +$CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null + +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'root'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/A' 'node-A'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/A/B' 'node-B'" +$CLICKHOUSE_KEEPER_CLIENT -q "create '$path/C' 'node-B'" + +echo 'initial' +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" + +echo 'simple copy' +$CLICKHOUSE_KEEPER_CLIENT -q "cp '$path/A' '$path/D'" +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" +$CLICKHOUSE_KEEPER_CLIENT -q "get '$path/D'" + +echo 'simple move' +$CLICKHOUSE_KEEPER_CLIENT -q "mv '$path/D' '$path/H'" +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" +$CLICKHOUSE_KEEPER_CLIENT -q "get '$path/H'" + +echo 'move node with childs -- must be error' +$CLICKHOUSE_KEEPER_CLIENT -q "mv '$path/A' '$path/ERROR'" 2>&1 +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" + +echo 'move node to existing' +$CLICKHOUSE_KEEPER_CLIENT -q "mv '$path/C' '$path/A'" 2>&1 +$CLICKHOUSE_KEEPER_CLIENT -q "ls '$path'" + +echo 'clean up' +$CLICKHOUSE_KEEPER_CLIENT -q "rmr '$path'" From d8f0ce92affe216a054e81fa0ea51e419bd0b81c Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 13:28:00 +0000 Subject: [PATCH 101/154] fix style --- programs/keeper-client/Commands.cpp | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/programs/keeper-client/Commands.cpp b/programs/keeper-client/Commands.cpp index 0025feacb4b..4ad2eb31e6d 100644 --- a/programs/keeper-client/Commands.cpp +++ b/programs/keeper-client/Commands.cpp @@ -690,15 +690,9 @@ public: { } - bool isTryLimitReached() const - { - return failed_tries_count >= kTryLimit; - } + bool isTryLimitReached() const { return failed_tries_count >= kTryLimit; } - bool isCompleted() const - { - return is_completed; - } + bool isCompleted() const { return is_completed; } void perform() { @@ -718,13 +712,11 @@ public: switch (code) { - case Coordination::Error::ZOK: - { + case Coordination::Error::ZOK: { is_completed = true; return; } - case Coordination::Error::ZBADVERSION: - { + case Coordination::Error::ZBADVERSION: { ++failed_tries_count; if (isTryLimitReached()) From 13311bd6665a48cc73ea8c6dbabfa1ab1f6af338 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Thu, 29 Aug 2024 16:02:06 +0200 Subject: [PATCH 102/154] fix test --- tests/queries/0_stateless/03232_file_path_normalizing.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference index fe3792e5062..d7dc12010f5 100644 --- a/tests/queries/0_stateless/03232_file_path_normalizing.reference +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -1 +1 @@ -/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Stacy/sample.parquet +/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet From 36725fb5e149ed532aca6c290585dec444ed03b4 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Thu, 29 Aug 2024 15:05:19 +0100 Subject: [PATCH 103/154] Improved comment to force CI to re-run --- tests/queries/0_stateless/03228_url_engine_response_headers.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03228_url_engine_response_headers.sql b/tests/queries/0_stateless/03228_url_engine_response_headers.sql index 20fcec41030..41a1a2406da 100644 --- a/tests/queries/0_stateless/03228_url_engine_response_headers.sql +++ b/tests/queries/0_stateless/03228_url_engine_response_headers.sql @@ -6,7 +6,7 @@ SELECT mapFromString(_headers['X-ClickHouse-Summary'])['read_rows'] FROM url('http://127.0.0.1:8123/?query=select+1&user=default', LineAsString, 's String'); --- The real_time_microseconds is not available in the `X-ClickHouse-Progress` header (it is available in the `X-ClickHouse-Summary` header). +-- The real_time_microseconds is not available in the `X-ClickHouse-Progress` header (it is only available in the `X-ClickHouse-Summary` header). -- We need to wait until the query is finished to get the real_time_microseconds. SELECT *, From 4499f20ca8af46f88da72f6eaeb2880ded348efa Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Aug 2024 14:38:59 +0000 Subject: [PATCH 104/154] better code in AsynchronousInsertQueue --- src/Interpreters/AsynchronousInsertQueue.cpp | 70 ++++++++------------ src/Interpreters/AsynchronousInsertQueue.h | 7 -- 2 files changed, 27 insertions(+), 50 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index 5ed0d6d6257..c619ea80c7c 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -805,12 +805,10 @@ try if (async_insert_log) log_elements.reserve(data->entries.size()); - auto add_entry_to_asynchronous_insert_log = [&]( + auto add_entry_to_asynchronous_insert_log = [&, query_by_format = NameToNameMap{}]( const InsertData::EntryPtr & entry, - const NameToNameMap & query_by_format, const String & parsing_exception, - size_t num_rows, - size_t num_bytes) + size_t num_rows) mutable { if (!async_insert_log) return; @@ -822,15 +820,29 @@ try elem.table = query_table; elem.format = entry->format; elem.query_id = entry->query_id; - elem.bytes = num_bytes; + elem.bytes = entry->chunk.byteSize(); elem.rows = num_rows; elem.exception = parsing_exception; elem.data_kind = entry->chunk.getDataKind(); elem.timeout_milliseconds = data->timeout_ms.count(); elem.flush_query_id = insert_query_id; - auto it = query_by_format.find(entry->format); - elem.query_for_logging = it != query_by_format.end() ? it->second : key.query_str; + auto get_query_by_format = [&](const String & format) -> const String & + { + auto [it, inserted] = query_by_format.try_emplace(format); + if (!inserted) + return it->second; + + auto query = key.query->clone(); + assert_cast(*query).format = format; + it->second = serializeQuery(*query, insert_context->getSettingsRef().log_queries_cut_to_length); + return it->second; + }; + + if (entry->chunk.getDataKind() == DataKind::Parsed) + elem.query_for_logging = key.query_str; + else + elem.query_for_logging = get_query_by_format(entry->format); /// If there was a parsing error, /// the entry won't be flushed anyway, @@ -843,7 +855,7 @@ try else { elem.status = AsynchronousInsertLogElement::Ok; - log_elements.push_back(elem); + log_elements.push_back(std::move(elem)); } }; @@ -878,9 +890,8 @@ try if (async_insert_log) { - auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); for (const auto & entry : data->entries) - add_entry_to_asynchronous_insert_log(entry, query_by_format, "", 0, entry->chunk.byteSize()); + add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0); auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); @@ -919,13 +930,13 @@ try if (key.data_kind == DataKind::Parsed) chunk = processEntriesWithParsing(key, data, header, insert_context, log, add_entry_to_asynchronous_insert_log); else - chunk = processPreprocessedEntries(key, data, header, insert_context, add_entry_to_asynchronous_insert_log); + chunk = processPreprocessedEntries(data, header, add_entry_to_asynchronous_insert_log); ProfileEvents::increment(ProfileEvents::AsyncInsertRows, chunk.getNumRows()); if (chunk.getNumRows() == 0) { - finish_entries(0, 0); + finish_entries(/*num_rows=*/ 0, /*num_bytes=*/ 0); return; } @@ -1012,7 +1023,6 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( StreamingFormatExecutor executor(header, format, std::move(on_error), std::move(adding_defaults_transform)); auto chunk_info = std::make_shared(); - auto query_for_logging = serializeQuery(*key.query, insert_context->getSettingsRef().log_queries_cut_to_length); for (const auto & entry : data->entries) { @@ -1025,15 +1035,13 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); - size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); - total_rows += num_rows; + chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, {}, current_exception, num_rows, num_bytes); - + add_to_async_insert_log(entry, current_exception, num_rows); current_exception.clear(); entry->resetChunk(); } @@ -1045,18 +1053,14 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( template Chunk AsynchronousInsertQueue::processPreprocessedEntries( - const InsertQuery & key, const InsertDataPtr & data, const Block & header, - const ContextPtr & insert_context, LogFunc && add_to_async_insert_log) { size_t total_rows = 0; auto chunk_info = std::make_shared(); auto result_columns = header.cloneEmptyColumns(); - auto query_by_format = getQueriesByFormat(key.query, data->entries, insert_context); - for (const auto & entry : data->entries) { const auto * block = entry->chunk.asBlock(); @@ -1073,10 +1077,11 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); total_rows += block_to_insert.rows(); + chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, query_by_format, "", block_to_insert.rows(), block_to_insert.bytes()); + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows()); entry->resetChunk(); } @@ -1085,27 +1090,6 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( return chunk; } -NameToNameMap AsynchronousInsertQueue::getQueriesByFormat( - const ASTPtr & query, - const std::list & entries, - const ContextPtr & insert_context) -{ - std::unordered_map format_to_query; - auto query_copy = query->clone(); - - for (const auto & entry : entries) - { - auto [it, inserted] = format_to_query.try_emplace(entry->format); - if (!inserted) - continue; - - assert_cast(*query_copy).format = entry->format; - it->second = serializeQuery(*query_copy, insert_context->getSettingsRef().log_queries_cut_to_length); - } - - return format_to_query; -} - template void AsynchronousInsertQueue::finishWithException( const ASTPtr & query, const std::list & entries, const E & exception) diff --git a/src/Interpreters/AsynchronousInsertQueue.h b/src/Interpreters/AsynchronousInsertQueue.h index 9a84fe8bb12..cbe998a2850 100644 --- a/src/Interpreters/AsynchronousInsertQueue.h +++ b/src/Interpreters/AsynchronousInsertQueue.h @@ -287,17 +287,10 @@ private: template static Chunk processPreprocessedEntries( - const InsertQuery & key, const InsertDataPtr & data, const Block & header, - const ContextPtr & insert_context, LogFunc && add_to_async_insert_log); - static NameToNameMap getQueriesByFormat( - const ASTPtr & query, - const std::list & entries, - const ContextPtr & insert_context); - template static void finishWithException(const ASTPtr & query, const std::list & entries, const E & exception); From 1aae191b810d30519cef1fd9784567ee03fd34e5 Mon Sep 17 00:00:00 2001 From: Mikhail Artemenko Date: Thu, 29 Aug 2024 15:26:42 +0000 Subject: [PATCH 105/154] use db name in keeper path --- .../queries/0_stateless/03230_keeper_cp_mv_commands.reference | 4 ++-- tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh | 3 +-- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference index 15f79fdebc7..f258af94664 100644 --- a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.reference @@ -7,9 +7,9 @@ simple move A C H node-A move node with childs -- must be error -Transaction failed (Not empty): Op #2, path: /test-keeper-client-cp-mv-commands/A +Transaction failed (Not empty): Op #2, path: /test-keeper-client-default/A A C H move node to existing -Transaction failed (Node exists): Op #1, path: /test-keeper-client-cp-mv-commands/A +Transaction failed (Node exists): Op #1, path: /test-keeper-client-default/A A C H clean up diff --git a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh index 15e117c7ca6..59b3547c36e 100755 --- a/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh +++ b/tests/queries/0_stateless/03230_keeper_cp_mv_commands.sh @@ -4,8 +4,7 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -path="/test-keeper-client-cp-mv-commands" - +path="/test-keeper-client-$CLICKHOUSE_DATABASE" $CLICKHOUSE_KEEPER_CLIENT -q "rm '$path'" >& /dev/null $CLICKHOUSE_KEEPER_CLIENT -q "create '$path' 'root'" From a4ddca773b4fdc5cc8d2b003bb64ae2de4790439 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 29 Aug 2024 16:03:41 +0000 Subject: [PATCH 106/154] return back num_bytes --- src/Interpreters/AsynchronousInsertQueue.cpp | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/AsynchronousInsertQueue.cpp b/src/Interpreters/AsynchronousInsertQueue.cpp index c619ea80c7c..9d7007d61aa 100644 --- a/src/Interpreters/AsynchronousInsertQueue.cpp +++ b/src/Interpreters/AsynchronousInsertQueue.cpp @@ -808,7 +808,8 @@ try auto add_entry_to_asynchronous_insert_log = [&, query_by_format = NameToNameMap{}]( const InsertData::EntryPtr & entry, const String & parsing_exception, - size_t num_rows) mutable + size_t num_rows, + size_t num_bytes) mutable { if (!async_insert_log) return; @@ -820,7 +821,7 @@ try elem.table = query_table; elem.format = entry->format; elem.query_id = entry->query_id; - elem.bytes = entry->chunk.byteSize(); + elem.bytes = num_bytes; elem.rows = num_rows; elem.exception = parsing_exception; elem.data_kind = entry->chunk.getDataKind(); @@ -891,7 +892,7 @@ try if (async_insert_log) { for (const auto & entry : data->entries) - add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0); + add_entry_to_asynchronous_insert_log(entry, /*parsing_exception=*/ "", /*num_rows=*/ 0, entry->chunk.byteSize()); auto exception = getCurrentExceptionMessage(false); auto flush_time = std::chrono::system_clock::now(); @@ -1035,13 +1036,15 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing( auto buffer = std::make_unique(*bytes); + size_t num_bytes = bytes->size(); size_t num_rows = executor.execute(*buffer); + total_rows += num_rows; chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, current_exception, num_rows); + add_to_async_insert_log(entry, current_exception, num_rows, num_bytes); current_exception.clear(); entry->resetChunk(); } @@ -1081,7 +1084,7 @@ Chunk AsynchronousInsertQueue::processPreprocessedEntries( chunk_info->offsets.push_back(total_rows); chunk_info->tokens.push_back(entry->async_dedup_token); - add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows()); + add_to_async_insert_log(entry, /*parsing_exception=*/ "", block_to_insert.rows(), block_to_insert.bytes()); entry->resetChunk(); } From aad57f438cf4ab5d09c0f02b8946f54e44e7d8f8 Mon Sep 17 00:00:00 2001 From: sakulali Date: Fri, 30 Aug 2024 00:16:10 +0800 Subject: [PATCH 107/154] CREATE TABLE AS copy PRIMARY KEY, ORDER BY, and similar clauses. --- src/Interpreters/InterpreterCreateQuery.cpp | 13 ++++ ...te_table_as_with_sorting_clauses.reference | 70 +++++++++++++++++++ ...6_create_table_as_with_sorting_clauses.sql | 37 ++++++++++ 3 files changed, 120 insertions(+) create mode 100644 tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference create mode 100644 tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index 80cb0510b35..e9f40bdbaf5 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -821,6 +821,19 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti { properties.indices = as_storage_metadata->getSecondaryIndices(); properties.projections = as_storage_metadata->getProjections().clone(); + + /// CREATE TABLE AS should copy PRIMARY KEY, ORDER BY, and similar clauses. + if (!create.storage->primary_key && as_storage_metadata->isPrimaryKeyDefined() && as_storage_metadata->hasPrimaryKey()) + create.storage->set(create.storage->primary_key, as_storage_metadata->getPrimaryKeyAST()->clone()); + + if (!create.storage->partition_by && as_storage_metadata->isPartitionKeyDefined() && as_storage_metadata->hasPartitionKey()) + create.storage->set(create.storage->partition_by, as_storage_metadata->getPartitionKeyAST()->clone()); + + if (!create.storage->order_by && as_storage_metadata->isSortingKeyDefined() && as_storage_metadata->hasSortingKey()) + create.storage->set(create.storage->order_by, as_storage_metadata->getSortingKeyAST()->clone()); + + if (!create.storage->sample_by && as_storage_metadata->isSamplingKeyDefined() && as_storage_metadata->hasSamplingKey()) + create.storage->set(create.storage->sample_by, as_storage_metadata->getSamplingKeyAST()->clone()); } else { diff --git a/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference new file mode 100644 index 00000000000..cebb99f005e --- /dev/null +++ b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.reference @@ -0,0 +1,70 @@ +-------------- Test copy sorting clauses from source table -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 +-------------- Test copy sorting clauses from destination table (source table without the same type clauses) -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PRIMARY KEY (CounterID, EventDate, intHash32(UserID)) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +PRIMARY KEY (CounterID, EventDate, intHash32(UserID)) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 +-------------- Test copy sorting clauses from destination table (source table with the same type clauses) -------------- +CREATE TABLE default.x +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +ORDER BY CounterID +SETTINGS index_granularity = 8192 +------------------------------------------------------------------------- +CREATE TABLE default.x_as +( + `CounterID` UInt32, + `EventDate` Date, + `UserID` UInt64 +) +ENGINE = MergeTree +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192 diff --git a/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql new file mode 100644 index 00000000000..96c2df54491 --- /dev/null +++ b/tests/queries/0_stateless/01056_create_table_as_with_sorting_clauses.sql @@ -0,0 +1,37 @@ +DROP TABLE IF EXISTS x; +DROP TABLE IF EXISTS x_as; + +SELECT '-------------- Test copy sorting clauses from source table --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID); +CREATE TABLE x_as AS x ENGINE = MergeTree SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + +SELECT '-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PRIMARY KEY (CounterID, EventDate, intHash32(UserID)); +CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + +SELECT '-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------'; +CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree ORDER BY (CounterID); +CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1; + +SHOW CREATE TABLE x FORMAT TSVRaw; +SELECT '-------------------------------------------------------------------------'; +SHOW CREATE TABLE x_as FORMAT TSVRaw; + +DROP TABLE x; +DROP TABLE x_as; + + From f5a88171a6c2959847beead82b204d000600aab4 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 29 Aug 2024 20:37:07 +0000 Subject: [PATCH 108/154] Fix ProgressIndication data race on `cancelQuery()` --- src/Common/ProgressIndication.cpp | 17 ++++++++++++----- src/Common/ProgressIndication.h | 2 ++ 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/src/Common/ProgressIndication.cpp b/src/Common/ProgressIndication.cpp index 8f0fb3cac6c..79c694574b0 100644 --- a/src/Common/ProgressIndication.cpp +++ b/src/Common/ProgressIndication.cpp @@ -34,13 +34,16 @@ bool ProgressIndication::updateProgress(const Progress & value) void ProgressIndication::resetProgress() { - watch.restart(); - progress.reset(); - show_progress_bar = false; - written_progress_chars = 0; - write_progress_on_update = false; + { + std::lock_guard lock(progress_mutex); + progress.reset(); + show_progress_bar = false; + written_progress_chars = 0; + write_progress_on_update = false; + } { std::lock_guard lock(profile_events_mutex); + watch.restart(); cpu_usage_meter.reset(getElapsedNanoseconds()); hosts_data.clear(); } @@ -90,6 +93,8 @@ ProgressIndication::MemoryUsage ProgressIndication::getMemoryUsage() const void ProgressIndication::writeFinalProgress() { + std::lock_guard lock(progress_mutex); + if (progress.read_rows < 1000) return; @@ -271,6 +276,8 @@ void ProgressIndication::writeProgress(WriteBufferFromFileDescriptor & message) void ProgressIndication::clearProgressOutput(WriteBufferFromFileDescriptor & message) { + std::lock_guard lock(progress_mutex); + if (written_progress_chars) { written_progress_chars = 0; diff --git a/src/Common/ProgressIndication.h b/src/Common/ProgressIndication.h index 474dd8db715..61b4ca1b305 100644 --- a/src/Common/ProgressIndication.h +++ b/src/Common/ProgressIndication.h @@ -115,6 +115,8 @@ private: /// It is possible concurrent access to the following: /// - writeProgress() (class properties) (guarded with progress_mutex) /// - hosts_data/cpu_usage_meter (guarded with profile_events_mutex) + /// + /// It is also possible to have more races if query is cancelled, so that clearProgressOutput() is called concurrently mutable std::mutex profile_events_mutex; mutable std::mutex progress_mutex; From f7fda2471cba5f5e975cef247f10060084734fcb Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 30 Aug 2024 14:09:31 +0800 Subject: [PATCH 109/154] fix typo --- src/Interpreters/ActionsDAG.cpp | 4 ++-- src/Interpreters/ActionsDAG.h | 4 ++-- .../QueryPlan/Optimizations/useDataParallelAggregation.cpp | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 2a594839c6a..9ebc73260b6 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -396,7 +396,7 @@ const ActionsDAG::Node * ActionsDAG::tryFindInOutputs(const std::string & name) return nullptr; } -ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutpus(const Names & names) const +ActionsDAG::NodeRawConstPtrs ActionsDAG::findInOutputs(const Names & names) const { NodeRawConstPtrs required_nodes; required_nodes.reserve(names.size()); @@ -524,7 +524,7 @@ void ActionsDAG::removeUnusedActions(const NameSet & required_names, bool allow_ void ActionsDAG::removeUnusedActions(const Names & required_names, bool allow_remove_inputs, bool allow_constant_folding) { - auto required_nodes = findInOutpus(required_names); + auto required_nodes = findInOutputs(required_names); outputs.swap(required_nodes); removeUnusedActions(allow_remove_inputs, allow_constant_folding); } diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index ee2b3fbf4f2..2459878ce20 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -156,7 +156,7 @@ public: const Node * tryFindInOutputs(const std::string & name) const; /// Same, but for the list of names. - NodeRawConstPtrs findInOutpus(const Names & names) const; + NodeRawConstPtrs findInOutputs(const Names & names) const; /// Find first node with the same name in output nodes and replace it. /// If was not found, add node to outputs end. @@ -436,7 +436,7 @@ public: /// Returns a list of nodes representing atomic predicates. static NodeRawConstPtrs extractConjunctionAtoms(const Node * predicate); - /// Get a list of nodes. For every node, check if it can be compused using allowed subset of inputs. + /// Get a list of nodes. For every node, check if it can be computed using allowed subset of inputs. /// Returns only those nodes from the list which can be computed. static NodeRawConstPtrs filterNodesByAllowedInputs( NodeRawConstPtrs nodes, diff --git a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp index 0eeaec9bde7..d943fea785b 100644 --- a/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp +++ b/src/Processors/QueryPlan/Optimizations/useDataParallelAggregation.cpp @@ -155,7 +155,7 @@ bool isPartitionKeySuitsGroupByKey( return false; /// We are interested only in calculations required to obtain group by keys (and not aggregate function arguments for example). - auto key_nodes = group_by_actions.findInOutpus(aggregating.getParams().keys); + auto key_nodes = group_by_actions.findInOutputs(aggregating.getParams().keys); auto group_by_key_actions = ActionsDAG::cloneSubDAG(key_nodes, /*remove_aliases=*/ true); const auto & gb_key_required_columns = group_by_key_actions.getRequiredColumnsNames(); From aeaaef4347d614bded7dcbe2c230924847ba579a Mon Sep 17 00:00:00 2001 From: Alejandro Date: Fri, 30 Aug 2024 08:59:27 +0100 Subject: [PATCH 110/154] Run tests without new analyzer --- tests/queries/0_stateless/02944_variant_as_common_type.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index b3b86427b06..1f19e74e24d 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,6 +1,9 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; +-- This test should be run with allow_experimental_analyzer=0, because it checks the behavior of the old analyzer. +set allow_experimental_analyzer=0; + select toTypeName(res), if(1, [1,2,3], 'str_1') as res; select toTypeName(res), if(1, [1,2,3], 'str_1'::Nullable(String)) as res; From 8ed128792d623d137efe5708908d5eead776a34f Mon Sep 17 00:00:00 2001 From: Alejandro Date: Fri, 30 Aug 2024 08:59:40 +0100 Subject: [PATCH 111/154] remove comment --- tests/queries/0_stateless/02944_variant_as_common_type.sql | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index 1f19e74e24d..567b1d20e3a 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,7 +1,5 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; - --- This test should be run with allow_experimental_analyzer=0, because it checks the behavior of the old analyzer. set allow_experimental_analyzer=0; select toTypeName(res), if(1, [1,2,3], 'str_1') as res; From b1fc15ffc968d6c96c9e440436a728e757812710 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:49:56 +0200 Subject: [PATCH 112/154] Update docs/en/sql-reference/aggregate-functions/parametric-functions.md --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 47fbb2ea31d..f6552dc4168 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -104,7 +104,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#pattern-syntax). **Returned values** From 9a4c32e8deb748f8ee84bcecf34716000060b9bc Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:50:02 +0200 Subject: [PATCH 113/154] Update docs/en/sql-reference/aggregate-functions/parametric-functions.md --- .../sql-reference/aggregate-functions/parametric-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index f6552dc4168..f391765139a 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -197,7 +197,7 @@ sequenceCount(pattern)(timestamp, cond1, cond2, ...) **Parameters** -- `pattern` — Pattern string. See [Pattern syntax](#sequence-function-pattern-syntax). +- `pattern` — Pattern string. See [Pattern syntax](#pattern-syntax). **Returned values** From 76b938acaf236196dcfa502fd9b0e09190180f13 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 12:50:08 +0200 Subject: [PATCH 114/154] Update docs/en/sql-reference/aggregate-functions/parametric-functions.md --- .../sql-reference/aggregate-functions/parametric-functions.md | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index f391765139a..b06717fcc8c 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -113,9 +113,7 @@ Events that occur at the same second may lay in the sequence in an undefined ord Type: `UInt8`. - - -**Pattern syntax** +#### Pattern syntax - `(?N)` — Matches the condition argument at position `N`. Conditions are numbered in the `[1, 32]` range. For example, `(?1)` matches the argument passed to the `cond1` parameter. From b7eb7cceebeadc2430f88a1775b4ef00ae06f8a0 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 30 Aug 2024 14:02:59 +0200 Subject: [PATCH 115/154] remake test to remove the head of path --- tests/queries/0_stateless/03232_file_path_normalizing.reference | 2 +- tests/queries/0_stateless/03232_file_path_normalizing.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.reference b/tests/queries/0_stateless/03232_file_path_normalizing.reference index d7dc12010f5..953db2c5dfe 100644 --- a/tests/queries/0_stateless/03232_file_path_normalizing.reference +++ b/tests/queries/0_stateless/03232_file_path_normalizing.reference @@ -1 +1 @@ -/repo/tests/queries/0_stateless/data_hive/partitioning/column0=Elizabeth/sample.parquet +data_hive/partitioning/column0=Elizabeth/sample.parquet diff --git a/tests/queries/0_stateless/03232_file_path_normalizing.sh b/tests/queries/0_stateless/03232_file_path_normalizing.sh index eeaa1f2014d..e7a7a65be51 100755 --- a/tests/queries/0_stateless/03232_file_path_normalizing.sh +++ b/tests/queries/0_stateless/03232_file_path_normalizing.sh @@ -5,4 +5,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL -q "SELECT _path FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" +$CLICKHOUSE_LOCAL -q "SELECT substring(_path, position(_path, 'data_hive')) FROM file('$CURDIR/data_hive/partitioning/column0=*/sample.parquet') LIMIT 1;" From b4b68196243ff8ec90cbc31c4294663127ad976f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 30 Aug 2024 14:07:30 +0200 Subject: [PATCH 116/154] Adapt backward compatibility test --- .../test_functions.py | 45 ++++++++++++++++--- 1 file changed, 38 insertions(+), 7 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_functions.py b/tests/integration/test_backward_compatibility/test_functions.py index 3231fb87f33..202a741bfb5 100644 --- a/tests/integration/test_backward_compatibility/test_functions.py +++ b/tests/integration/test_backward_compatibility/test_functions.py @@ -67,6 +67,11 @@ def test_aggregate_states(start_cluster): f"select hex(initializeAggregation('{function_name}State', 'foo'))" ).strip() + def get_final_value_unhex(node, function_name, value): + return node.query( + f"select finalizeAggregation(unhex('{value}')::AggregateFunction({function_name}, String))" + ).strip() + for aggregate_function in aggregate_functions: logging.info("Checking %s", aggregate_function) @@ -99,13 +104,39 @@ def test_aggregate_states(start_cluster): upstream_state = get_aggregate_state_hex(upstream, aggregate_function) if upstream_state != backward_state: - logging.info( - "Failed %s, %s (backward) != %s (upstream)", - aggregate_function, - backward_state, - upstream_state, - ) - failed += 1 + allowed_changes_if_result_is_the_same = ["anyHeavy"] + + if aggregate_function in allowed_changes_if_result_is_the_same: + backward_final_from_upstream = get_final_value_unhex( + backward, aggregate_function, upstream_state + ) + upstream_final_from_backward = get_final_value_unhex( + upstream, aggregate_function, backward_state + ) + + if backward_final_from_upstream == upstream_final_from_backward: + logging.info( + "OK %s (but different intermediate states)", aggregate_function + ) + passed += 1 + else: + logging.error( + "Failed %s, Intermediate: %s (backward) != %s (upstream). Final from intermediate: %s (backward from upstream state) != %s (upstream from backward state)", + aggregate_function, + backward_state, + upstream_state, + backward_final_from_upstream, + upstream_final_from_backward, + ) + failed += 1 + else: + logging.error( + "Failed %s, %s (backward) != %s (upstream)", + aggregate_function, + backward_state, + upstream_state, + ) + failed += 1 else: logging.info("OK %s", aggregate_function) passed += 1 From 91b345714dac713f115525df48d3872c26868002 Mon Sep 17 00:00:00 2001 From: Alejandro Date: Fri, 30 Aug 2024 13:16:31 +0100 Subject: [PATCH 117/154] Revert enabling the legacy analyzer --- tests/queries/0_stateless/02944_variant_as_common_type.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02944_variant_as_common_type.sql b/tests/queries/0_stateless/02944_variant_as_common_type.sql index 567b1d20e3a..b3b86427b06 100644 --- a/tests/queries/0_stateless/02944_variant_as_common_type.sql +++ b/tests/queries/0_stateless/02944_variant_as_common_type.sql @@ -1,6 +1,5 @@ set allow_experimental_variant_type=1; set use_variant_as_common_type=1; -set allow_experimental_analyzer=0; select toTypeName(res), if(1, [1,2,3], 'str_1') as res; select toTypeName(res), if(1, [1,2,3], 'str_1'::Nullable(String)) as res; From cbf82712542fb192ac4bbf1102e894d46ecb2ee1 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 30 Aug 2024 15:45:51 +0200 Subject: [PATCH 118/154] Better --- tests/ci/changelog.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/changelog.py b/tests/ci/changelog.py index 554ba339892..8e7900de353 100755 --- a/tests/ci/changelog.py +++ b/tests/ci/changelog.py @@ -288,7 +288,7 @@ def generate_description(item: PullRequest, repo: Repository) -> Optional[Descri # Normalize bug fixes if ( re.match( - r".*(?i)bug\Wfix", + r"(?i).*bug\Wfix", category, ) # Map "Critical Bug Fix" to "Bug fix" category for changelog From 30dd82324aac0053d340d0c08b3f056f8255b6d9 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 30 Aug 2024 19:26:59 +0000 Subject: [PATCH 119/154] revert wrong data race fix --- src/Common/HTTPConnectionPool.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Common/HTTPConnectionPool.cpp b/src/Common/HTTPConnectionPool.cpp index 68c13838c04..7a65863180e 100644 --- a/src/Common/HTTPConnectionPool.cpp +++ b/src/Common/HTTPConnectionPool.cpp @@ -391,15 +391,11 @@ private: { auto idle = idleTime(); - // Reset data hooks for IO scheduling + // Set data hooks for IO scheduling if (ResourceLink link = CurrentThread::getReadResourceLink()) Session::setReceiveDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIORead(), log, request.getMethod(), request.getURI())); - else - Session::setReceiveDataHooks(); if (ResourceLink link = CurrentThread::getWriteResourceLink()) Session::setSendDataHooks(std::make_shared(link, ResourceGuard::Metrics::getIOWrite(), log, request.getMethod(), request.getURI())); - else - Session::setSendDataHooks(); std::ostream & result = Session::sendRequest(request); result.exceptions(std::ios::badbit); @@ -457,6 +453,8 @@ private: } } response_stream = nullptr; + Session::setSendDataHooks(); + Session::setReceiveDataHooks(); group->atConnectionDestroy(); From 3675e83a3f064c6972546537367cc53e36ccb236 Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 30 Aug 2024 19:27:26 +0000 Subject: [PATCH 120/154] Fix data race in ResourceGuard --- src/Common/Scheduler/ResourceGuard.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 2e735aae656..5947022d9b1 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -107,8 +107,8 @@ public: std::unique_lock lock(mutex); chassert(state == Enqueued); state = Dequeued; + dequeued_cv.notify_one(); } - dequeued_cv.notify_one(); } void wait() From 9f96d180604cba6467e64a51bf98118ebafc598d Mon Sep 17 00:00:00 2001 From: serxa Date: Fri, 30 Aug 2024 19:33:34 +0000 Subject: [PATCH 121/154] cleanup --- src/Common/Scheduler/ResourceGuard.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Common/Scheduler/ResourceGuard.h b/src/Common/Scheduler/ResourceGuard.h index 5947022d9b1..cf97f7acf93 100644 --- a/src/Common/Scheduler/ResourceGuard.h +++ b/src/Common/Scheduler/ResourceGuard.h @@ -103,12 +103,10 @@ public: // That thread will continue execution and do real consumption of requested resource synchronously. void execute() override { - { - std::unique_lock lock(mutex); - chassert(state == Enqueued); - state = Dequeued; - dequeued_cv.notify_one(); - } + std::unique_lock lock(mutex); + chassert(state == Enqueued); + state = Dequeued; + dequeued_cv.notify_one(); } void wait() From 74f99d1f07dc838513eca06fd2f9e39540b7720d Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Mon, 26 Aug 2024 16:47:59 +0800 Subject: [PATCH 122/154] support datetrunc and toStartOfxxx on date32 --- src/Functions/date_trunc.cpp | 8 ++++++-- src/Functions/toStartOfInterval.cpp | 13 +++++++++++-- .../0_stateless/00403_to_start_of_day.reference | 1 + tests/queries/0_stateless/00403_to_start_of_day.sql | 1 + .../02935_date_trunc_case_unsensitiveness.reference | 5 +++++ .../02935_date_trunc_case_unsensitiveness.sql | 8 ++++++++ 6 files changed, 32 insertions(+), 4 deletions(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index dd3ea0b877b..7b094174047 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -43,6 +44,7 @@ public: enum ResultType { Date, + Date32, DateTime, DateTime64, }; @@ -75,11 +77,11 @@ public: bool second_argument_is_date = false; auto check_second_argument = [&] { - if (!isDate(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) + if (!isDateOrDate32(arguments[1].type) && !isDateTime(arguments[1].type) && !isDateTime64(arguments[1].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of 2nd argument of function {}. " "Should be a date or a date with time", arguments[1].type->getName(), getName()); - second_argument_is_date = isDate(arguments[1].type); + second_argument_is_date = isDateOrDate32(arguments[1].type); if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour) || (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))) @@ -119,6 +121,8 @@ public: if (result_type == ResultType::Date) return std::make_shared(); + if (result_type == ResultType::Date32) + return std::make_shared(); else if (result_type == ResultType::DateTime) return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false)); else diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 21b7cf895d2..35fb43df131 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -44,9 +44,9 @@ public: auto check_first_argument = [&] { const DataTypePtr & type_arg1 = arguments[0].type; - if (!isDate(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) + if (!isDateOrDate32(type_arg1) && !isDateTime(type_arg1) && !isDateTime64(type_arg1)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1st argument of function {}, expected a Date, DateTime or DateTime64", + "Illegal type {} of 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64", type_arg1->getName(), getName()); value_is_date = isDate(type_arg1); }; @@ -56,6 +56,7 @@ public: enum class ResultType : uint8_t { Date, + Date32, DateTime, DateTime64 }; @@ -128,6 +129,8 @@ public: { case ResultType::Date: return std::make_shared(); + case ResultType::Date32: + return std::make_shared(); case ResultType::DateTime: return std::make_shared(extractTimeZoneNameFromFunctionArguments(arguments, 2, 0, false)); case ResultType::DateTime64: @@ -185,6 +188,12 @@ private: if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); } + else if (isDate32(time_column_type)) + { + const auto * time_column_vec = checkAndGetColumn(&time_column_col); + if (time_column_vec) + return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); + } throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); } diff --git a/tests/queries/0_stateless/00403_to_start_of_day.reference b/tests/queries/0_stateless/00403_to_start_of_day.reference index d00491fd7e5..1c96b0ed64f 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.reference +++ b/tests/queries/0_stateless/00403_to_start_of_day.reference @@ -1 +1,2 @@ 1 +2024-08-26 00:00:00 diff --git a/tests/queries/0_stateless/00403_to_start_of_day.sql b/tests/queries/0_stateless/00403_to_start_of_day.sql index e298afd80ba..2dfcfccec38 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.sql +++ b/tests/queries/0_stateless/00403_to_start_of_day.sql @@ -1 +1,2 @@ SELECT toStartOfDay(now()) = toDateTime(toDate(now())); +SELECT toStartOfDay(toDate32(now())); diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference index 70a26e5447d..066ca3cfd55 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference @@ -1,11 +1,16 @@ 2022-03-01 00:00:00 2022-03-01 2022-02-28 +2022-02-28 2022-03-01 00:00:00 2022-03-01 2022-02-28 +2022-02-28 2022-03-01 00:00:00 2022-03-01 +2022-03-01 00:00:00 +2022-03-01 +2022-02-28 2022-02-28 2022-03-01 12:12:12.012000000 2022-03-01 12:12:12.012346 diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql index df290f3deac..2e8e6dcd093 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql @@ -1,12 +1,17 @@ SELECT dateTrunc('DAY', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('MONTH', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('WEEK', toDate('2022-03-01')); +SELECT dateTrunc('WEEK', toDate32('2022-03-01')); SELECT dateTrunc('Day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('Month', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('Week', toDate('2022-03-01')); +SELECT dateTrunc('Week', toDate32('2022-03-01')); SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2)); +SELECT dateTrunc('day', toDate32('2022-03-01')); +SELECT dateTrunc('month', toDate32('2022-03-01')); SELECT dateTrunc('week', toDate('2022-03-01')); +SELECT dateTrunc('week', toDate32('2022-03-01')); SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3)); SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7)); SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9)); @@ -25,3 +30,6 @@ SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILL SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT dateTrunc('Nanosecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT dateTrunc('MicroSecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +SELECT dateTrunc('MILLISECOND', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } From 5e0000c4ca21d84756576ebb563fd9e1d98433e7 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Wed, 28 Aug 2024 16:06:09 +0800 Subject: [PATCH 123/154] add test case and doc --- .../functions/date-time-functions.md | 2 +- src/Functions/toStartOfInterval.cpp | 2 +- .../00403_to_start_of_day.reference | 1 - .../0_stateless/00403_to_start_of_day.sql | 1 - ..._date_trunc_case_unsensitiveness.reference | 5 --- .../02935_date_trunc_case_unsensitiveness.sql | 8 ----- ...d_to_start_of_interval_on_date32.reference | 33 +++++++++++++++++++ ...unc_and_to_start_of_interval_on_date32.sql | 26 +++++++++++++++ 8 files changed, 61 insertions(+), 17 deletions(-) create mode 100644 tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference create mode 100644 tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 3d95ae2cb74..b65fb3d7e95 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -2019,7 +2019,7 @@ Alias: `dateTrunc`. `unit` argument is case-insensitive. -- `value` — Date and time. [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md). +- `value` — Date and time. [Date](../data-types/date.md), [Date32](../data-types/date32.md), [DateTime](../data-types/datetime.md) or [DateTime64](../data-types/datetime64.md). - `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). If not specified, the function uses the timezone of the `value` parameter. [String](../data-types/string.md). **Returned value** diff --git a/src/Functions/toStartOfInterval.cpp b/src/Functions/toStartOfInterval.cpp index 35fb43df131..709f5f86d80 100644 --- a/src/Functions/toStartOfInterval.cpp +++ b/src/Functions/toStartOfInterval.cpp @@ -194,7 +194,7 @@ private: if (time_column_vec) return dispatchForIntervalColumn(assert_cast(time_column_type), *time_column_vec, interval_column, result_type, time_zone, input_rows_count); } - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, DateTime or DateTime64", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for 1st argument of function {}, expected a Date, Date32, DateTime or DateTime64", getName()); } template diff --git a/tests/queries/0_stateless/00403_to_start_of_day.reference b/tests/queries/0_stateless/00403_to_start_of_day.reference index 1c96b0ed64f..d00491fd7e5 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.reference +++ b/tests/queries/0_stateless/00403_to_start_of_day.reference @@ -1,2 +1 @@ 1 -2024-08-26 00:00:00 diff --git a/tests/queries/0_stateless/00403_to_start_of_day.sql b/tests/queries/0_stateless/00403_to_start_of_day.sql index 2dfcfccec38..e298afd80ba 100644 --- a/tests/queries/0_stateless/00403_to_start_of_day.sql +++ b/tests/queries/0_stateless/00403_to_start_of_day.sql @@ -1,2 +1 @@ SELECT toStartOfDay(now()) = toDateTime(toDate(now())); -SELECT toStartOfDay(toDate32(now())); diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference index 066ca3cfd55..70a26e5447d 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference @@ -1,16 +1,11 @@ 2022-03-01 00:00:00 2022-03-01 2022-02-28 -2022-02-28 2022-03-01 00:00:00 2022-03-01 2022-02-28 -2022-02-28 2022-03-01 00:00:00 2022-03-01 -2022-03-01 00:00:00 -2022-03-01 -2022-02-28 2022-02-28 2022-03-01 12:12:12.012000000 2022-03-01 12:12:12.012346 diff --git a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql index 2e8e6dcd093..df290f3deac 100644 --- a/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql +++ b/tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql @@ -1,17 +1,12 @@ SELECT dateTrunc('DAY', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('MONTH', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('WEEK', toDate('2022-03-01')); -SELECT dateTrunc('WEEK', toDate32('2022-03-01')); SELECT dateTrunc('Day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('Month', toDateTime64('2022-03-01 12:55:55', 2)); SELECT dateTrunc('Week', toDate('2022-03-01')); -SELECT dateTrunc('Week', toDate32('2022-03-01')); SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55')); SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2)); -SELECT dateTrunc('day', toDate32('2022-03-01')); -SELECT dateTrunc('month', toDate32('2022-03-01')); SELECT dateTrunc('week', toDate('2022-03-01')); -SELECT dateTrunc('week', toDate32('2022-03-01')); SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3)); SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7)); SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9)); @@ -30,6 +25,3 @@ SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILL SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT dateTrunc('Nanosecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT dateTrunc('MicroSecond', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } -SELECT dateTrunc('MILLISECOND', toDate32('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference new file mode 100644 index 00000000000..ea0c96ab2d2 --- /dev/null +++ b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.reference @@ -0,0 +1,33 @@ +-- { echoOn } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 YEAR); +2022-01-01 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 QUARTER); +2022-07-01 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MONTH); +2022-09-01 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 WEEK); +2022-09-12 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 DAY); +2022-09-16 00:00:00 +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 HOUR); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 SECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MILLISECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MICROSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 NANOSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('YEAR', toDate32('2022-09-16')); +2022-01-01 +select date_trunc('QUARTER', toDate32('2022-09-16')); +2022-07-01 +select date_trunc('MONTH', toDate32('2022-09-16')); +2022-09-01 +select date_trunc('WEEK', toDate32('2022-09-16')); +2022-09-12 +select date_trunc('DAY', toDate32('2022-09-16')); +2022-09-16 00:00:00 +select date_trunc('HOUR', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MINUTE', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('SECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MILLISECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MICROSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('NANOSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } diff --git a/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql new file mode 100644 index 00000000000..b2b6385f00b --- /dev/null +++ b/tests/queries/0_stateless/03230_date_trunc_and_to_start_of_interval_on_date32.sql @@ -0,0 +1,26 @@ +-- { echoOn } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 YEAR); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 QUARTER); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MONTH); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 WEEK); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 DAY); +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 HOUR); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MINUTE); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 SECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MILLISECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 MICROSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select toStartOfInterval(toDate32('2022-09-16'), INTERVAL 1 NANOSECOND); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + +select date_trunc('YEAR', toDate32('2022-09-16')); +select date_trunc('QUARTER', toDate32('2022-09-16')); +select date_trunc('MONTH', toDate32('2022-09-16')); +select date_trunc('WEEK', toDate32('2022-09-16')); +select date_trunc('DAY', toDate32('2022-09-16')); +select date_trunc('HOUR', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MINUTE', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('SECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MILLISECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('MICROSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } +select date_trunc('NANOSECOND', toDate32('2022-09-16')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT } + + From cd1685d409a481e7573bcacc73b61f886ca99101 Mon Sep 17 00:00:00 2001 From: liuneng <1398775315@qq.com> Date: Fri, 30 Aug 2024 13:55:37 +0800 Subject: [PATCH 124/154] fix error message --- src/Functions/date_trunc.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/date_trunc.cpp b/src/Functions/date_trunc.cpp index 7b094174047..15d50724506 100644 --- a/src/Functions/date_trunc.cpp +++ b/src/Functions/date_trunc.cpp @@ -85,7 +85,7 @@ public: if (second_argument_is_date && ((datepart_kind == IntervalKind::Kind::Hour) || (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))) - throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type Date of argument for function {}", getName()); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument for function {}", arguments[1].type->getName(), getName()); }; auto check_timezone_argument = [&] { From c702d2581e50d042eed7b8374464e3e979cde10d Mon Sep 17 00:00:00 2001 From: wxybear Date: Sat, 31 Aug 2024 14:50:35 +0800 Subject: [PATCH 125/154] fix: handle insert query in explain queries in multiquery mode --- src/Client/ClientBase.cpp | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 01d03006eec..6132f744098 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -2111,6 +2112,15 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( // - Other formats (e.g. FORMAT CSV) are arbitrarily more complex and tricky to parse. For example, we may be unable to distinguish if the semicolon // is part of the data or ends the statement. In this case, we simply assume that the end of the INSERT statement is determined by \n\n (two newlines). auto * insert_ast = parsed_query->as(); + // We also consider the INSERT query in EXPLAIN queries (same as normal INSERT queries) + if (!insert_ast) + { + auto * explain_ast = parsed_query->as(); + if (explain_ast) + { + insert_ast = explain_ast->getExplainedQuery()->as(); + } + } const char * query_to_execute_end = this_query_end; if (insert_ast && insert_ast->data) { From 199aaff7118ca04c8c7c32b9aaf9ea5cd660c700 Mon Sep 17 00:00:00 2001 From: sakulali Date: Sat, 31 Aug 2024 18:13:07 +0800 Subject: [PATCH 126/154] Kindly ping CI and try to reproduce failed CI tests From 5ef1830f559d565060d92ff9ccbfc9ebc11773f5 Mon Sep 17 00:00:00 2001 From: wxybear Date: Sat, 31 Aug 2024 18:39:01 +0800 Subject: [PATCH 127/154] feat: add explain ast insert queries test --- .../0_stateless/03156_default_multiquery_split.reference | 4 ++++ .../queries/0_stateless/03156_default_multiquery_split.sh | 7 +++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.reference b/tests/queries/0_stateless/03156_default_multiquery_split.reference index 0f3a1baff45..02a3a522ecd 100644 --- a/tests/queries/0_stateless/03156_default_multiquery_split.reference +++ b/tests/queries/0_stateless/03156_default_multiquery_split.reference @@ -8,3 +8,7 @@ Syntax error 7 8 9 +InsertQuery (children 1) + Identifier TEST2 +InsertQuery (children 1) + Identifier TEST1 diff --git a/tests/queries/0_stateless/03156_default_multiquery_split.sh b/tests/queries/0_stateless/03156_default_multiquery_split.sh index d849fb5a162..08ee9bcad63 100755 --- a/tests/queries/0_stateless/03156_default_multiquery_split.sh +++ b/tests/queries/0_stateless/03156_default_multiquery_split.sh @@ -51,6 +51,13 @@ INSERT INTO TEST2 VALUES SELECT * FROM TEST1 ORDER BY value; SELECT * FROM TEST2 ORDER BY value; DROP TABLE TEST1; DROP TABLE TEST2; + +EXPLAIN AST INSERT INTO TEST2 FORMAT CSV +1 +2 + +EXPLAIN AST INSERT INTO TEST1 VALUES (101),(102); + EOF $CLICKHOUSE_CLIENT -m < "$SQL_FILE_NAME" From a6145e509908eb1ad14c4952900676d92c458b90 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 31 Aug 2024 12:09:02 +0000 Subject: [PATCH 128/154] Fix: expression description in plan after lift up unioin optimization --- ...imize_distributed_group_by_sharding_key.reference | 4 ++-- .../02496_remove_redundant_sorting.reference | 12 ++++++------ ...02496_remove_redundant_sorting_analyzer.reference | 12 ++++++------ .../02500_remove_redundant_distinct.reference | 4 ++-- ...2500_remove_redundant_distinct_analyzer.reference | 4 ++-- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index e786532f25a..a807bf7096e 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -21,7 +21,7 @@ Expression (Projection) Union Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) ReadFromSystemNumbers - Expression + Expression (Before LIMIT BY) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union @@ -96,7 +96,7 @@ Expression (Project names) LimitBy Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))) ReadFromSystemNumbers - Expression + Expression (Before LIMIT BY) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 4d004f2f78f..7824fd8cba9 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -395,9 +395,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 @@ -427,9 +427,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Nullable(Float64) 9007199254740994 @@ -459,9 +459,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index dd5ac7bf706..3c68d14fdf2 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -394,9 +394,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 @@ -426,9 +426,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Nullable(Float64) 9007199254740994 @@ -458,9 +458,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference index d7623cd5541..9bb8f4a4017 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference @@ -53,7 +53,7 @@ Expression (Projection) Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) ReadFromSystemNumbers - Expression (( + Projection)) + Expression ((Before ORDER BY + Projection)) Distinct Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) @@ -536,7 +536,7 @@ Expression (Projection) Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) ReadFromSystemNumbers - Expression (( + Projection)) + Expression ((Before ORDER BY + Projection)) Distinct Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index b79f6310166..27b01cf1158 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -54,7 +54,7 @@ Expression (Project names) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) ReadFromSystemNumbers - Expression (( + ( + Project names))) + Expression ((Projection + (Change column names to column identifiers + Project names))) Distinct (DISTINCT) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) @@ -542,7 +542,7 @@ Expression (Project names) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) ReadFromSystemNumbers - Expression (( + ( + Project names))) + Expression ((Projection + (Change column names to column identifiers + Project names))) Distinct (DISTINCT) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) From 425a93e36040d66dc6b2a0fa4bf2ac50fbe8d876 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 31 Aug 2024 12:12:32 +0000 Subject: [PATCH 129/154] Fix --- src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index c48551732c9..43cf166002e 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -50,6 +50,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), expression->getExpression().clone()); + expr_node.step->setStepDescription(expression->getStepDescription()); } /// - Expression - Something From 0857a10fc1c9f0863a0059bfb0961c131a0c75a9 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Sat, 31 Aug 2024 14:17:24 +0200 Subject: [PATCH 130/154] Fix types in JSON docs --- docs/en/sql-reference/data-types/newjson.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/data-types/newjson.md b/docs/en/sql-reference/data-types/newjson.md index f7fc7e1498e..8218ba89176 100644 --- a/docs/en/sql-reference/data-types/newjson.md +++ b/docs/en/sql-reference/data-types/newjson.md @@ -453,8 +453,8 @@ As we can see, after inserting paths `e` and `f.g` the limit was reached and we ### During merges of data parts in MergeTree table engines -During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths won't be able to store all paths from source parts as subcolumns. -In this case ClickHouse chooses what paths will remain as subcolumns after merge and what types will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contains +During merge of several data parts in MergeTree table the `JSON` column in the resulting data part can reach the limit of dynamic paths and won't be able to store all paths from source parts as subcolumns. +In this case ClickHouse chooses what paths will remain as subcolumns after merge and what paths will be stored in the shared data structure. In most cases ClickHouse tries to keep paths that contain the largest number of non-null values and move the rarest paths to the shared data structure, but it depends on the implementation. Let's see an example of such merge. First, let's create a table with `JSON` column, set the limit of dynamic paths to `3` and insert values with `5` different paths: From b066760a21cdefbabae5172624ca8c65d5200f70 Mon Sep 17 00:00:00 2001 From: wxybear Date: Sat, 31 Aug 2024 22:18:33 +0800 Subject: [PATCH 131/154] fix: explainedQuery nullptr && 02504_explain_ast_insert --- src/Client/ClientBase.cpp | 2 +- tests/queries/0_stateless/02504_explain_ast_insert.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 6132f744098..e34e263beb5 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2116,7 +2116,7 @@ MultiQueryProcessingStage ClientBase::analyzeMultiQueryText( if (!insert_ast) { auto * explain_ast = parsed_query->as(); - if (explain_ast) + if (explain_ast && explain_ast->getExplainedQuery()) { insert_ast = explain_ast->getExplainedQuery()->as(); } diff --git a/tests/queries/0_stateless/02504_explain_ast_insert.sql b/tests/queries/0_stateless/02504_explain_ast_insert.sql index fc50feebaa4..3b8a64e6ea2 100644 --- a/tests/queries/0_stateless/02504_explain_ast_insert.sql +++ b/tests/queries/0_stateless/02504_explain_ast_insert.sql @@ -1,2 +1,2 @@ -explain ast insert into test values balabala; +explain ast insert into test values (balabala); explain ast insert into test format TabSeparated balabala; \ No newline at end of file From 6fd7656aebd0d747ddd06a069139cb935031492e Mon Sep 17 00:00:00 2001 From: m4xxx1m Date: Sat, 31 Aug 2024 18:22:03 +0300 Subject: [PATCH 132/154] Added Poco::Util::AbstractConfiguration::getHost method --- .../include/Poco/Util/AbstractConfiguration.h | 40 +++++ base/poco/Util/src/AbstractConfiguration.cpp | 153 ++++++++++++++++++ 2 files changed, 193 insertions(+) diff --git a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h index 926ac3ba8a9..7f58df905c0 100644 --- a/base/poco/Util/include/Poco/Util/AbstractConfiguration.h +++ b/base/poco/Util/include/Poco/Util/AbstractConfiguration.h @@ -241,6 +241,20 @@ namespace Util /// If the value contains references to other properties (${}), these /// are expanded. + std::string getHost(const std::string & key) const; + /// Returns the string value of the host property with the given name. + /// Throws a NotFoundException if the key does not exist. + /// Throws a SyntaxException if the property is not a valid host (IP address or domain). + /// If the value contains references to other properties (${}), these + /// are expanded. + + std::string getHost(const std::string & key, const std::string & defaultValue) const; + /// If a property with the given key exists, returns the host property's string value, + /// otherwise returns the given default value. + /// Throws a SyntaxException if the property is not a valid host (IP address or domain). + /// If the value contains references to other properties (${}), these + /// are expanded. + virtual void setString(const std::string & key, const std::string & value); /// Sets the property with the given key to the given value. /// An already existing value for the key is overwritten. @@ -339,12 +353,38 @@ namespace Util static bool parseBool(const std::string & value); void setRawWithEvent(const std::string & key, std::string value); + static void checkHostValidity(const std::string & value); + /// Throws a SyntaxException if the value is not a valid host (IP address or domain). + virtual ~AbstractConfiguration(); private: std::string internalExpand(const std::string & value) const; std::string uncheckedExpand(const std::string & value) const; + static bool isValidIPv4Address(const std::string & value); + /// A string value is considered to be a valid IPv4 address if it matches + /// "x1.x2.x3.x4", where xi - integer in range 0..255 and may have leading zeroes + + static bool isValidIPv6Address(const std::string & value); + /// A string value is considered to be a valid IPv6 address if it matches + /// "x1:x2:x3:x4:x5:x6:x7:x8", where xi is hexadecimal integer and consist of 4 + /// characters or less (but at least 1), xi may have leading zeroes. + /// Letters in hexadecimal representation can be in upper case or lower case. + /// One or more consecutive hextets of zeroes can be replaced with "::", but + /// "::" can appear only once in a valid IPv6 address. + + static bool isValidDomainName(const std::string & value); + /// ::= [ "." ] + /// ::=