From 7d81ecb1835e7818020ad795e63d20372b6bf9ce Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 02:16:50 +0200 Subject: [PATCH 01/91] Parallel compression --- src/Common/CurrentMetrics.cpp | 4 + src/Compression/CompressedWriteBuffer.cpp | 1 - src/Compression/ICompressionCodec.cpp | 11 ++ .../ParallelCompressedWriteBuffer.cpp | 118 ++++++++++++++++++ .../ParallelCompressedWriteBuffer.h | 87 +++++++++++++ 5 files changed, 220 insertions(+), 1 deletion(-) create mode 100644 src/Compression/ParallelCompressedWriteBuffer.cpp create mode 100644 src/Compression/ParallelCompressedWriteBuffer.h diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index bd62e7e8aae..da3b5557dbf 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -41,6 +41,10 @@ M(PostgreSQLConnection, "Number of client connections using PostgreSQL protocol") \ M(OpenFileForRead, "Number of files open for reading") \ M(OpenFileForWrite, "Number of files open for writing") \ + M(Compressing, "Number of compress operations using internal compression codecs") \ + M(Decompressing, "Number of decompress operations using internal compression codecs") \ + M(ParallelCompressedWriteBufferThreads, "Number of threads in all instances of ParallelCompressedWriteBuffer - these threads are doing parallel compression and writing") \ + M(ParallelCompressedWriteBufferWait, "Number of threads in all instances of ParallelCompressedWriteBuffer that are currently waiting for buffer to become available for writing") \ M(TotalTemporaryFiles, "Number of temporary files created") \ M(TemporaryFilesForSort, "Number of temporary files created for external sorting") \ M(TemporaryFilesForAggregation, "Number of temporary files created for external aggregation") \ diff --git a/src/Compression/CompressedWriteBuffer.cpp b/src/Compression/CompressedWriteBuffer.cpp index c3acfcb7da6..b6dab2a190e 100644 --- a/src/Compression/CompressedWriteBuffer.cpp +++ b/src/Compression/CompressedWriteBuffer.cpp @@ -2,7 +2,6 @@ #include #include -#include #include #include diff --git a/src/Compression/ICompressionCodec.cpp b/src/Compression/ICompressionCodec.cpp index 418667a3a8f..a31d0485982 100644 --- a/src/Compression/ICompressionCodec.cpp +++ b/src/Compression/ICompressionCodec.cpp @@ -5,11 +5,18 @@ #include #include #include +#include #include #include #include +namespace CurrentMetrics +{ + extern const Metric Compressing; + extern const Metric Decompressing; +} + namespace DB { @@ -80,6 +87,8 @@ UInt32 ICompressionCodec::compress(const char * source, UInt32 source_size, char { assert(source != nullptr && dest != nullptr); + CurrentMetrics::Increment metric_increment(CurrentMetrics::Compressing); + dest[0] = getMethodByte(); UInt8 header_size = getHeaderSize(); /// Write data from header_size @@ -93,6 +102,8 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch { assert(source != nullptr && dest != nullptr); + CurrentMetrics::Increment metric_increment(CurrentMetrics::Decompressing); + UInt8 header_size = getHeaderSize(); if (source_size < header_size) throw Exception(decompression_error_code, diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp new file mode 100644 index 00000000000..270c331e4df --- /dev/null +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -0,0 +1,118 @@ +#include +#include + +#include +#include + +#include +#include +#include +#include +#include + +#include + + +namespace CurrentMetrics +{ + extern const Metric ParallelCompressedWriteBufferThreads; + extern const Metric ParallelCompressedWriteBufferWait; +} + +namespace DB +{ + +ParallelCompressedWriteBuffer::ParallelCompressedWriteBuffer( + WriteBuffer & out_, + CompressionCodecPtr codec_, + size_t buf_size_, + size_t num_threads_, + ThreadPool & pool_) + : WriteBuffer(nullptr, 0), out(out_), codec(codec_), buf_size(buf_size_), num_threads(num_threads_), pool(pool_) +{ + buffers.emplace_back(buf_size); + current_buffer = buffers.begin(); + BufferBase::set(current_buffer->uncompressed.data(), buf_size, 0); +} + +void ParallelCompressedWriteBuffer::nextImpl() +{ + if (!offset()) + return; + + std::unique_lock lock(mutex); + + /// The buffer will be compressed and processed in the thread. + current_buffer->busy = true; + pool.trySchedule([this, my_current_buffer = current_buffer, thread_group = CurrentThread::getGroup()] + { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + + if (thread_group) + CurrentThread::attachToGroupIfDetached(thread_group); + setThreadName("ParallelCompres"); + + compress(my_current_buffer); + }); + + const BufferPair * previous_buffer = &*current_buffer; + ++current_buffer; + if (current_buffer == buffers.end()) + { + if (buffers.size() < num_threads) + { + /// If we didn't use all num_threads buffers yet, create a new one. + current_buffer = buffers.emplace(current_buffer, buf_size); + } + else + { + /// Otherwise, wrap around to the first buffer in the list. + current_buffer = buffers.begin(); + } + } + + /// Wait while the buffer becomes not busy + { + CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); + cond.wait(lock, [&]{ return !current_buffer->busy; }); + } + + /// Now this buffer can be used. + current_buffer->previous = previous_buffer; + BufferBase::set(current_buffer->uncompressed.data(), buf_size, 0); +} + +void ParallelCompressedWriteBuffer::compress(Iterator buffer) +{ + CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferThreads); + + chassert(offset() <= INT_MAX); + UInt32 decompressed_size = static_cast(offset()); + UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); + + buffer->compressed.resize(compressed_reserve_size); + UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, buffer->compressed.data()); + + CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(buffer->compressed.data(), compressed_size); + + /// Wait while all previous buffers have been written. + { + CurrentMetrics::Increment metric_wait_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); + std::unique_lock lock(mutex); + cond.wait(lock, [&]{ return !buffer->previous || !buffer->previous->busy; }); + } + + writeBinaryLittleEndian(checksum.low64, out); + writeBinaryLittleEndian(checksum.high64, out); + + out.write(buffer->compressed.data(), compressed_size); + + std::unique_lock lock(mutex); + buffer->busy = false; + cond.notify_all(); +} + +} diff --git a/src/Compression/ParallelCompressedWriteBuffer.h b/src/Compression/ParallelCompressedWriteBuffer.h new file mode 100644 index 00000000000..e824dcacb46 --- /dev/null +++ b/src/Compression/ParallelCompressedWriteBuffer.h @@ -0,0 +1,87 @@ +#pragma once + +#include +#include + +#include + +#include +#include +#include +#include +#include + + +namespace DB +{ + +/** Uses multi-buffering for parallel compression. + * When the buffer is filled, it will be compressed in the background, + * and a new buffer is created for the next input data. + */ +class ParallelCompressedWriteBuffer final : public WriteBuffer +{ +public: + explicit ParallelCompressedWriteBuffer( + WriteBuffer & out_, + CompressionCodecPtr codec_, + size_t buf_size_, + size_t num_threads_, + ThreadPool & pool_); + + ~ParallelCompressedWriteBuffer() override; + + /// The amount of compressed data + size_t getCompressedBytes() + { + nextIfAtEnd(); + return out.count(); + } + + /// How many uncompressed bytes were written to the buffer + size_t getUncompressedBytes() + { + return count(); + } + + /// How many bytes are in the buffer (not yet compressed) + size_t getRemainingBytes() + { + nextIfAtEnd(); + return offset(); + } + +private: + void nextImpl() override; + void finalizeImpl() override; + + WriteBuffer & out; + CompressionCodecPtr codec; + size_t buf_size; + size_t num_threads; + ThreadPool & pool; + + struct BufferPair + { + BufferPair(size_t input_size) + : uncompressed(input_size) + { + } + + Memory<> uncompressed; + PODArray compressed; + const BufferPair * previous = nullptr; + bool busy = false; + }; + + std::mutex mutex; + std::condition_variable cond; + std::list buffers; + + using Iterator = std::list::iterator; + Iterator current_buffer; + + void compress(Iterator buffer); +}; + +} From 5e433ea537d42aca8fa1076f7c054b1b3dc83854 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 03:11:16 +0200 Subject: [PATCH 02/91] Parallel compression: development --- programs/compressor/Compressor.cpp | 40 ++++++++++++++++--- .../ParallelCompressedWriteBuffer.cpp | 38 +++++++++++++++--- .../ParallelCompressedWriteBuffer.h | 3 ++ 3 files changed, 69 insertions(+), 12 deletions(-) diff --git a/programs/compressor/Compressor.cpp b/programs/compressor/Compressor.cpp index 050bb495024..aac56fba94a 100644 --- a/programs/compressor/Compressor.cpp +++ b/programs/compressor/Compressor.cpp @@ -9,6 +9,7 @@ #include #include #include +#include #include #include #include @@ -17,6 +18,8 @@ #include #include #include +#include +#include #include @@ -29,6 +32,13 @@ namespace DB } } +namespace CurrentMetrics +{ + extern const Metric LocalThread; + extern const Metric LocalThreadActive; + extern const Metric LocalThreadScheduled; +} + namespace { @@ -77,12 +87,13 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) ("decompress,d", "decompress") ("offset-in-compressed-file", po::value()->default_value(0ULL), "offset to the compressed block (i.e. physical file offset)") ("offset-in-decompressed-block", po::value()->default_value(0ULL), "offset to the decompressed block (i.e. virtual offset)") - ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") + ("block-size,b", po::value()->default_value(DBMS_DEFAULT_BUFFER_SIZE), "compress in blocks of specified size") ("hc", "use LZ4HC instead of LZ4") ("zstd", "use ZSTD instead of LZ4") ("deflate_qpl", "use deflate_qpl instead of LZ4") ("codec", po::value>()->multitoken(), "use codecs combination instead of LZ4") ("level", po::value(), "compression level for codecs specified via flags") + ("threads", po::value()->default_value(1), "number of threads for parallel compression") ("none", "use no compression instead of LZ4") ("stat", "print block statistics of compressed data") ("stacktrace", "print stacktrace of exception") @@ -111,7 +122,8 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) bool stat_mode = options.count("stat"); bool use_none = options.count("none"); print_stacktrace = options.count("stacktrace"); - unsigned block_size = options["block-size"].as(); + size_t block_size = options["block-size"].as(); + size_t num_threads = options["threads"].as(); std::vector codecs; if (options.count("codec")) codecs = options["codec"].as>(); @@ -119,6 +131,12 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) if ((use_lz4hc || use_zstd || use_deflate_qpl || use_none) && !codecs.empty()) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong options, codec flags like --zstd and --codec options are mutually exclusive"); + if (num_threads < 1) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Invalid value of `threads` parameter"); + + if (num_threads > 1 && decompress) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Parallel mode is only implemented for compression (not for decompression)"); + if (!codecs.empty() && options.count("level")) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Wrong options, --level is not compatible with --codec list"); @@ -149,7 +167,6 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else codec = CompressionCodecFactory::instance().get(method_family, level); - std::unique_ptr rb; std::unique_ptr wb; @@ -190,9 +207,20 @@ int mainEntryClickHouseCompressor(int argc, char ** argv) else { /// Compression - CompressedWriteBuffer to(*wb, codec, block_size); - copyData(*rb, to); - to.finalize(); + + if (num_threads == 1) + { + CompressedWriteBuffer to(*wb, codec, block_size); + copyData(*rb, to); + to.finalize(); + } + else + { + ThreadPool pool(CurrentMetrics::LocalThread, CurrentMetrics::LocalThreadActive, CurrentMetrics::LocalThreadScheduled, num_threads); + ParallelCompressedWriteBuffer to(*wb, codec, block_size, num_threads, pool); + copyData(*rb, to); + to.finalize(); + } } } catch (...) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 270c331e4df..4ffb6056d18 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -1,8 +1,9 @@ #include -#include +#include #include #include +#include #include #include @@ -30,6 +31,7 @@ ParallelCompressedWriteBuffer::ParallelCompressedWriteBuffer( ThreadPool & pool_) : WriteBuffer(nullptr, 0), out(out_), codec(codec_), buf_size(buf_size_), num_threads(num_threads_), pool(pool_) { + std::cerr << getThreadId() << " Create a new buffer 1\n"; buffers.emplace_back(buf_size); current_buffer = buffers.begin(); BufferBase::set(current_buffer->uncompressed.data(), buf_size, 0); @@ -44,6 +46,9 @@ void ParallelCompressedWriteBuffer::nextImpl() /// The buffer will be compressed and processed in the thread. current_buffer->busy = true; + current_buffer->sequence_num = current_sequence_num; + ++current_sequence_num; + current_buffer->uncompressed_size = offset(); pool.trySchedule([this, my_current_buffer = current_buffer, thread_group = CurrentThread::getGroup()] { SCOPE_EXIT_SAFE( @@ -65,15 +70,19 @@ void ParallelCompressedWriteBuffer::nextImpl() if (buffers.size() < num_threads) { /// If we didn't use all num_threads buffers yet, create a new one. + std::cerr << getThreadId() << " Create a new buffer " << (buffers.size() + 1) << "\n"; current_buffer = buffers.emplace(current_buffer, buf_size); } else { /// Otherwise, wrap around to the first buffer in the list. + std::cerr << getThreadId() << " Wrap around\n"; current_buffer = buffers.begin(); } } + if (current_buffer->busy) + std::cerr << getThreadId() << " Wait while the buffer " << current_buffer->sequence_num << " becomes not busy\n"; /// Wait while the buffer becomes not busy { CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); @@ -85,26 +94,37 @@ void ParallelCompressedWriteBuffer::nextImpl() BufferBase::set(current_buffer->uncompressed.data(), buf_size, 0); } +void ParallelCompressedWriteBuffer::finalizeImpl() +{ + next(); + pool.wait(); +} + void ParallelCompressedWriteBuffer::compress(Iterator buffer) { + std::cerr << getThreadId() << " Compressing " << buffer->sequence_num << "...\n"; CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferThreads); - chassert(offset() <= INT_MAX); - UInt32 decompressed_size = static_cast(offset()); - UInt32 compressed_reserve_size = codec->getCompressedReserveSize(decompressed_size); + chassert(buffer->uncompressed_size <= INT_MAX); + UInt32 uncompressed_size = static_cast(buffer->uncompressed_size); + UInt32 compressed_reserve_size = codec->getCompressedReserveSize(uncompressed_size); buffer->compressed.resize(compressed_reserve_size); - UInt32 compressed_size = codec->compress(working_buffer.begin(), decompressed_size, buffer->compressed.data()); + UInt32 compressed_size = codec->compress(buffer->uncompressed.data(), uncompressed_size, buffer->compressed.data()); CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(buffer->compressed.data(), compressed_size); + if (buffer->previous && buffer->previous->busy) + std::cerr << getThreadId() << " Compressed " << buffer->sequence_num << ", waiting for prev buffer to be written...\n"; /// Wait while all previous buffers have been written. { CurrentMetrics::Increment metric_wait_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); std::unique_lock lock(mutex); - cond.wait(lock, [&]{ return !buffer->previous || !buffer->previous->busy; }); + cond.wait(lock, [&]{ return !buffer->previous || !buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num; }); } + std::cerr << getThreadId() << " Writing " << buffer->sequence_num << "...\n"; + writeBinaryLittleEndian(checksum.low64, out); writeBinaryLittleEndian(checksum.high64, out); @@ -115,4 +135,10 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) cond.notify_all(); } +ParallelCompressedWriteBuffer::~ParallelCompressedWriteBuffer() +{ + if (!canceled) + finalize(); +} + } diff --git a/src/Compression/ParallelCompressedWriteBuffer.h b/src/Compression/ParallelCompressedWriteBuffer.h index e824dcacb46..ade49837f6b 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.h +++ b/src/Compression/ParallelCompressedWriteBuffer.h @@ -69,8 +69,10 @@ private: } Memory<> uncompressed; + size_t uncompressed_size = 0; PODArray compressed; const BufferPair * previous = nullptr; + size_t sequence_num = 0; bool busy = false; }; @@ -80,6 +82,7 @@ private: using Iterator = std::list::iterator; Iterator current_buffer; + size_t current_sequence_num = 0; void compress(Iterator buffer); }; From 7229ffd507db02a2b4c2a468ce1ac3dfcff26901 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 03:15:10 +0200 Subject: [PATCH 03/91] Parallel compression: development --- src/Compression/ParallelCompressedWriteBuffer.cpp | 12 ------------ 1 file changed, 12 deletions(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 4ffb6056d18..30eaba33570 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -1,9 +1,7 @@ #include -#include #include #include -#include #include #include @@ -31,7 +29,6 @@ ParallelCompressedWriteBuffer::ParallelCompressedWriteBuffer( ThreadPool & pool_) : WriteBuffer(nullptr, 0), out(out_), codec(codec_), buf_size(buf_size_), num_threads(num_threads_), pool(pool_) { - std::cerr << getThreadId() << " Create a new buffer 1\n"; buffers.emplace_back(buf_size); current_buffer = buffers.begin(); BufferBase::set(current_buffer->uncompressed.data(), buf_size, 0); @@ -70,19 +67,15 @@ void ParallelCompressedWriteBuffer::nextImpl() if (buffers.size() < num_threads) { /// If we didn't use all num_threads buffers yet, create a new one. - std::cerr << getThreadId() << " Create a new buffer " << (buffers.size() + 1) << "\n"; current_buffer = buffers.emplace(current_buffer, buf_size); } else { /// Otherwise, wrap around to the first buffer in the list. - std::cerr << getThreadId() << " Wrap around\n"; current_buffer = buffers.begin(); } } - if (current_buffer->busy) - std::cerr << getThreadId() << " Wait while the buffer " << current_buffer->sequence_num << " becomes not busy\n"; /// Wait while the buffer becomes not busy { CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); @@ -102,7 +95,6 @@ void ParallelCompressedWriteBuffer::finalizeImpl() void ParallelCompressedWriteBuffer::compress(Iterator buffer) { - std::cerr << getThreadId() << " Compressing " << buffer->sequence_num << "...\n"; CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferThreads); chassert(buffer->uncompressed_size <= INT_MAX); @@ -114,8 +106,6 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(buffer->compressed.data(), compressed_size); - if (buffer->previous && buffer->previous->busy) - std::cerr << getThreadId() << " Compressed " << buffer->sequence_num << ", waiting for prev buffer to be written...\n"; /// Wait while all previous buffers have been written. { CurrentMetrics::Increment metric_wait_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); @@ -123,8 +113,6 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) cond.wait(lock, [&]{ return !buffer->previous || !buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num; }); } - std::cerr << getThreadId() << " Writing " << buffer->sequence_num << "...\n"; - writeBinaryLittleEndian(checksum.low64, out); writeBinaryLittleEndian(checksum.high64, out); From 66024821cf591d790ba1017b25dc4ebd75e8ea41 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 03:23:07 +0200 Subject: [PATCH 04/91] Parallel compression: development --- src/Compression/ParallelCompressedWriteBuffer.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 30eaba33570..954fae242e4 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -77,6 +77,7 @@ void ParallelCompressedWriteBuffer::nextImpl() } /// Wait while the buffer becomes not busy + if (current_buffer->busy) { CurrentMetrics::Increment metric_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); cond.wait(lock, [&]{ return !current_buffer->busy; }); @@ -107,10 +108,11 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(buffer->compressed.data(), compressed_size); /// Wait while all previous buffers have been written. + if (buffer->previous) { CurrentMetrics::Increment metric_wait_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); std::unique_lock lock(mutex); - cond.wait(lock, [&]{ return !buffer->previous || !buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num; }); + cond.wait(lock, [&]{ return !buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num; }); } writeBinaryLittleEndian(checksum.low64, out); From d6e0da177744a9890ff269aad9168725f6a717c3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 03:33:42 +0200 Subject: [PATCH 05/91] Less memcpy --- .../ParallelCompressedWriteBuffer.cpp | 54 +++++++++++++++---- 1 file changed, 43 insertions(+), 11 deletions(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 954fae242e4..1041a14979e 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -102,23 +102,55 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) UInt32 uncompressed_size = static_cast(buffer->uncompressed_size); UInt32 compressed_reserve_size = codec->getCompressedReserveSize(uncompressed_size); - buffer->compressed.resize(compressed_reserve_size); - UInt32 compressed_size = codec->compress(buffer->uncompressed.data(), uncompressed_size, buffer->compressed.data()); + /// If all previous buffers have been written, + /// and if the output buffer has the required capacity, + /// we can compress data directly into the output buffer. + size_t required_out_capacity = compressed_reserve_size + sizeof(CityHash_v1_0_2::uint128); + bool can_write_directly = false; - CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(buffer->compressed.data(), compressed_size); - - /// Wait while all previous buffers have been written. - if (buffer->previous) + if (!buffer->previous) + { + can_write_directly = out.available() >= required_out_capacity; + } + else { - CurrentMetrics::Increment metric_wait_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); std::unique_lock lock(mutex); - cond.wait(lock, [&]{ return !buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num; }); + can_write_directly = (!buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num) + && out.available() >= required_out_capacity; } - writeBinaryLittleEndian(checksum.low64, out); - writeBinaryLittleEndian(checksum.high64, out); + if (can_write_directly) + { + char * out_compressed_ptr = out.position() + sizeof(CityHash_v1_0_2::uint128); + UInt32 compressed_size = codec->compress(working_buffer.begin(), uncompressed_size, out_compressed_ptr); - out.write(buffer->compressed.data(), compressed_size); + CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(out_compressed_ptr, compressed_size); + + writeBinaryLittleEndian(checksum.low64, out); + writeBinaryLittleEndian(checksum.high64, out); + + out.position() += compressed_size; + } + else + { + buffer->compressed.resize(compressed_reserve_size); + UInt32 compressed_size = codec->compress(buffer->uncompressed.data(), uncompressed_size, buffer->compressed.data()); + + CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(buffer->compressed.data(), compressed_size); + + /// Wait while all previous buffers have been written. + if (buffer->previous) + { + CurrentMetrics::Increment metric_wait_increment(CurrentMetrics::ParallelCompressedWriteBufferWait); + std::unique_lock lock(mutex); + cond.wait(lock, [&]{ return !buffer->previous->busy || buffer->previous->sequence_num > buffer->sequence_num; }); + } + + writeBinaryLittleEndian(checksum.low64, out); + writeBinaryLittleEndian(checksum.high64, out); + + out.write(buffer->compressed.data(), compressed_size); + } std::unique_lock lock(mutex); buffer->busy = false; From b03d055aab13526bd25b3ea63017f02314cd8f67 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 18:19:16 +0200 Subject: [PATCH 06/91] Fix clang-tidy --- src/Compression/ParallelCompressedWriteBuffer.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.h b/src/Compression/ParallelCompressedWriteBuffer.h index ade49837f6b..4d1dfc79797 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.h +++ b/src/Compression/ParallelCompressedWriteBuffer.h @@ -63,7 +63,7 @@ private: struct BufferPair { - BufferPair(size_t input_size) + explicit BufferPair(size_t input_size) : uncompressed(input_size) { } From 8f038e2e1cd3c6635f328c7bbece1573c225dfb3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 23:08:22 +0200 Subject: [PATCH 07/91] Preparation --- .../MergeTree/MergeTreeDataPartWriterOnDisk.cpp | 10 +++++----- src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h | 5 +++-- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 58a67fc4ba2..a006e2da368 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -34,7 +34,7 @@ void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() /// Also the order is important compressed_hashing.finalize(); - compressor.finalize(); + compressor->finalize(); plain_hashing.finalize(); if constexpr (!only_plain_file) @@ -92,8 +92,8 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( marks_file_extension{marks_file_extension_}, plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), plain_hashing(*plain_file), - compressor(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), - compressed_hashing(compressor), + compressor(std::make_unique(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), + compressed_hashing(*compressor), marks_file(data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings)), marks_hashing(*marks_file), marks_compressor(marks_hashing, marks_compression_codec_, marks_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), @@ -115,8 +115,8 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( data_file_extension{data_file_extension_}, plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), plain_hashing(*plain_file), - compressor(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), - compressed_hashing(compressor), + compressor(std::make_unique(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), + compressed_hashing(*compressor), compress_marks(false) { } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 8d84442981e..3b6687dff99 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -44,7 +44,7 @@ public: /// Helper class, which holds chain of buffers to write data file with marks. /// It is used to write: one column, skip index or all columns (in compact format). - template + template struct Stream { Stream( @@ -76,7 +76,8 @@ public: /// compressed_hashing -> compressor -> plain_hashing -> plain_file std::unique_ptr plain_file; HashingWriteBuffer plain_hashing; - CompressedWriteBuffer compressor; + /// This could be either CompressedWriteBuffer or ParallelCompressedWriteBuffer + std::unique_ptr compressor; HashingWriteBuffer compressed_hashing; /// marks_compressed_hashing -> marks_compressor -> marks_hashing -> marks_file From 1236422559c5fd957f99e65f08e25d7a9806190e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 20 Oct 2024 23:28:23 +0200 Subject: [PATCH 08/91] Templates are shit --- .../MergeTreeDataPartWriterOnDisk.cpp | 83 +++++++++---------- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 12 ++- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 4 +- 3 files changed, 44 insertions(+), 55 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index a006e2da368..c250726aba1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -25,8 +25,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -template -void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() +void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() { /// Here the main goal is to do preFinalize calls for plain_file and marks_file /// Before that all hashing and compression buffers have to be finalized @@ -37,45 +36,42 @@ void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() compressor->finalize(); plain_hashing.finalize(); - if constexpr (!only_plain_file) + if (marks_hashing) { if (compress_marks) { - marks_compressed_hashing.finalize(); - marks_compressor.finalize(); + marks_compressed_hashing->finalize(); + marks_compressor->finalize(); } - marks_hashing.finalize(); + marks_hashing->finalize(); } plain_file->preFinalize(); - if constexpr (!only_plain_file) + if (marks_file) marks_file->preFinalize(); is_prefinalized = true; } -template -void MergeTreeDataPartWriterOnDisk::Stream::finalize() +void MergeTreeDataPartWriterOnDisk::Stream::finalize() { if (!is_prefinalized) preFinalize(); plain_file->finalize(); - if constexpr (!only_plain_file) + if (marks_file) marks_file->finalize(); } -template -void MergeTreeDataPartWriterOnDisk::Stream::sync() const +void MergeTreeDataPartWriterOnDisk::Stream::sync() const { plain_file->sync(); - if constexpr (!only_plain_file) + if (marks_file) marks_file->sync(); } -template<> -MergeTreeDataPartWriterOnDisk::Stream::Stream( +MergeTreeDataPartWriterOnDisk::Stream::Stream( const String & escaped_column_name_, const MutableDataPartStoragePtr & data_part_storage, const String & data_path_, @@ -94,16 +90,15 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( plain_hashing(*plain_file), compressor(std::make_unique(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), compressed_hashing(*compressor), - marks_file(data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings)), - marks_hashing(*marks_file), - marks_compressor(marks_hashing, marks_compression_codec_, marks_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), - marks_compressed_hashing(marks_compressor), compress_marks(MarkType(marks_file_extension).compressed) { + marks_file = data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings); + marks_hashing.emplace(*marks_file); + marks_compressor.emplace(*marks_hashing, marks_compression_codec_, marks_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size); + marks_compressed_hashing.emplace(*marks_compressor); } -template<> -MergeTreeDataPartWriterOnDisk::Stream::Stream( +MergeTreeDataPartWriterOnDisk::Stream::Stream( const String & escaped_column_name_, const MutableDataPartStoragePtr & data_part_storage, const String & data_path_, @@ -121,8 +116,7 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( { } -template -void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) +void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) { String name = escaped_column_name; @@ -132,17 +126,17 @@ void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(Merg checksums.files[name + data_file_extension].file_size = plain_hashing.count(); checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash(); - if constexpr (!only_plain_file) + if (marks_hashing) { if (compress_marks) { checksums.files[name + marks_file_extension].is_compressed = true; - checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing.count(); - checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing.getHash(); + checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing->count(); + checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing->getHash(); } - checksums.files[name + marks_file_extension].file_size = marks_hashing.count(); - checksums.files[name + marks_file_extension].file_hash = marks_hashing.getHash(); + checksums.files[name + marks_file_extension].file_size = marks_hashing->count(); + checksums.files[name + marks_file_extension].file_hash = marks_hashing->getHash(); } } @@ -276,12 +270,12 @@ void MergeTreeDataPartWriterOnDisk::initStatistics() for (const auto & stat_ptr : stats) { String stats_name = stat_ptr->getFileName(); - stats_streams.emplace_back(std::make_unique>( - stats_name, - data_part_storage, - stats_name, STATS_FILE_SUFFIX, - default_codec, settings.max_compress_block_size, - settings.query_write_settings)); + stats_streams.emplace_back(std::make_unique( + stats_name, + data_part_storage, + stats_name, STATS_FILE_SUFFIX, + default_codec, settings.max_compress_block_size, + settings.query_write_settings)); } } @@ -298,14 +292,14 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() { String stream_name = skip_index->getFileName(); skip_indices_streams.emplace_back( - std::make_unique>( - stream_name, - data_part_storage, - stream_name, skip_index->getSerializedFileExtension(), - stream_name, marks_file_extension, - default_codec, settings.max_compress_block_size, - marks_compression_codec, settings.marks_compress_block_size, - settings.query_write_settings)); + std::make_unique( + stream_name, + data_part_storage, + stream_name, skip_index->getSerializedFileExtension(), + stream_name, marks_file_extension, + default_codec, settings.max_compress_block_size, + marks_compression_codec, settings.marks_compress_block_size, + settings.query_write_settings)); GinIndexStorePtr store = nullptr; if (typeid_cast(&*skip_index) != nullptr) @@ -381,7 +375,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { const auto index_helper = skip_indices[i]; auto & stream = *skip_indices_streams[i]; - WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; + WriteBuffer & marks_out = stream.compress_marks ? *stream.marks_compressed_hashing : *stream.marks_hashing; GinIndexStorePtr store; if (typeid_cast(&*index_helper) != nullptr) @@ -564,7 +558,4 @@ Names MergeTreeDataPartWriterOnDisk::getSkipIndicesColumns() const return Names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); } -template struct MergeTreeDataPartWriterOnDisk::Stream; -template struct MergeTreeDataPartWriterOnDisk::Stream; - } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 3b6687dff99..0d80333368d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -44,7 +44,6 @@ public: /// Helper class, which holds chain of buffers to write data file with marks. /// It is used to write: one column, skip index or all columns (in compact format). - template struct Stream { Stream( @@ -82,9 +81,9 @@ public: /// marks_compressed_hashing -> marks_compressor -> marks_hashing -> marks_file std::unique_ptr marks_file; - std::conditional_t marks_hashing; - std::conditional_t marks_compressor; - std::conditional_t marks_compressed_hashing; + std::optional marks_hashing; + std::optional marks_compressor; + std::optional marks_compressed_hashing; bool compress_marks; bool is_prefinalized = false; @@ -98,8 +97,7 @@ public: void addToChecksums(MergeTreeDataPartChecksums & checksums); }; - using StreamPtr = std::unique_ptr>; - using StatisticStreamPtr = std::unique_ptr>; + using StreamPtr = std::unique_ptr; MergeTreeDataPartWriterOnDisk( const String & data_part_name_, @@ -157,7 +155,7 @@ protected: const MergeTreeIndices skip_indices; const ColumnsStatistics stats; - std::vector stats_streams; + std::vector stats_streams; const String marks_file_extension; const CompressionCodecPtr default_codec; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 459ddc1ca79..d1d4aa4f5b0 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -187,7 +187,7 @@ void MergeTreeDataPartWriterWide::addStreams( query_write_settings.use_adaptive_write_buffer = settings.use_adaptive_write_buffer_for_dynamic_subcolumns && ISerialization::isDynamicSubcolumn(substream_path, substream_path.size()); query_write_settings.adaptive_write_buffer_initial_size = settings.adaptive_write_buffer_initial_size; - column_streams[stream_name] = std::make_unique>( + column_streams[stream_name] = std::make_unique( stream_name, data_part_storage, stream_name, DATA_FILE_EXTENSION, @@ -362,7 +362,7 @@ void MergeTreeDataPartWriterWide::writeSingleMark( void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stream_with_mark, size_t rows_in_mark) { auto & stream = *column_streams[stream_with_mark.stream_name]; - WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; + WriteBuffer & marks_out = stream.compress_marks ? *stream.marks_compressed_hashing : *stream.marks_hashing; writeBinaryLittleEndian(stream_with_mark.mark.offset_in_compressed_file, marks_out); writeBinaryLittleEndian(stream_with_mark.mark.offset_in_decompressed_block, marks_out); From ab10830317d1a0d155aceec3f5285f299f661f02 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Oct 2024 00:12:06 +0200 Subject: [PATCH 09/91] Preparation --- src/Common/CurrentMetrics.cpp | 3 + src/IO/WriteSettings.h | 2 + .../MergeTreeDataPartWriterOnDisk.cpp | 74 ++++++++++++++----- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 11 +-- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 13 ++-- 5 files changed, 72 insertions(+), 31 deletions(-) diff --git a/src/Common/CurrentMetrics.cpp b/src/Common/CurrentMetrics.cpp index da3b5557dbf..c9737e2e846 100644 --- a/src/Common/CurrentMetrics.cpp +++ b/src/Common/CurrentMetrics.cpp @@ -103,6 +103,9 @@ M(IOThreads, "Number of threads in the IO thread pool.") \ M(IOThreadsActive, "Number of threads in the IO thread pool running a task.") \ M(IOThreadsScheduled, "Number of queued or active jobs in the IO thread pool.") \ + M(CompressionThread, "Number of threads in compression thread pools.") \ + M(CompressionThreadActive, "Number of threads in compression thread pools running a task.") \ + M(CompressionThreadScheduled, "Number of queued or active jobs in compression thread pools.") \ M(ThreadPoolRemoteFSReaderThreads, "Number of threads in the thread pool for remote_filesystem_read_method=threadpool.") \ M(ThreadPoolRemoteFSReaderThreadsActive, "Number of threads in the thread pool for remote_filesystem_read_method=threadpool running a task.") \ M(ThreadPoolRemoteFSReaderThreadsScheduled, "Number of queued or active jobs in the thread pool for remote_filesystem_read_method=threadpool.") \ diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 94410f787f0..8016dede6ea 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -28,6 +28,8 @@ struct WriteSettings bool use_adaptive_write_buffer = false; size_t adaptive_write_buffer_initial_size = 16 * 1024; + size_t max_compression_threads = 1; + bool write_through_distributed_cache = false; DistributedCacheSettings distributed_cache_settings; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index c250726aba1..8047d1cf2e9 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -3,16 +3,26 @@ #include #include #include +#include #include + namespace ProfileEvents { -extern const Event MergeTreeDataWriterSkipIndicesCalculationMicroseconds; -extern const Event MergeTreeDataWriterStatisticsCalculationMicroseconds; + extern const Event MergeTreeDataWriterSkipIndicesCalculationMicroseconds; + extern const Event MergeTreeDataWriterStatisticsCalculationMicroseconds; +} + +namespace CurrentMetrics +{ + extern const Metric CompressionThread; + extern const Metric CompressionThreadActive; + extern const Metric CompressionThreadScheduled; } namespace DB { + namespace MergeTreeSetting { extern const MergeTreeSettingsUInt64 index_granularity; @@ -32,9 +42,9 @@ void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() /// Otherwise some data might stuck in the buffers above plain_file and marks_file /// Also the order is important - compressed_hashing.finalize(); + compressed_hashing->finalize(); compressor->finalize(); - plain_hashing.finalize(); + plain_hashing->finalize(); if (marks_hashing) { @@ -86,12 +96,36 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( escaped_column_name(escaped_column_name_), data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, - plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), - plain_hashing(*plain_file), - compressor(std::make_unique(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), - compressed_hashing(*compressor), compress_marks(MarkType(marks_file_extension).compressed) { + plain_file = data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings); + plain_hashing.emplace(*plain_file); + + if (query_write_settings.max_compression_threads > 1) + { + compression_thread_pool.emplace( + CurrentMetrics::CompressionThread, CurrentMetrics::CompressionThreadActive, CurrentMetrics::CompressionThreadScheduled, + query_write_settings.max_compression_threads); + + compressor = std::make_unique( + *plain_hashing, + compression_codec_, + max_compress_block_size_, + query_write_settings.max_compression_threads, + *compression_thread_pool); + } + else + { + compressor = std::make_unique( + *plain_hashing, + compression_codec_, + max_compress_block_size_, + query_write_settings.use_adaptive_write_buffer, + query_write_settings.adaptive_write_buffer_initial_size); + } + + compressed_hashing.emplace(*compressor); + marks_file = data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings); marks_hashing.emplace(*marks_file); marks_compressor.emplace(*marks_hashing, marks_compression_codec_, marks_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size); @@ -110,7 +144,7 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( data_file_extension{data_file_extension_}, plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), plain_hashing(*plain_file), - compressor(std::make_unique(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), + compressor(std::make_unique(*plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), compressed_hashing(*compressor), compress_marks(false) { @@ -121,10 +155,10 @@ void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPa String name = escaped_column_name; checksums.files[name + data_file_extension].is_compressed = true; - checksums.files[name + data_file_extension].uncompressed_size = compressed_hashing.count(); - checksums.files[name + data_file_extension].uncompressed_hash = compressed_hashing.getHash(); - checksums.files[name + data_file_extension].file_size = plain_hashing.count(); - checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash(); + checksums.files[name + data_file_extension].uncompressed_size = compressed_hashing->count(); + checksums.files[name + data_file_extension].uncompressed_hash = compressed_hashing->getHash(); + checksums.files[name + data_file_extension].file_size = plain_hashing->count(); + checksums.files[name + data_file_extension].file_hash = plain_hashing->getHash(); if (marks_hashing) { @@ -391,7 +425,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { if (skip_index_accumulated_marks[i] == index_helper->index.granularity) { - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing); + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(*stream.compressed_hashing); skip_index_accumulated_marks[i] = 0; } @@ -399,11 +433,11 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store, settings); - if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) - stream.compressed_hashing.next(); + if (stream.compressed_hashing->offset() >= settings.min_compress_block_size) + stream.compressed_hashing->next(); - writeBinaryLittleEndian(stream.plain_hashing.count(), marks_out); - writeBinaryLittleEndian(stream.compressed_hashing.offset(), marks_out); + writeBinaryLittleEndian(stream.plain_hashing->count(), marks_out); + writeBinaryLittleEndian(stream.compressed_hashing->offset(), marks_out); /// Actually this numbers is redundant, but we have to store them /// to be compatible with the normal .mrk2 file format @@ -483,7 +517,7 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data { auto & stream = *skip_indices_streams[i]; if (!skip_indices_aggregators[i]->empty()) - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing); + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(*stream.compressed_hashing); /// Register additional files written only by the full-text index. Required because otherwise DROP TABLE complains about unknown /// files. Note that the provided actual checksums are bogus. The problem is that at this point the file writes happened already and @@ -523,7 +557,7 @@ void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataP for (size_t i = 0; i < stats.size(); i++) { auto & stream = *stats_streams[i]; - stats[i]->serialize(stream.compressed_hashing); + stats[i]->serialize(*stream.compressed_hashing); stream.preFinalize(); stream.addToChecksums(checksums); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 0d80333368d..046571cb83f 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -27,7 +27,7 @@ struct Granule /// this granule can be continuation of the previous one. bool mark_on_start; /// if true: When this granule will be written to disk all rows for corresponding mark will - /// be wrtten. It doesn't mean that rows_to_write == index_granularity.getMarkRows(mark_number), + /// be written. It doesn't mean that rows_to_write == index_granularity.getMarkRows(mark_number), /// We may have a lot of small blocks between two marks and this may be the last one. bool is_complete; }; @@ -74,10 +74,10 @@ public: /// compressed_hashing -> compressor -> plain_hashing -> plain_file std::unique_ptr plain_file; - HashingWriteBuffer plain_hashing; + std::optional plain_hashing; /// This could be either CompressedWriteBuffer or ParallelCompressedWriteBuffer std::unique_ptr compressor; - HashingWriteBuffer compressed_hashing; + std::optional compressed_hashing; /// marks_compressed_hashing -> marks_compressor -> marks_hashing -> marks_file std::unique_ptr marks_file; @@ -88,10 +88,11 @@ public: bool is_prefinalized = false; + /// Thread pool for parallel compression. + std::optional compression_thread_pool; + void preFinalize(); - void finalize(); - void sync() const; void addToChecksums(MergeTreeDataPartChecksums & checksums); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index d1d4aa4f5b0..523e4c4a31d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -9,6 +9,7 @@ #include #include + namespace DB { @@ -230,7 +231,7 @@ ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGett if (is_offsets && offset_columns.contains(stream_name)) return nullptr; - return &column_streams.at(stream_name)->compressed_hashing; + return &column_streams.at(stream_name)->compressed_hashing.value(); }; } @@ -399,13 +400,13 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( auto & stream = *column_streams[stream_name]; /// There could already be enough data to compress into the new block. - if (stream.compressed_hashing.offset() >= min_compress_block_size) - stream.compressed_hashing.next(); + if (stream.compressed_hashing->offset() >= min_compress_block_size) + stream.compressed_hashing->next(); StreamNameAndMark stream_with_mark; stream_with_mark.stream_name = stream_name; - stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing.count(); - stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing.offset(); + stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing->count(); + stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing->offset(); result.push_back(stream_with_mark); }, name_and_type.type, column_sample); @@ -438,7 +439,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( if (is_offsets && offset_columns.contains(stream_name)) return; - column_streams.at(stream_name)->compressed_hashing.nextIfAtEnd(); + column_streams.at(stream_name)->compressed_hashing->nextIfAtEnd(); }, name_and_type.type, column.getPtr()); } From bb3bfa536ac54185fea01d23fa22d970a2cccf84 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Oct 2024 00:35:01 +0200 Subject: [PATCH 10/91] Make it configurable --- src/IO/WriteSettings.h | 1 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 22 ++++++++++++++++--- .../MergeTree/IMergedBlockOutputStream.cpp | 1 + .../MergeTree/IMergedBlockOutputStream.h | 1 + src/Storages/MergeTree/MergeTask.cpp | 9 +------- .../MergeTree/MergeTreeIOSettings.cpp | 2 ++ src/Storages/MergeTree/MergeTreeSettings.cpp | 3 +++ 7 files changed, 28 insertions(+), 11 deletions(-) diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 8016dede6ea..4eeb01b5acc 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -4,6 +4,7 @@ #include #include + namespace DB { diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index 3d6366f9217..a9f188338e1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -178,9 +178,24 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeIndexGranularity & computed_index_granularity) { if (part_type == MergeTreeDataPartType::Compact) - return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, - index_granularity_info_, storage_settings_, columns_list, column_positions, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, - marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); + return createMergeTreeDataPartCompactWriter( + data_part_name_, + logger_name_, + serializations_, + data_part_storage_, + index_granularity_info_, + storage_settings_, + columns_list, + column_positions, + metadata_snapshot, + virtual_columns, + indices_to_recalc, + stats_to_recalc_, + marks_file_extension_, + default_codec_, + writer_settings, + computed_index_granularity); + if (part_type == MergeTreeDataPartType::Wide) return createMergeTreeDataPartWideWriter( data_part_name_, @@ -198,6 +213,7 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( default_codec_, writer_settings, computed_index_granularity); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString()); } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index eb904a8e2ef..209b274ee6a 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -4,6 +4,7 @@ #include #include + namespace DB { diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index cfcfb177e05..f67cf66ee50 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -7,6 +7,7 @@ #include #include + namespace DB { diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 74d6d60ba1b..b03fb1b12cf 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -6,11 +6,8 @@ #include #include -#include #include #include -#include -#include #include #include #include @@ -20,10 +17,8 @@ #include #include #include -#include #include #include -#include #include #include #include @@ -34,9 +29,6 @@ #include #include #include -#include -#include -#include #include #include #include @@ -48,6 +40,7 @@ #include #include + namespace ProfileEvents { extern const Event Merge; diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.cpp b/src/Storages/MergeTree/MergeTreeIOSettings.cpp index 8b87c35b4e6..6705d75af41 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeIOSettings.cpp @@ -26,6 +26,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString primary_key_compression_codec; extern const MergeTreeSettingsBool use_adaptive_write_buffer_for_dynamic_subcolumns; extern const MergeTreeSettingsBool use_compact_variant_discriminators_serialization; + extern const MergeTreeSettingsUInt64 max_compression_threads; } MergeTreeWriterSettings::MergeTreeWriterSettings( @@ -54,6 +55,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( , use_adaptive_write_buffer_for_dynamic_subcolumns((*storage_settings)[MergeTreeSetting::use_adaptive_write_buffer_for_dynamic_subcolumns]) , adaptive_write_buffer_initial_size((*storage_settings)[MergeTreeSetting::adaptive_write_buffer_initial_size]) { + query_write_settings.max_compression_threads = (*storage_settings)[MergeTreeSetting::max_compression_threads]; } } diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 86d95aee242..4e7d0c0a721 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -53,6 +53,9 @@ namespace ErrorCodes M(Bool, load_existing_rows_count_for_old_parts, false, "Whether to load existing_rows_count for existing parts. If false, existing_rows_count will be equal to rows_count for existing parts.", 0) \ M(Bool, use_compact_variant_discriminators_serialization, true, "Use compact version of Variant discriminators serialization.", 0) \ \ + /** Merge and insert settings */ \ + M(UInt64, max_compression_threads, 1, "Maximum number of threads for writing compressed data. This is an expert-level setting, do not change it.", 0) \ + \ /** Merge selector settings. */ \ M(UInt64, merge_selector_blurry_base_scale_factor, 0, "Controls when the logic kicks in relatively to the number of parts in partition. The bigger the factor the more belated reaction will be.", 0) \ M(UInt64, merge_selector_window_size, 1000, "How many parts to look at once.", 0) \ From 0f3f15338d5c85fa159a85e929c689d69610d4ef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Oct 2024 02:09:15 +0200 Subject: [PATCH 11/91] Something --- src/Common/ThreadPool.h | 4 +-- .../ParallelCompressedWriteBuffer.cpp | 2 +- src/Dictionaries/HashedDictionary.h | 28 +++++++++++-------- .../Transforms/AggregatingTransform.h | 2 +- 4 files changed, 20 insertions(+), 16 deletions(-) diff --git a/src/Common/ThreadPool.h b/src/Common/ThreadPool.h index 7e497245acc..b52e4a60571 100644 --- a/src/Common/ThreadPool.h +++ b/src/Common/ThreadPool.h @@ -122,7 +122,7 @@ public: void scheduleOrThrowOnError(Job job, Priority priority = {}); /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or return false. - bool trySchedule(Job job, Priority priority = {}, uint64_t wait_microseconds = 0) noexcept; + [[nodiscard]] bool trySchedule(Job job, Priority priority = {}, uint64_t wait_microseconds = 0) noexcept; /// Similar to scheduleOrThrowOnError(...). Wait for specified amount of time and schedule a job or throw an exception. void scheduleOrThrow(Job job, Priority priority = {}, uint64_t wait_microseconds = 0, bool propagate_opentelemetry_tracing_context = true); @@ -142,7 +142,7 @@ public: /// Returns true if the pool already terminated /// (and any further scheduling will produce CANNOT_SCHEDULE_TASK exception) - bool finished() const; + [[nodiscard]] bool finished() const; void setMaxThreads(size_t value); void setMaxFreeThreads(size_t value); diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 1041a14979e..bd8d6371501 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -46,7 +46,7 @@ void ParallelCompressedWriteBuffer::nextImpl() current_buffer->sequence_num = current_sequence_num; ++current_sequence_num; current_buffer->uncompressed_size = offset(); - pool.trySchedule([this, my_current_buffer = current_buffer, thread_group = CurrentThread::getGroup()] + pool.scheduleOrThrowOnError([this, my_current_buffer = current_buffer, thread_group = CurrentThread::getGroup()] { SCOPE_EXIT_SAFE( if (thread_group) diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 7e935fe4855..ec5bc0a8a35 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -334,22 +334,26 @@ HashedDictionary::~HashedDictionary() if (container.empty()) return; - pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] - { - SCOPE_EXIT_SAFE( + if (!pool.trySchedule([&container, thread_group = CurrentThread::getGroup()] + { + SCOPE_EXIT_SAFE( + if (thread_group) + CurrentThread::detachFromGroupIfNotDetached(); + ); + + /// Do not account memory that was occupied by the dictionaries for the query/user context. + MemoryTrackerBlockerInThread memory_blocker; + if (thread_group) - CurrentThread::detachFromGroupIfNotDetached(); - ); + CurrentThread::attachToGroupIfDetached(thread_group); + setThreadName("HashedDictDtor"); - /// Do not account memory that was occupied by the dictionaries for the query/user context. + clearContainer(container); + })) + { MemoryTrackerBlockerInThread memory_blocker; - - if (thread_group) - CurrentThread::attachToGroupIfDetached(thread_group); - setThreadName("HashedDictDtor"); - clearContainer(container); - }); + } ++hash_tables_count; }; diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index b9212375c91..398d7efa97e 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -107,7 +107,7 @@ struct ManyAggregatedData if (variant->aggregator) { // variant is moved here and will be destroyed in the destructor of the lambda function. - pool->trySchedule( + pool->scheduleOrThrowOnError( [my_variant = std::move(variant), thread_group = CurrentThread::getGroup()]() { SCOPE_EXIT_SAFE( From dffaf9b9a5b69ec4e342192ecbf00ad2c90208d8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Oct 2024 03:32:19 +0200 Subject: [PATCH 12/91] Fix error --- src/Compression/ParallelCompressedWriteBuffer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index bd8d6371501..08c2a78c80b 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -122,7 +122,7 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) if (can_write_directly) { char * out_compressed_ptr = out.position() + sizeof(CityHash_v1_0_2::uint128); - UInt32 compressed_size = codec->compress(working_buffer.begin(), uncompressed_size, out_compressed_ptr); + UInt32 compressed_size = codec->compress(buffer->uncompressed.data(), uncompressed_size, out_compressed_ptr); CityHash_v1_0_2::uint128 checksum = CityHash_v1_0_2::CityHash128(out_compressed_ptr, compressed_size); From dba7c9cf4a990c2b29f8351b51f16a3614802499 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Oct 2024 05:13:34 +0200 Subject: [PATCH 13/91] Add a test --- .../0_stateless/03254_parallel_compression.sql | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/03254_parallel_compression.sql diff --git a/tests/queries/0_stateless/03254_parallel_compression.sql b/tests/queries/0_stateless/03254_parallel_compression.sql new file mode 100644 index 00000000000..a17deed7d8c --- /dev/null +++ b/tests/queries/0_stateless/03254_parallel_compression.sql @@ -0,0 +1,11 @@ +DROP TABLE IF EXISTS test2; + +CREATE TABLE test2 +( + k UInt64 +) ENGINE = MergeTree ORDER BY k SETTINGS min_compress_block_size = 10240, min_bytes_for_wide_part = 1, max_compression_threads = 64; + +INSERT INTO test2 SELECT number FROM numbers(20000); +SELECT sum(k) = (9999 * 10000 / 2 + 10000 * 9999) FROM test2 WHERE k > 10000; + +DROP TABLE test2; From e6bae901ed9a149e81cdf50e358470326b140c32 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 21 Oct 2024 18:39:20 +0200 Subject: [PATCH 14/91] Incomplete --- .../ParallelCompressedWriteBuffer.cpp | 6 ++++- .../ParallelCompressedWriteBuffer.h | 26 ++++++------------- .../MergeTreeDataPartWriterOnDisk.cpp | 2 ++ .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 1 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 19 +++++++++----- 5 files changed, 29 insertions(+), 25 deletions(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 08c2a78c80b..303e1ece68a 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -44,6 +44,8 @@ void ParallelCompressedWriteBuffer::nextImpl() /// The buffer will be compressed and processed in the thread. current_buffer->busy = true; current_buffer->sequence_num = current_sequence_num; + current_buffer->out_callback = callback; + callback = {}; ++current_sequence_num; current_buffer->uncompressed_size = offset(); pool.scheduleOrThrowOnError([this, my_current_buffer = current_buffer, thread_group = CurrentThread::getGroup()] @@ -60,7 +62,7 @@ void ParallelCompressedWriteBuffer::nextImpl() compress(my_current_buffer); }); - const BufferPair * previous_buffer = &*current_buffer; + BufferPair * previous_buffer = &*current_buffer; ++current_buffer; if (current_buffer == buffers.end()) { @@ -153,6 +155,8 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) } std::unique_lock lock(mutex); + if (buffer->out_callback) + buffer->out_callback(); buffer->busy = false; cond.notify_all(); } diff --git a/src/Compression/ParallelCompressedWriteBuffer.h b/src/Compression/ParallelCompressedWriteBuffer.h index 4d1dfc79797..8c5f249b06c 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.h +++ b/src/Compression/ParallelCompressedWriteBuffer.h @@ -31,24 +31,11 @@ public: ~ParallelCompressedWriteBuffer() override; - /// The amount of compressed data - size_t getCompressedBytes() + /// This function will be called once after compressing the next data and sending it to the out. + /// It can be used to fill information about marks. + void setCompletionCallback(std::function callback_) { - nextIfAtEnd(); - return out.count(); - } - - /// How many uncompressed bytes were written to the buffer - size_t getUncompressedBytes() - { - return count(); - } - - /// How many bytes are in the buffer (not yet compressed) - size_t getRemainingBytes() - { - nextIfAtEnd(); - return offset(); + callback = callback_; } private: @@ -71,15 +58,18 @@ private: Memory<> uncompressed; size_t uncompressed_size = 0; PODArray compressed; - const BufferPair * previous = nullptr; + BufferPair * previous = nullptr; size_t sequence_num = 0; bool busy = false; + std::function out_callback; }; std::mutex mutex; std::condition_variable cond; std::list buffers; + std::function callback; + using Iterator = std::list::iterator; Iterator current_buffer; size_t current_sequence_num = 0; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 8047d1cf2e9..89db8174636 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -113,6 +113,8 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( max_compress_block_size_, query_write_settings.max_compression_threads, *compression_thread_pool); + + is_compressor_parallel = true; } else { diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index 046571cb83f..cb46785ccbd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -76,6 +76,7 @@ public: std::unique_ptr plain_file; std::optional plain_hashing; /// This could be either CompressedWriteBuffer or ParallelCompressedWriteBuffer + bool is_compressor_parallel = false; std::unique_ptr compressor; std::optional compressed_hashing; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 523e4c4a31d..860722ba870 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -400,15 +400,22 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( auto & stream = *column_streams[stream_name]; /// There could already be enough data to compress into the new block. + auto push_mark = [&] + { + StreamNameAndMark stream_with_mark; + stream_with_mark.stream_name = stream_name; + stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing->count(); + stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing->offset(); + result.push_back(stream_with_mark); + }; + if (stream.compressed_hashing->offset() >= min_compress_block_size) + { + stream.compressed_hashing->next(); + } - StreamNameAndMark stream_with_mark; - stream_with_mark.stream_name = stream_name; - stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing->count(); - stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing->offset(); - - result.push_back(stream_with_mark); + push_mark(); }, name_and_type.type, column_sample); return result; From e19bf218f69448c9605f269ae7a3894bc24f0003 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 25 Oct 2024 06:12:50 +0000 Subject: [PATCH 15/91] Fix 'Unknown executor' when reading from stdin in clickhouse local --- programs/local/LocalServer.cpp | 5 +++++ programs/local/LocalServer.h | 2 +- src/Client/ClientBase.cpp | 3 ++- src/Client/ClientBase.h | 2 ++ .../03031_clickhouse_local_input.reference | 4 +++- .../0_stateless/03031_clickhouse_local_input.sh | 17 ++++++++++++++--- 6 files changed, 27 insertions(+), 6 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index b6b67724b0a..4b861d579ab 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -130,6 +130,11 @@ void applySettingsOverridesForLocal(ContextMutablePtr context) context->setSettings(settings); } +LocalServer::LocalServer() +{ + is_local = true; +} + Poco::Util::LayeredConfiguration & LocalServer::getClientConfiguration() { return config(); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index 7e92e92d345..ced25dbdf90 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -23,7 +23,7 @@ namespace DB class LocalServer : public ClientApplicationBase, public Loggers { public: - LocalServer() = default; + LocalServer(); void initialize(Poco::Util::Application & self) override; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 23aa7e841cb..b6223cf6872 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1748,7 +1748,8 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des } else if (!is_interactive) { - sendDataFromStdin(sample, columns_description_for_query, parsed_query); + if (!is_local) + sendDataFromStdin(sample, columns_description_for_query, parsed_query); } else throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert"); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index b06958f1d14..daf3ee7e3e4 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -263,6 +263,8 @@ protected: bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool delayed_interactive = false; + bool is_local = false; /// clickhouse-local, otherwise clickhouse-client + bool echo_queries = false; /// Print queries before execution in batch mode. bool ignore_error = false; /// In case of errors, don't print error message, continue to next query. Only applicable for non-interactive mode. diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.reference b/tests/queries/0_stateless/03031_clickhouse_local_input.reference index a6feeef100d..529f1832598 100644 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.reference +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.reference @@ -1,4 +1,6 @@ -# foo +# foo (pipe) +foo +# foo (file) foo # !foo # bar diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.sh b/tests/queries/0_stateless/03031_clickhouse_local_input.sh index e2f9cf48108..540e1203154 100755 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.sh +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.sh @@ -4,17 +4,28 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -tmp_file="$CUR_DIR/$CLICKHOUSE_DATABASE.txt" -echo '# foo' +tmp_file="$CUR_DIR/03031_$CLICKHOUSE_DATABASE.txt" +tmp_input="$CUR_DIR/03031_${CLICKHOUSE_DATABASE}_in.txt" + +echo '# foo (pipe)' $CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') format LineAsString" << "$tmp_input" +$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') format LineAsString" <"$tmp_input" +cat "$tmp_file" + echo '# !foo' $CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select * from input('x String') where x != 'foo' format LineAsString" << Date: Fri, 25 Oct 2024 08:24:30 +0000 Subject: [PATCH 16/91] Also fix 'Input initializer is not set' in another query --- src/Interpreters/InterpreterInsertQuery.cpp | 3 +-- .../0_stateless/03031_clickhouse_local_input.reference | 2 ++ tests/queries/0_stateless/03031_clickhouse_local_input.sh | 4 ++++ 3 files changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 80b9d91a248..797895e4a93 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -121,8 +121,7 @@ StoragePtr InterpreterInsertQuery::getTable(ASTInsertQuery & query) if (current_context->getSettingsRef()[Setting::allow_experimental_analyzer]) { - InterpreterSelectQueryAnalyzer interpreter_select(query.select, current_context, select_query_options); - header_block = interpreter_select.getSampleBlock(); + header_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query.select, current_context, select_query_options); } else { diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.reference b/tests/queries/0_stateless/03031_clickhouse_local_input.reference index 529f1832598..c6e6b743759 100644 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.reference +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.reference @@ -7,3 +7,5 @@ foo bar # defaults bam +# inferred destination table structure +foo diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.sh b/tests/queries/0_stateless/03031_clickhouse_local_input.sh index 540e1203154..cfd8c2957bb 100755 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.sh +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.sh @@ -28,4 +28,8 @@ echo '# defaults' $CLICKHOUSE_LOCAL --input_format_tsv_empty_as_default=1 --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'LineAsString', 'x String') select y from input('x String, y String DEFAULT \\'bam\\'') format TSV" <<<$'foo\t' cat "$tmp_file" +echo '# inferred destination table structure' +$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'TSV') select * from input('x String') format LineAsString" <"$tmp_input" +cat "$tmp_file" + rm -f "${tmp_file:?}" "${tmp_input:?}" From 3da0b2573a5e13c715562d5f8e544480ebf9cc2b Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 25 Oct 2024 09:12:52 +0000 Subject: [PATCH 17/91] Better fix --- programs/local/LocalServer.cpp | 5 ----- programs/local/LocalServer.h | 2 +- src/Client/ClientBase.cpp | 8 ++++++-- src/Client/ClientBase.h | 2 -- src/Client/IServerConnection.h | 4 ++++ src/Client/LocalConnection.cpp | 5 +++++ src/Client/LocalConnection.h | 2 ++ .../0_stateless/03031_clickhouse_local_input.reference | 4 ++++ tests/queries/0_stateless/03031_clickhouse_local_input.sh | 8 ++++++++ 9 files changed, 30 insertions(+), 10 deletions(-) diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 4b861d579ab..b6b67724b0a 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -130,11 +130,6 @@ void applySettingsOverridesForLocal(ContextMutablePtr context) context->setSettings(settings); } -LocalServer::LocalServer() -{ - is_local = true; -} - Poco::Util::LayeredConfiguration & LocalServer::getClientConfiguration() { return config(); diff --git a/programs/local/LocalServer.h b/programs/local/LocalServer.h index ced25dbdf90..7e92e92d345 100644 --- a/programs/local/LocalServer.h +++ b/programs/local/LocalServer.h @@ -23,7 +23,7 @@ namespace DB class LocalServer : public ClientApplicationBase, public Loggers { public: - LocalServer(); + LocalServer() = default; void initialize(Poco::Util::Application & self) override; diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index b6223cf6872..f5351b94a94 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1630,6 +1630,11 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des if (!parsed_insert_query) return; + /// If it's clickhouse-local, and the input data reading is already baked into the query pipeline, + /// don't read the data again here. + if (!connection->isSendDataNeeded()) + return; + bool have_data_in_stdin = !is_interactive && !stdin_is_a_tty && isStdinNotEmptyAndValid(std_in); if (need_render_progress) @@ -1748,8 +1753,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des } else if (!is_interactive) { - if (!is_local) - sendDataFromStdin(sample, columns_description_for_query, parsed_query); + sendDataFromStdin(sample, columns_description_for_query, parsed_query); } else throw Exception(ErrorCodes::NO_DATA_TO_INSERT, "No data to insert"); diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index daf3ee7e3e4..b06958f1d14 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -263,8 +263,6 @@ protected: bool is_interactive = false; /// Use either interactive line editing interface or batch mode. bool delayed_interactive = false; - bool is_local = false; /// clickhouse-local, otherwise clickhouse-client - bool echo_queries = false; /// Print queries before execution in batch mode. bool ignore_error = false; /// In case of errors, don't print error message, continue to next query. Only applicable for non-interactive mode. diff --git a/src/Client/IServerConnection.h b/src/Client/IServerConnection.h index 6ab4234bca2..fe69be8788a 100644 --- a/src/Client/IServerConnection.h +++ b/src/Client/IServerConnection.h @@ -109,6 +109,10 @@ public: /// Send block of data; if name is specified, server will write it to external (temporary) table of that name. virtual void sendData(const Block & block, const String & name, bool scalar) = 0; + /// Whether the client needs to read and send the data for the INSERT. + /// False if the server will read the data through other means (in particular if clickhouse-local added input reading step directly into the query pipeline). + virtual bool isSendDataNeeded() const { return true; } + /// Send all contents of external (temporary) tables. virtual void sendExternalTablesData(ExternalTablesData & data) = 0; diff --git a/src/Client/LocalConnection.cpp b/src/Client/LocalConnection.cpp index e4915a77c83..4ca209c29c7 100644 --- a/src/Client/LocalConnection.cpp +++ b/src/Client/LocalConnection.cpp @@ -328,6 +328,11 @@ void LocalConnection::sendData(const Block & block, const String &, bool) sendProfileEvents(); } +bool LocalConnection::isSendDataNeeded() const +{ + return !state || state->input_pipeline == nullptr; +} + void LocalConnection::sendCancel() { state->is_cancelled = true; diff --git a/src/Client/LocalConnection.h b/src/Client/LocalConnection.h index b424c5b5aa3..a70ed6ffa7e 100644 --- a/src/Client/LocalConnection.h +++ b/src/Client/LocalConnection.h @@ -120,6 +120,8 @@ public: void sendData(const Block & block, const String & name/* = "" */, bool scalar/* = false */) override; + bool isSendDataNeeded() const override; + void sendExternalTablesData(ExternalTablesData &) override; void sendMergeTreeReadTaskResponse(const ParallelReadResponse & response) override; diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.reference b/tests/queries/0_stateless/03031_clickhouse_local_input.reference index c6e6b743759..bbb57da94ce 100644 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.reference +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.reference @@ -9,3 +9,7 @@ bar bam # inferred destination table structure foo +# direct +foo +# infile +foo diff --git a/tests/queries/0_stateless/03031_clickhouse_local_input.sh b/tests/queries/0_stateless/03031_clickhouse_local_input.sh index cfd8c2957bb..f271a5184fd 100755 --- a/tests/queries/0_stateless/03031_clickhouse_local_input.sh +++ b/tests/queries/0_stateless/03031_clickhouse_local_input.sh @@ -32,4 +32,12 @@ echo '# inferred destination table structure' $CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'TSV') select * from input('x String') format LineAsString" <"$tmp_input" cat "$tmp_file" +echo '# direct' +$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'LineAsString', 'x String') format LineAsString" <"$tmp_input" +cat "$tmp_file" + +echo '# infile' +$CLICKHOUSE_LOCAL --engine_file_truncate_on_insert=1 -q "insert into function file('$tmp_file', 'LineAsString', 'x String') from infile '$tmp_input' format LineAsString" +cat "$tmp_file" + rm -f "${tmp_file:?}" "${tmp_input:?}" From 45e23584f4cee58bf9c0f0612e4799076c0d21e8 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 25 Oct 2024 09:15:53 +0000 Subject: [PATCH 18/91] Comment --- src/Client/ClientBase.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index f5351b94a94..6475d682b65 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -1631,7 +1631,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des return; /// If it's clickhouse-local, and the input data reading is already baked into the query pipeline, - /// don't read the data again here. + /// don't read the data again here. This happens in some cases (e.g. input() table function) but not others (e.g. INFILE). if (!connection->isSendDataNeeded()) return; From c58afb753c3a9b394f6f88cc8fad3e13897c5e57 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 27 Oct 2024 00:29:36 +0200 Subject: [PATCH 19/91] Retry more errors from S3 --- src/IO/S3/Client.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/IO/S3/Client.cpp b/src/IO/S3/Client.cpp index 9a0eccd8783..088087458c7 100644 --- a/src/IO/S3/Client.cpp +++ b/src/IO/S3/Client.cpp @@ -645,7 +645,7 @@ Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request try { /// S3 does retries network errors actually. - /// But it is matter when errors occur. + /// But it does matter when errors occur. /// This code retries a specific case when /// network error happens when XML document is being read from the response body. /// Hence, the response body is a stream, network errors are possible at reading. @@ -656,8 +656,9 @@ Client::doRequestWithRetryNetworkErrors(RequestType & request, RequestFn request /// Requests that expose the response stream as an answer are not retried with that code. E.g. GetObject. return request_fn_(request_); } - catch (Poco::Net::ConnectionResetException &) + catch (Poco::Net::NetException &) { + /// This includes "connection reset", "malformed message", and possibly other exceptions. if constexpr (IsReadMethod) { From a77caf42149ab864a3c96df09d7fc8771362adaa Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Fri, 1 Nov 2024 03:41:03 +0000 Subject: [PATCH 20/91] Exempt refreshable materialized views from ignore_empty_sql_security_in_create_view_query --- src/Interpreters/InterpreterCreateQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index a38a7ab45d1..f6586f8bfc2 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -1467,7 +1467,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) bool is_secondary_query = getContext()->getZooKeeperMetadataTransaction() && !getContext()->getZooKeeperMetadataTransaction()->isInitialQuery(); auto mode = getLoadingStrictnessLevel(create.attach, /*force_attach*/ false, /*has_force_restore_data_flag*/ false, is_secondary_query || is_restore_from_backup); - if (!create.sql_security && create.supportSQLSecurity() && !getContext()->getServerSettings()[ServerSetting::ignore_empty_sql_security_in_create_view_query]) + if (!create.sql_security && create.supportSQLSecurity() && (create.refresh_strategy || !getContext()->getServerSettings()[ServerSetting::ignore_empty_sql_security_in_create_view_query])) create.sql_security = std::make_shared(); if (create.sql_security) From 45bdc4d4deaf6a48ec08f52a9bc8a765730a9b88 Mon Sep 17 00:00:00 2001 From: Michael Kolupaev Date: Wed, 6 Nov 2024 01:12:07 +0000 Subject: [PATCH 21/91] Update tests --- .../02932_refreshable_materialized_views_1.reference | 8 ++++---- .../02932_refreshable_materialized_views_2.reference | 4 ++-- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference index 3ec0d3b9ee2..b50ea042e86 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_1.reference @@ -1,14 +1,14 @@ <1: created view> a -CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x +CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 SECOND\n(\n `x` UInt64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT number AS x\nFROM numbers(2)\nUNION ALL\nSELECT rand64() AS x <2: refreshed> 3 1 1 <3: time difference at least> 1000 <4.1: fake clock> Scheduled 2050-01-01 00:00:01 2050-01-01 00:00:02 1 3 3 3 0 <4.5: altered> Scheduled 2050-01-01 00:00:01 2052-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nAS SELECT x * 2 AS x\nFROM default.src +CREATE MATERIALIZED VIEW default.a\nREFRESH EVERY 2 YEAR\n(\n `x` UInt64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 2 AS x\nFROM default.src <5: no refresh> 3 <6: refreshed> 2 <7: refreshed> Scheduled 2052-02-03 04:05:06 2054-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a <7.5: created dependent> 2052-11-11 11:11:11 <8: refreshed> 20 <9: refreshed> a Scheduled 2054-01-01 00:00:00 @@ -26,4 +26,4 @@ CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR DEPENDS ON default.a\n( <17: chain-refreshed> a Scheduled 2062-01-01 00:00:00 <17: chain-refreshed> b Scheduled 2062-01-01 00:00:00 <18: removed dependency> b Scheduled 2062-03-03 03:03:03 2062-03-03 03:03:03 2064-01-01 00:00:00 -CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nAS SELECT x * 10 AS y\nFROM default.a +CREATE MATERIALIZED VIEW default.b\nREFRESH EVERY 2 YEAR\n(\n `y` Int32\n)\nENGINE = MergeTree\nORDER BY y\nSETTINGS index_granularity = 8192\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS y\nFROM default.a diff --git a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference index 3eeab4f574e..8dcc3d55603 100644 --- a/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference +++ b/tests/queries/0_stateless/02932_refreshable_materialized_views_2.reference @@ -7,9 +7,9 @@ <25: rename during refresh> f Running <27: cancelled> f Scheduled cancelled <28: drop during refresh> 0 0 -CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nAS SELECT 42 AS x +CREATE MATERIALIZED VIEW default.g\nREFRESH EVERY 1 WEEK OFFSET 3 DAY 4 HOUR RANDOMIZE FOR 4 DAY 1 HOUR\n(\n `x` Int64\n)\nENGINE = Memory\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT 42 AS x <29: randomize> 1 1 -CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nAS SELECT x * 10 AS x\nFROM default.src +CREATE MATERIALIZED VIEW default.h\nREFRESH EVERY 1 SECOND TO default.dest\n(\n `x` Int64\n)\nDEFINER = default SQL SECURITY DEFINER\nAS SELECT x * 10 AS x\nFROM default.src <30: to existing table> 10 <31: to existing table> 10 <31: to existing table> 20 From f229fc5b40bd0faa3f312bcdc3123cfdfb6a70fc Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 12:14:09 +0100 Subject: [PATCH 22/91] Deprecate CLICKHOUSE_UID/CLICKHOUSE_GID docker ENV --- docker/keeper/entrypoint.sh | 14 ++++++++------ docker/server/entrypoint.sh | 14 ++++++++------ 2 files changed, 16 insertions(+), 12 deletions(-) diff --git a/docker/keeper/entrypoint.sh b/docker/keeper/entrypoint.sh index 68bd0ef9d87..c5d5d26ec11 100644 --- a/docker/keeper/entrypoint.sh +++ b/docker/keeper/entrypoint.sh @@ -9,13 +9,15 @@ if [ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" ]; then DO_CHOWN=0 fi -CLICKHOUSE_UID="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" -CLICKHOUSE_GID="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" - -# support --user +# support `docker run --user=xxx:xxxx` if [ "$(id -u)" = "0" ]; then - USER=$CLICKHOUSE_UID - GROUP=$CLICKHOUSE_GID + # CLICKHOUSE_UID and CLICKHOUSE_GID are kept for backward compatibility + if [[ "${CLICKHOUSE_UID:-}" || "${CLICKHOUSE_GID:-}" ]]; then + echo 'WARNING: consider using a proper "--user=xxx:xxxx" running argument instead of CLICKHOUSE_UID/CLICKHOUSE_GID' >&2 + echo 'Support for CLICKHOUSE_UID/CLICKHOUSE_GID will be removed in a couple of releases' >&2 + fi + USER="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" + GROUP="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" if command -v gosu &> /dev/null; then gosu="gosu $USER:$GROUP" elif command -v su-exec &> /dev/null; then diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 3102ab8297c..a60643c63f1 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -8,13 +8,15 @@ if [ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" ]; then DO_CHOWN=0 fi -CLICKHOUSE_UID="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" -CLICKHOUSE_GID="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" - -# support --user +# support `docker run --user=xxx:xxxx` if [ "$(id -u)" = "0" ]; then - USER=$CLICKHOUSE_UID - GROUP=$CLICKHOUSE_GID + # CLICKHOUSE_UID and CLICKHOUSE_GID are kept for backward compatibility + if [[ "${CLICKHOUSE_UID:-}" || "${CLICKHOUSE_GID:-}" ]]; then + echo 'WARNING: consider using a proper "--user=xxx:xxxx" running argument instead of CLICKHOUSE_UID/CLICKHOUSE_GID' >&2 + echo 'Support for CLICKHOUSE_UID/CLICKHOUSE_GID will be removed in a couple of releases' >&2 + fi + USER="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" + GROUP="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" else USER="$(id -u)" GROUP="$(id -g)" From b82658a28524f47356ab63a3c367489e10c83791 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 12:16:19 +0100 Subject: [PATCH 23/91] Remove processing of CLICKHOUSE_DOCKER_RESTART_ON_EXIT --- docker/server/entrypoint.sh | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index a60643c63f1..6aa031b1352 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -205,18 +205,8 @@ if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then CLICKHOUSE_WATCHDOG_ENABLE=${CLICKHOUSE_WATCHDOG_ENABLE:-0} export CLICKHOUSE_WATCHDOG_ENABLE - # An option for easy restarting and replacing clickhouse-server in a container, especially in Kubernetes. - # For example, you can replace the clickhouse-server binary to another and restart it while keeping the container running. - if [[ "${CLICKHOUSE_DOCKER_RESTART_ON_EXIT:-0}" -eq "1" ]]; then - while true; do - # This runs the server as a child process of the shell script: - /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" ||: - echo >&2 'ClickHouse Server exited, and the environment variable CLICKHOUSE_DOCKER_RESTART_ON_EXIT is set to 1. Restarting the server.' - done - else - # This replaces the shell script with the server: - exec /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" - fi + # This replaces the shell script with the server: + exec /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" fi # Otherwise, we assume the user want to run his own process, for example a `bash` shell to explore this image From ae97149041d2c489617f242ce2c96648c98ae620 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 12:18:11 +0100 Subject: [PATCH 24/91] Remove `/usr/bin` for clickhouse/clickhouse-server/clickhouse-keeper --- docker/keeper/entrypoint.sh | 4 ++-- docker/server/entrypoint.sh | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/keeper/entrypoint.sh b/docker/keeper/entrypoint.sh index c5d5d26ec11..92b91a0f8c3 100644 --- a/docker/keeper/entrypoint.sh +++ b/docker/keeper/entrypoint.sh @@ -84,11 +84,11 @@ if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then # There is a config file. It is already tested with gosu (if it is readably by keeper user) if [ -f "$KEEPER_CONFIG" ]; then - exec $gosu /usr/bin/clickhouse-keeper --config-file="$KEEPER_CONFIG" "$@" + exec $gosu clickhouse-keeper --config-file="$KEEPER_CONFIG" "$@" fi # There is no config file. Will use embedded one - exec $gosu /usr/bin/clickhouse-keeper --log-file="$LOG_PATH" --errorlog-file="$ERROR_LOG_PATH" "$@" + exec $gosu clickhouse-keeper --log-file="$LOG_PATH" --errorlog-file="$ERROR_LOG_PATH" "$@" fi # Otherwise, we assume the user want to run his own process, for example a `bash` shell to explore this image diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 6aa031b1352..7a990e7d889 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -62,7 +62,7 @@ function create_directory_and_do_chown() { # if DO_CHOWN=0 it means that the system does not map root user to "admin" permissions # it mainly happens on NFS mounts where root==nobody for security reasons # thus mkdir MUST run with user id/gid and not from nobody that has zero permissions - mkdir="/usr/bin/clickhouse su "${USER}:${GROUP}" mkdir" + mkdir="clickhouse su ""${USER}:${GROUP}"" mkdir" fi if ! $mkdir -p "$dir"; then echo "Couldn't create necessary directory: $dir" @@ -145,7 +145,7 @@ if [ -n "${RUN_INITDB_SCRIPTS}" ]; then fi # Listen only on localhost until the initialization is done - /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 & + clickhouse su "${USER}:${GROUP}" clickhouse-server --config-file="$CLICKHOUSE_CONFIG" -- --listen_host=127.0.0.1 & pid="$!" # check if clickhouse is ready to accept connections @@ -206,7 +206,7 @@ if [[ $# -lt 1 ]] || [[ "$1" == "--"* ]]; then export CLICKHOUSE_WATCHDOG_ENABLE # This replaces the shell script with the server: - exec /usr/bin/clickhouse su "${USER}:${GROUP}" /usr/bin/clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" + exec clickhouse su "${USER}:${GROUP}" clickhouse-server --config-file="$CLICKHOUSE_CONFIG" "$@" fi # Otherwise, we assume the user want to run his own process, for example a `bash` shell to explore this image From 1babb919c3450969b2ecc810705854e702458110 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 12:18:57 +0100 Subject: [PATCH 25/91] Follow the DOI review recommendations/requirements --- docker/server/Dockerfile.ubuntu | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 0d5c983f5e6..2b023a9cf03 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -91,7 +91,6 @@ RUN if [ -n "${single_binary_location_url}" ]; then \ RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \ apt-get update \ && apt-get install --yes --no-install-recommends \ - apt-transport-https \ dirmngr \ gnupg2 \ && mkdir -p /etc/apt/sources.list.d \ @@ -108,13 +107,12 @@ RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \ && for package in ${PACKAGES}; do \ packages="${packages} ${package}=${VERSION}" \ ; done \ - && apt-get install --allow-unauthenticated --yes --no-install-recommends ${packages} || exit 1 \ + && apt-get install --yes --no-install-recommends ${packages} || exit 1 \ && rm -rf \ /var/lib/apt/lists/* \ /var/cache/debconf \ /tmp/* \ - && apt-get autoremove --purge -yq libksba8 \ - && apt-get autoremove -yq \ + && apt-get autoremove --purge -yq dirmngr gnupg2 \ ; fi # post install @@ -126,8 +124,6 @@ RUN clickhouse-local -q 'SELECT * FROM system.build_options' \ RUN locale-gen en_US.UTF-8 ENV LANG en_US.UTF-8 -ENV LANGUAGE en_US:en -ENV LC_ALL en_US.UTF-8 ENV TZ UTC RUN mkdir /docker-entrypoint-initdb.d From d8ff6f868fe6cb346ac751b468b462b857399480 Mon Sep 17 00:00:00 2001 From: Pablo Marcos Date: Thu, 7 Nov 2024 12:36:21 +0000 Subject: [PATCH 26/91] bitShift: return 0 instead of throwing an exception if overflow --- src/Functions/bitShiftLeft.cpp | 20 +++++++++++-------- src/Functions/bitShiftRight.cpp | 20 +++++++++++-------- .../02766_bitshift_with_const_arguments.sql | 2 +- ...t_throws_error_for_out_of_bounds.reference | 6 ++++++ ...t_shift_throws_error_for_out_of_bounds.sql | 12 +++++------ 5 files changed, 37 insertions(+), 23 deletions(-) diff --git a/src/Functions/bitShiftLeft.cpp b/src/Functions/bitShiftLeft.cpp index 0eb0d82ef0f..7fd0f7cf631 100644 --- a/src/Functions/bitShiftLeft.cpp +++ b/src/Functions/bitShiftLeft.cpp @@ -25,8 +25,10 @@ struct BitShiftLeftImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftLeft is not implemented for big integers as second argument"); - else if (b < 0 || static_cast(b) > 8 * sizeof(A)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); + else if (b < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value"); + else if (static_cast(b) > 8 * sizeof(A)) + return static_cast(0); else if constexpr (is_big_int_v) return static_cast(a) << static_cast(b); else @@ -43,9 +45,10 @@ struct BitShiftLeftImpl const UInt8 word_size = 8 * sizeof(*pos); size_t n = end - pos; const UInt128 bit_limit = static_cast(word_size) * n; - if (b < 0 || static_cast(b) > bit_limit) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - if (b == bit_limit) + if (b < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value"); + + if (b == bit_limit || static_cast(b) > bit_limit) { // insert default value out_vec.push_back(0); @@ -111,9 +114,10 @@ struct BitShiftLeftImpl const UInt8 word_size = 8; size_t n = end - pos; const UInt128 bit_limit = static_cast(word_size) * n; - if (b < 0 || static_cast(b) > bit_limit) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - if (b == bit_limit) + if (b < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value"); + + if (b == bit_limit || static_cast(b) > bit_limit) { // insert default value out_vec.resize_fill(out_vec.size() + n); diff --git a/src/Functions/bitShiftRight.cpp b/src/Functions/bitShiftRight.cpp index 16032b32f68..19ea7b8c751 100644 --- a/src/Functions/bitShiftRight.cpp +++ b/src/Functions/bitShiftRight.cpp @@ -26,8 +26,10 @@ struct BitShiftRightImpl { if constexpr (is_big_int_v) throw Exception(ErrorCodes::NOT_IMPLEMENTED, "BitShiftRight is not implemented for big integers as second argument"); - else if (b < 0 || static_cast(b) > 8 * sizeof(A)) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); + else if (b < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value"); + else if (static_cast(b) > 8 * sizeof(A)) + return static_cast(0); else if constexpr (is_big_int_v) return static_cast(a) >> static_cast(b); else @@ -59,9 +61,10 @@ struct BitShiftRightImpl const UInt8 word_size = 8; size_t n = end - pos; const UInt128 bit_limit = static_cast(word_size) * n; - if (b < 0 || static_cast(b) > bit_limit) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - if (b == bit_limit) + if (b < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value"); + + if (b == bit_limit || static_cast(b) > bit_limit) { /// insert default value out_vec.push_back(0); @@ -99,9 +102,10 @@ struct BitShiftRightImpl const UInt8 word_size = 8; size_t n = end - pos; const UInt128 bit_limit = static_cast(word_size) * n; - if (b < 0 || static_cast(b) > bit_limit) - throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value and less or equal to the bit width of the value to shift"); - if (b == bit_limit) + if (b < 0) + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "The number of shift positions needs to be a non-negative value"); + + if (b == bit_limit || static_cast(b) > bit_limit) { // insert default value out_vec.resize_fill(out_vec.size() + n); diff --git a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql index 91e8624057c..6b2961f0555 100644 --- a/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql +++ b/tests/queries/0_stateless/02766_bitshift_with_const_arguments.sql @@ -10,7 +10,7 @@ DROP TABLE IF EXISTS t1; CREATE TABLE t0 (vkey UInt32, pkey UInt32, c0 UInt32) engine = TinyLog; CREATE TABLE t1 (vkey UInt32) ENGINE = AggregatingMergeTree ORDER BY vkey; INSERT INTO t0 VALUES (15, 25000, 58); -SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT ref_5.pkey AS c_2_c2392_6 FROM t0 AS ref_5 WHERE 'J[' < multiIf(ref_5.pkey IN ( SELECT 1 ), bitShiftLeft(multiIf(ref_5.c0 > NULL, '1', ')'), 40), NULL); DROP TABLE t0; DROP TABLE t1; diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference index 33b8cd6ee26..1fda82a9747 100644 --- a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.reference @@ -1,3 +1,9 @@ -- bitShiftRight +0 + +\0\0\0\0\0\0\0\0 -- bitShiftLeft +0 + +\0\0\0\0\0\0\0\0 OK diff --git a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql index aec01753673..340cc1292e4 100644 --- a/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql +++ b/tests/queries/0_stateless/03198_bit_shift_throws_error_for_out_of_bounds.sql @@ -1,17 +1,17 @@ SELECT '-- bitShiftRight'; SELECT bitShiftRight(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftRight(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight(toUInt8(1), 8 + 1); SELECT bitShiftRight('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftRight('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight('hola', 4 * 8 + 1); SELECT bitShiftRight(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftRight(toFixedString('hola', 8), 8 * 8 + 1); SELECT '-- bitShiftLeft'; SELECT bitShiftLeft(1, -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftLeft(toUInt8(1), 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft(toUInt8(1), 8 + 1); SELECT bitShiftLeft('hola', -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftLeft('hola', 4 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft('hola', 4 * 8 + 1); SELECT bitShiftLeft(toFixedString('hola', 8), -1); -- { serverError ARGUMENT_OUT_OF_BOUND } -SELECT bitShiftLeft(toFixedString('hola', 8), 8 * 8 + 1); -- { serverError ARGUMENT_OUT_OF_BOUND } +SELECT bitShiftLeft(toFixedString('hola', 8), 8 * 8 + 1); SELECT 'OK'; \ No newline at end of file From a01c2e3f8c265aceb3042cdee1abafeed4f68485 Mon Sep 17 00:00:00 2001 From: Pervakov Grigorii Date: Thu, 7 Nov 2024 16:51:53 +0300 Subject: [PATCH 27/91] Keep materialized view security overriden context until end of query --- src/Processors/Sinks/SinkToStorage.h | 4 ++++ src/Storages/StorageMaterializedView.cpp | 2 ++ ...67_materialized_view_keeps_security_context.reference | 1 + .../03267_materialized_view_keeps_security_context.sql | 9 +++++++++ 4 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/03267_materialized_view_keeps_security_context.reference create mode 100644 tests/queries/0_stateless/03267_materialized_view_keeps_security_context.sql diff --git a/src/Processors/Sinks/SinkToStorage.h b/src/Processors/Sinks/SinkToStorage.h index c728fa87b1e..4bdcb2fe855 100644 --- a/src/Processors/Sinks/SinkToStorage.h +++ b/src/Processors/Sinks/SinkToStorage.h @@ -5,6 +5,8 @@ namespace DB { +class Context; + /// Sink which is returned from Storage::write. class SinkToStorage : public ExceptionKeepingTransform { @@ -16,12 +18,14 @@ public: const Block & getHeader() const { return inputs.front().getHeader(); } void addTableLock(const TableLockHolder & lock) { table_locks.push_back(lock); } + void addInterpreterContext(std::shared_ptr context) { interpreter_context.emplace_back(std::move(context)); } protected: virtual void consume(Chunk & chunk) = 0; private: std::vector table_locks; + std::vector> interpreter_context; void onConsume(Chunk chunk) override; GenerateResult onGenerate() override; diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index d047b28e076..3289ff1ae25 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -382,6 +382,7 @@ void StorageMaterializedView::read( } query_plan.addStorageHolder(storage); + query_plan.addInterpreterContext(context); query_plan.addTableLock(std::move(lock)); } } @@ -405,6 +406,7 @@ SinkToStoragePtr StorageMaterializedView::write(const ASTPtr & query, const Stor auto sink = storage->write(query, metadata_snapshot, context, async_insert); + sink->addInterpreterContext(context); sink->addTableLock(lock); return sink; } diff --git a/tests/queries/0_stateless/03267_materialized_view_keeps_security_context.reference b/tests/queries/0_stateless/03267_materialized_view_keeps_security_context.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03267_materialized_view_keeps_security_context.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03267_materialized_view_keeps_security_context.sql b/tests/queries/0_stateless/03267_materialized_view_keeps_security_context.sql new file mode 100644 index 00000000000..bb44e4920af --- /dev/null +++ b/tests/queries/0_stateless/03267_materialized_view_keeps_security_context.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.rview; +DROP TABLE IF EXISTS {CLICKHOUSE_DATABASE:Identifier}.wview; + +-- Read from view +CREATE MATERIALIZED VIEW rview ENGINE = File(CSV) POPULATE AS SELECT 1 AS c0; +SELECT 1 FROM rview; + +-- Write through view populate +CREATE MATERIALIZED VIEW wview ENGINE = Join(ALL, INNER, c0) POPULATE AS SELECT 1 AS c0; From 96b59a2ef679b6b23ffcecafd59c05a0ea784ada Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Thu, 7 Nov 2024 13:43:58 +0100 Subject: [PATCH 28/91] Avoid port clash in CoordinationTest/0.TestSummingRaft1 --- src/Coordination/tests/gtest_coordination.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index 9648fdd4530..c56e698766a 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -330,7 +330,7 @@ TYPED_TEST(CoordinationTest, TestSummingRaft1) this->setLogDirectory("./logs"); this->setStateFileDirectory("."); - SummingRaftServer s1(1, "localhost", 44444, this->keeper_context); + SummingRaftServer s1(1, "localhost", 0, this->keeper_context); SCOPE_EXIT(if (std::filesystem::exists("./state")) std::filesystem::remove("./state");); /// Single node is leader From 4e53dda5801cf797a85ad07b9fb55e08aa0cdcf8 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 20:45:31 +0100 Subject: [PATCH 29/91] Use array for conditional mkdir --- docker/server/entrypoint.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 7a990e7d889..5a91d54d32b 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -57,14 +57,14 @@ function create_directory_and_do_chown() { [ -z "$dir" ] && return # ensure directories exist if [ "$DO_CHOWN" = "1" ]; then - mkdir="mkdir" + mkdir=( mkdir ) else # if DO_CHOWN=0 it means that the system does not map root user to "admin" permissions # it mainly happens on NFS mounts where root==nobody for security reasons # thus mkdir MUST run with user id/gid and not from nobody that has zero permissions - mkdir="clickhouse su ""${USER}:${GROUP}"" mkdir" + mkdir=( clickhouse su "${USER}:${GROUP}" mkdir ) fi - if ! $mkdir -p "$dir"; then + if ! "${mkdir[@]}" -p "$dir"; then echo "Couldn't create necessary directory: $dir" exit 1 fi From 3525954fa3cd116bf0b7ec70dc70be3999cf0090 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Nov 2024 20:55:04 +0100 Subject: [PATCH 30/91] Implicit SELECT in clickhouse-local --- programs/local/LocalServer.cpp | 6 +++--- src/Client/ClientBase.cpp | 5 ++++- src/Client/ClientBaseHelpers.cpp | 11 +++++++++-- src/Client/ClientBaseHelpers.h | 4 +++- src/Core/Settings.cpp | 2 ++ .../0_stateless/03267_implicit_select.reference | 5 +++++ tests/queries/0_stateless/03267_implicit_select.sh | 11 +++++++++++ 7 files changed, 37 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/03267_implicit_select.reference create mode 100755 tests/queries/0_stateless/03267_implicit_select.sh diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 1dcef5eb25e..145cac02a3c 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -22,7 +22,6 @@ #include #include #include -#include #include #include #include @@ -31,7 +30,6 @@ #include #include #include -#include #include #include #include @@ -50,7 +48,6 @@ #include #include #include -#include #include #include #include @@ -71,9 +68,11 @@ namespace CurrentMetrics namespace DB { + namespace Setting { extern const SettingsBool allow_introspection_functions; + extern const SettingsBool implicit_select; extern const SettingsLocalFSReadMethod storage_file_read_method; } @@ -126,6 +125,7 @@ void applySettingsOverridesForLocal(ContextMutablePtr context) settings[Setting::allow_introspection_functions] = true; settings[Setting::storage_file_read_method] = LocalFSReadMethod::mmap; + settings[Setting::implicit_select] = true; context->setSettings(settings); } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 0a824753dc0..29abed7e52d 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -2674,7 +2674,10 @@ void ClientBase::runInteractive() #if USE_REPLXX replxx::Replxx::highlighter_callback_t highlight_callback{}; if (getClientConfiguration().getBool("highlight", true)) - highlight_callback = highlight; + highlight_callback = [this](const String & query, std::vector & colors) + { + highlight(query, colors, *client_context); + }; ReplxxLineReader lr( *suggest, diff --git a/src/Client/ClientBaseHelpers.cpp b/src/Client/ClientBaseHelpers.cpp index 156c0c87fb6..ea2a5fd42f5 100644 --- a/src/Client/ClientBaseHelpers.cpp +++ b/src/Client/ClientBaseHelpers.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include @@ -12,6 +14,11 @@ namespace DB { +namespace Setting +{ + extern const SettingsBool implicit_select; +} + /// Should we celebrate a bit? bool isNewYearMode() { @@ -95,7 +102,7 @@ bool isChineseNewYearMode(const String & local_tz) } #if USE_REPLXX -void highlight(const String & query, std::vector & colors) +void highlight(const String & query, std::vector & colors, const Context & context) { using namespace replxx; @@ -135,7 +142,7 @@ void highlight(const String & query, std::vector & colors /// Currently we highlight only the first query in the multi-query mode. - ParserQuery parser(end); + ParserQuery parser(end, false, context.getSettingsRef()[Setting::implicit_select]); ASTPtr ast; bool parse_res = false; diff --git a/src/Client/ClientBaseHelpers.h b/src/Client/ClientBaseHelpers.h index adc1c81b3c5..dcfac21c500 100644 --- a/src/Client/ClientBaseHelpers.h +++ b/src/Client/ClientBaseHelpers.h @@ -11,13 +11,15 @@ namespace DB { +class Context; + /// Should we celebrate a bit? bool isNewYearMode(); bool isChineseNewYearMode(const String & local_tz); #if USE_REPLXX -void highlight(const String & query, std::vector & colors); +void highlight(const String & query, std::vector & colors, const Context & context); #endif } diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index f3ada33cb37..049e29dc8d8 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -5708,6 +5708,8 @@ If enabled, MongoDB tables will return an error when a MongoDB query cannot be b )", 0) \ DECLARE(Bool, implicit_select, false, R"( Allow writing simple SELECT queries without the leading SELECT keyword, which makes it simple for calculator-style usage, e.g. `1 + 2` becomes a valid query. + +In `clickhouse-local` it is enabled by default and can be explicitly disabled. )", 0) \ \ \ diff --git a/tests/queries/0_stateless/03267_implicit_select.reference b/tests/queries/0_stateless/03267_implicit_select.reference new file mode 100644 index 00000000000..97c1fd4333b --- /dev/null +++ b/tests/queries/0_stateless/03267_implicit_select.reference @@ -0,0 +1,5 @@ +3 +3 +3 +Syntax error +3 diff --git a/tests/queries/0_stateless/03267_implicit_select.sh b/tests/queries/0_stateless/03267_implicit_select.sh new file mode 100755 index 00000000000..068fb457bb1 --- /dev/null +++ b/tests/queries/0_stateless/03267_implicit_select.sh @@ -0,0 +1,11 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +$CLICKHOUSE_LOCAL "1 + 2" +$CLICKHOUSE_LOCAL -q "1 + 2" +$CLICKHOUSE_LOCAL --query "1 + 2" +$CLICKHOUSE_LOCAL --implicit_select 0 --query "1 + 2" 2>&1 | grep -oF 'Syntax error' +$CLICKHOUSE_LOCAL --implicit_select 0 --query "SELECT 1 + 2" From 8f98f2333f21566ab62430a8bc9379e6b24f6062 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 20:49:06 +0100 Subject: [PATCH 31/91] Make `clickhouse local` fuse in the repository install RUN --- docker/server/Dockerfile.ubuntu | 52 ++++++++++++++++----------------- 1 file changed, 26 insertions(+), 26 deletions(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index 2b023a9cf03..0fe9a409ee4 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -88,32 +88,32 @@ RUN if [ -n "${single_binary_location_url}" ]; then \ #docker-official-library:on # A fallback to installation from ClickHouse repository -RUN if ! clickhouse local -q "SELECT ''" > /dev/null 2>&1; then \ - apt-get update \ - && apt-get install --yes --no-install-recommends \ - dirmngr \ - gnupg2 \ - && mkdir -p /etc/apt/sources.list.d \ - && GNUPGHOME=$(mktemp -d) \ - && GNUPGHOME="$GNUPGHOME" gpg --batch --no-default-keyring \ - --keyring /usr/share/keyrings/clickhouse-keyring.gpg \ - --keyserver hkp://keyserver.ubuntu.com:80 \ - --recv-keys 3a9ea1193a97b548be1457d48919f6bd2b48d754 \ - && rm -rf "$GNUPGHOME" \ - && chmod +r /usr/share/keyrings/clickhouse-keyring.gpg \ - && echo "${REPOSITORY}" > /etc/apt/sources.list.d/clickhouse.list \ - && echo "installing from repository: ${REPOSITORY}" \ - && apt-get update \ - && for package in ${PACKAGES}; do \ - packages="${packages} ${package}=${VERSION}" \ - ; done \ - && apt-get install --yes --no-install-recommends ${packages} || exit 1 \ - && rm -rf \ - /var/lib/apt/lists/* \ - /var/cache/debconf \ - /tmp/* \ - && apt-get autoremove --purge -yq dirmngr gnupg2 \ - ; fi +# It works unless the clickhouse binary already exists +RUN clickhouse local -q 'SELECT 1' >/dev/null 2>&1 && exit 0 || : \ + ; apt-get update \ + && apt-get install --yes --no-install-recommends \ + dirmngr \ + gnupg2 \ + && mkdir -p /etc/apt/sources.list.d \ + && GNUPGHOME=$(mktemp -d) \ + && GNUPGHOME="$GNUPGHOME" gpg --batch --no-default-keyring \ + --keyring /usr/share/keyrings/clickhouse-keyring.gpg \ + --keyserver hkp://keyserver.ubuntu.com:80 \ + --recv-keys 3a9ea1193a97b548be1457d48919f6bd2b48d754 \ + && rm -rf "$GNUPGHOME" \ + && chmod +r /usr/share/keyrings/clickhouse-keyring.gpg \ + && echo "${REPOSITORY}" > /etc/apt/sources.list.d/clickhouse.list \ + && echo "installing from repository: ${REPOSITORY}" \ + && apt-get update \ + && for package in ${PACKAGES}; do \ + packages="${packages} ${package}=${VERSION}" \ + ; done \ + && apt-get install --yes --no-install-recommends ${packages} || exit 1 \ + && rm -rf \ + /var/lib/apt/lists/* \ + /var/cache/debconf \ + /tmp/* \ + && apt-get autoremove --purge -yq dirmngr gnupg2 # post install # we need to allow "others" access to clickhouse folder, because docker container From 76b6cf96eb3f548bc442f645a8cd8999cf3c6f63 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Nov 2024 21:26:23 +0100 Subject: [PATCH 32/91] Highlight multi-statements in the client --- src/Client/ClientBaseHelpers.cpp | 21 +++++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/src/Client/ClientBaseHelpers.cpp b/src/Client/ClientBaseHelpers.cpp index 156c0c87fb6..f7ecbfeeb43 100644 --- a/src/Client/ClientBaseHelpers.cpp +++ b/src/Client/ClientBaseHelpers.cpp @@ -141,7 +141,24 @@ void highlight(const String & query, std::vector & colors try { - parse_res = parser.parse(token_iterator, ast, expected); + while (true) + { + parse_res = parser.parse(token_iterator, ast, expected); + if (!parse_res) + break; + + if (!token_iterator->isEnd() && token_iterator->type != TokenType::Semicolon) + { + parse_res = false; + break; + } + + while (token_iterator->type == TokenType::Semicolon) + ++token_iterator; + + if (token_iterator->isEnd()) + break; + } } catch (...) { @@ -175,7 +192,7 @@ void highlight(const String & query, std::vector & colors /// Highlight the last error in red. If the parser failed or the lexer found an invalid token, /// or if it didn't parse all the data (except, the data for INSERT query, which is legitimately unparsed) - if ((!parse_res || last_token.isError() || (!token_iterator->isEnd() && token_iterator->type != TokenType::Semicolon)) + if ((!parse_res || last_token.isError()) && !(insert_data && expected.max_parsed_pos >= insert_data) && expected.max_parsed_pos >= prev) { From c8104cb2ee0f366a56bfd79a07071173a8a5a815 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Nov 2024 21:28:06 +0100 Subject: [PATCH 33/91] Correct and unify exit codes --- programs/client/Client.cpp | 10 ++++++---- programs/disks/DisksApp.cpp | 8 +++++--- programs/keeper-client/KeeperClient.cpp | 6 ++++-- programs/keeper/Keeper.cpp | 4 ++-- programs/library-bridge/LibraryBridge.cpp | 2 +- programs/local/LocalServer.cpp | 12 +++++++----- programs/obfuscator/Obfuscator.cpp | 2 +- programs/odbc-bridge/ODBCBridge.cpp | 2 +- programs/server/Server.cpp | 4 ++-- 9 files changed, 29 insertions(+), 21 deletions(-) diff --git a/programs/client/Client.cpp b/programs/client/Client.cpp index d7190444f0b..05e1e61be7b 100644 --- a/programs/client/Client.cpp +++ b/programs/client/Client.cpp @@ -431,7 +431,7 @@ catch (const Exception & e) bool need_print_stack_trace = config().getBool("stacktrace", false) && e.code() != ErrorCodes::NETWORK_ERROR; std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl << std::endl; /// If exception code isn't zero, we should return non-zero return code anyway. - return e.code() ? e.code() : -1; + return static_cast(e.code()) ? e.code() : -1; } catch (...) { @@ -1390,7 +1390,8 @@ int mainEntryClickHouseClient(int argc, char ** argv) catch (const DB::Exception & e) { std::cerr << DB::getExceptionMessage(e, false) << std::endl; - return 1; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } catch (const boost::program_options::error & e) { @@ -1399,7 +1400,8 @@ int mainEntryClickHouseClient(int argc, char ** argv) } catch (...) { - std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 1; + std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } } diff --git a/programs/disks/DisksApp.cpp b/programs/disks/DisksApp.cpp index 610d8eaa638..d6541e99288 100644 --- a/programs/disks/DisksApp.cpp +++ b/programs/disks/DisksApp.cpp @@ -546,16 +546,18 @@ int mainEntryClickHouseDisks(int argc, char ** argv) catch (const DB::Exception & e) { std::cerr << DB::getExceptionMessage(e, false) << std::endl; - return 0; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } catch (const boost::program_options::error & e) { std::cerr << "Bad arguments: " << e.what() << std::endl; - return 0; + return DB::ErrorCodes::BAD_ARGUMENTS; } catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 0; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } } diff --git a/programs/keeper-client/KeeperClient.cpp b/programs/keeper-client/KeeperClient.cpp index 2a426fad7ac..4bdddaec59c 100644 --- a/programs/keeper-client/KeeperClient.cpp +++ b/programs/keeper-client/KeeperClient.cpp @@ -448,7 +448,8 @@ int mainEntryClickHouseKeeperClient(int argc, char ** argv) catch (const DB::Exception & e) { std::cerr << DB::getExceptionMessage(e, false) << std::endl; - return 1; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } catch (const boost::program_options::error & e) { @@ -458,6 +459,7 @@ int mainEntryClickHouseKeeperClient(int argc, char ** argv) catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 1; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } } diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 74af9950e13..936ce15f4c9 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -81,7 +81,7 @@ int mainEntryClickHouseKeeper(int argc, char ** argv) { std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } } @@ -672,7 +672,7 @@ catch (...) /// Poco does not provide stacktrace. tryLogCurrentException("Application"); auto code = getCurrentExceptionCode(); - return code ? code : -1; + return static_cast(code) ? code : -1; } diff --git a/programs/library-bridge/LibraryBridge.cpp b/programs/library-bridge/LibraryBridge.cpp index 261484ac744..62dbd12aaf0 100644 --- a/programs/library-bridge/LibraryBridge.cpp +++ b/programs/library-bridge/LibraryBridge.cpp @@ -13,7 +13,7 @@ int mainEntryClickHouseLibraryBridge(int argc, char ** argv) { std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } } diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 1dcef5eb25e..d6bf0353e89 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -615,12 +615,14 @@ catch (const DB::Exception & e) { bool need_print_stack_trace = getClientConfiguration().getBool("stacktrace", false); std::cerr << getExceptionMessage(e, need_print_stack_trace, true) << std::endl; - return e.code() ? e.code() : -1; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } catch (...) { - std::cerr << getCurrentExceptionMessage(false) << std::endl; - return getCurrentExceptionCode(); + std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; + auto code = DB::getCurrentExceptionCode(); + return static_cast(code) ? code : 1; } void LocalServer::updateLoggerLevel(const String & logs_level) @@ -1029,7 +1031,7 @@ int mainEntryClickHouseLocal(int argc, char ** argv) { std::cerr << DB::getExceptionMessage(e, false) << std::endl; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } catch (const boost::program_options::error & e) { @@ -1040,6 +1042,6 @@ int mainEntryClickHouseLocal(int argc, char ** argv) { std::cerr << DB::getCurrentExceptionMessage(true) << '\n'; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } } diff --git a/programs/obfuscator/Obfuscator.cpp b/programs/obfuscator/Obfuscator.cpp index 324a4573b24..6bd3865b591 100644 --- a/programs/obfuscator/Obfuscator.cpp +++ b/programs/obfuscator/Obfuscator.cpp @@ -1480,5 +1480,5 @@ catch (...) { std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } diff --git a/programs/odbc-bridge/ODBCBridge.cpp b/programs/odbc-bridge/ODBCBridge.cpp index 096d1b2dcca..e5ae3272d40 100644 --- a/programs/odbc-bridge/ODBCBridge.cpp +++ b/programs/odbc-bridge/ODBCBridge.cpp @@ -13,7 +13,7 @@ int mainEntryClickHouseODBCBridge(int argc, char ** argv) { std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5159f95419e..68f262079ff 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -343,7 +343,7 @@ int mainEntryClickHouseServer(int argc, char ** argv) { std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; + return static_cast(code) ? code : 1; } } @@ -2537,7 +2537,7 @@ catch (...) /// Poco does not provide stacktrace. tryLogCurrentException("Application"); auto code = getCurrentExceptionCode(); - return code ? code : -1; + return static_cast(code) ? code : -1; } std::unique_ptr Server::buildProtocolStackFromConfig( From a027f1bf3cde1442a427610cf17967147cb0d60c Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Thu, 7 Nov 2024 15:59:11 -0500 Subject: [PATCH 34/91] Revert "Revert "Enable enable_job_stack_trace by default"" --- src/Core/Settings.cpp | 2 +- src/Core/SettingsChangesHistory.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.cpp b/src/Core/Settings.cpp index 6f0109fa300..01339226c2d 100644 --- a/src/Core/Settings.cpp +++ b/src/Core/Settings.cpp @@ -2869,7 +2869,7 @@ Limit on size of multipart/form-data content. This setting cannot be parsed from DECLARE(Bool, calculate_text_stack_trace, true, R"( Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when a huge amount of wrong queries are executed. In normal cases, you should not disable this option. )", 0) \ - DECLARE(Bool, enable_job_stack_trace, false, R"( + DECLARE(Bool, enable_job_stack_trace, true, R"( Output stack trace of a job creator when job results in exception )", 0) \ DECLARE(Bool, allow_ddl, true, R"( diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index c6223bef2b2..edf4e60706b 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -80,6 +80,7 @@ static std::initializer_list Date: Thu, 7 Nov 2024 16:01:02 -0500 Subject: [PATCH 35/91] move enable_job_stack_trace change to 24.11 --- 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 edf4e60706b..0ff9d0a6833 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -64,6 +64,7 @@ static std::initializer_list Date: Thu, 7 Nov 2024 22:40:06 +0100 Subject: [PATCH 36/91] Update src/Client/ClientBaseHelpers.cpp Co-authored-by: Konstantin Bogdanov --- src/Client/ClientBaseHelpers.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBaseHelpers.cpp b/src/Client/ClientBaseHelpers.cpp index f7ecbfeeb43..555e95f7a25 100644 --- a/src/Client/ClientBaseHelpers.cpp +++ b/src/Client/ClientBaseHelpers.cpp @@ -141,7 +141,7 @@ void highlight(const String & query, std::vector & colors try { - while (true) + while (!token_iterator->isEnd()) { parse_res = parser.parse(token_iterator, ast, expected); if (!parse_res) From 1e87298a1ceafcf10fe0e5586604387bab0c6048 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Nov 2024 22:40:21 +0100 Subject: [PATCH 37/91] Update ClientBaseHelpers.cpp --- src/Client/ClientBaseHelpers.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Client/ClientBaseHelpers.cpp b/src/Client/ClientBaseHelpers.cpp index 555e95f7a25..8bdbab99e13 100644 --- a/src/Client/ClientBaseHelpers.cpp +++ b/src/Client/ClientBaseHelpers.cpp @@ -155,9 +155,6 @@ void highlight(const String & query, std::vector & colors while (token_iterator->type == TokenType::Semicolon) ++token_iterator; - - if (token_iterator->isEnd()) - break; } } catch (...) From dc9e1e047b5cf27dde9dd8b0184cdcdd006202ed Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 7 Nov 2024 23:18:39 +0100 Subject: [PATCH 38/91] Fix tests --- tests/queries/0_stateless/02751_multiquery_with_argument.sh | 2 +- tests/queries/0_stateless/02771_multiple_query_arguments.sh | 2 +- .../02800_clickhouse_local_default_settings.reference | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02751_multiquery_with_argument.sh b/tests/queries/0_stateless/02751_multiquery_with_argument.sh index 4021194656b..4378786c145 100755 --- a/tests/queries/0_stateless/02751_multiquery_with_argument.sh +++ b/tests/queries/0_stateless/02751_multiquery_with_argument.sh @@ -9,7 +9,7 @@ $CLICKHOUSE_LOCAL "SELECT 101;" $CLICKHOUSE_LOCAL "SELECT 102;SELECT 103;" # Invalid SQL. -$CLICKHOUSE_LOCAL "SELECT 200; S" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_LOCAL --implicit-select 0 "SELECT 200; S" 2>&1 | grep -o 'Syntax error' $CLICKHOUSE_LOCAL "; SELECT 201;" 2>&1 | grep -o 'Empty query' $CLICKHOUSE_LOCAL "; S; SELECT 202" 2>&1 | grep -o 'Empty query' diff --git a/tests/queries/0_stateless/02771_multiple_query_arguments.sh b/tests/queries/0_stateless/02771_multiple_query_arguments.sh index ae6e23eb61a..fcc1394573a 100755 --- a/tests/queries/0_stateless/02771_multiple_query_arguments.sh +++ b/tests/queries/0_stateless/02771_multiple_query_arguments.sh @@ -18,4 +18,4 @@ $CLICKHOUSE_LOCAL --query "SELECT 202;" --query "SELECT 202;" $CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELECT 303; SELECT 303" $CLICKHOUSE_LOCAL --query "" --query "" $CLICKHOUSE_LOCAL --query "SELECT 303" --query 2>&1 | grep -o 'Bad arguments' -$CLICKHOUSE_LOCAL --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error' +$CLICKHOUSE_LOCAL --implicit-select 0 --query "SELECT 303" --query "SELE" 2>&1 | grep -o 'Syntax error' diff --git a/tests/queries/0_stateless/02800_clickhouse_local_default_settings.reference b/tests/queries/0_stateless/02800_clickhouse_local_default_settings.reference index 0f18d1a3897..54c6f7ce397 100644 --- a/tests/queries/0_stateless/02800_clickhouse_local_default_settings.reference +++ b/tests/queries/0_stateless/02800_clickhouse_local_default_settings.reference @@ -1,2 +1,3 @@ allow_introspection_functions 1 storage_file_read_method mmap +implicit_select 1 From 6054f43000c645a6a470d06e8d935cf792da3011 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:14:25 +0100 Subject: [PATCH 39/91] Make Vertical format prettier --- src/Formats/PrettyFormatHelpers.cpp | 102 ++ src/Formats/PrettyFormatHelpers.h | 18 + src/Interpreters/InterpreterSystemQuery.cpp | 4 +- .../Formats/Impl/PrettyBlockOutputFormat.cpp | 94 +- .../Formats/Impl/PrettyBlockOutputFormat.h | 5 +- .../Impl/PrettyCompactBlockOutputFormat.cpp | 4 +- .../Impl/PrettySpaceBlockOutputFormat.cpp | 4 +- .../Formats/Impl/VerticalRowOutputFormat.cpp | 28 +- .../Formats/Impl/VerticalRowOutputFormat.h | 3 + .../03268_vertical_pretty_numbers.reference | 1532 +++++++++++++++++ .../03268_vertical_pretty_numbers.sql | 11 + 11 files changed, 1707 insertions(+), 98 deletions(-) create mode 100644 src/Formats/PrettyFormatHelpers.cpp create mode 100644 src/Formats/PrettyFormatHelpers.h create mode 100644 tests/queries/0_stateless/03268_vertical_pretty_numbers.reference create mode 100644 tests/queries/0_stateless/03268_vertical_pretty_numbers.sql diff --git a/src/Formats/PrettyFormatHelpers.cpp b/src/Formats/PrettyFormatHelpers.cpp new file mode 100644 index 00000000000..6e2af036651 --- /dev/null +++ b/src/Formats/PrettyFormatHelpers.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +void writeReadableNumberTipIfSingleValue(WriteBuffer & out, const Chunk & chunk, const FormatSettings & settings, bool color) +{ + if (chunk.getNumRows() == 1 && chunk.getNumColumns() == 1) + writeReadableNumberTip(out, *chunk.getColumns()[0], 0, settings, color); +} + +void writeReadableNumberTip(WriteBuffer & out, const IColumn & column, size_t row, const FormatSettings & settings, bool color) +{ + if (column.isNullAt(row)) + return; + + auto value = column.getFloat64(row); + auto threshold = settings.pretty.output_format_pretty_single_large_number_tip_threshold; + + if (threshold && isFinite(value) && abs(value) > threshold) + { + if (color) + writeCString("\033[90m", out); + writeCString(" -- ", out); + formatReadableQuantity(value, out, 2); + if (color) + writeCString("\033[0m", out); + } +} + + +String highlightDigitGroups(String source) +{ + if (source.size() <= 4) + return source; + + bool is_regular_number = true; + size_t num_digits_before_decimal = 0; + for (auto c : source) + { + if (c == '-' || c == ' ') + continue; + if (c == '.') + break; + if (c >= '0' && c <= '9') + { + ++num_digits_before_decimal; + } + else + { + is_regular_number = false; + break; + } + } + + if (!is_regular_number || num_digits_before_decimal <= 4) + return source; + + String result; + size_t size = source.size(); + result.reserve(2 * size); + + bool before_decimal = true; + size_t digit_num = 0; + for (size_t i = 0; i < size; ++i) + { + auto c = source[i]; + if (before_decimal && c >= '0' && c <= '9') + { + ++digit_num; + size_t offset = num_digits_before_decimal - digit_num; + if (offset && offset % 3 == 0) + { + result += "\033[4m"; + result += c; + result += "\033[0m"; + } + else + { + result += c; + } + } + else if (c == '.') + { + before_decimal = false; + result += c; + } + else + { + result += c; + } + } + + return result; +} + +} diff --git a/src/Formats/PrettyFormatHelpers.h b/src/Formats/PrettyFormatHelpers.h new file mode 100644 index 00000000000..72ab5e3c2a0 --- /dev/null +++ b/src/Formats/PrettyFormatHelpers.h @@ -0,0 +1,18 @@ +#include + +namespace DB +{ + +class Chunk; +class IColumn; +class WriteBuffer; +struct FormatSettings; + +/// Prints text describing the number in the form of: -- 12.34 million +void writeReadableNumberTip(WriteBuffer & out, const IColumn & column, size_t row, const FormatSettings & settings, bool color); +void writeReadableNumberTipIfSingleValue(WriteBuffer & out, const Chunk & chunk, const FormatSettings & settings, bool color); + +/// Underscores digit groups related to thousands using terminal ANSI escape sequences. +String highlightDigitGroups(String source); + +} diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 4c875026ace..b651bfb245e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -795,9 +795,9 @@ BlockIO InterpreterSystemQuery::execute() case Type::WAIT_FAILPOINT: { getContext()->checkAccess(AccessType::SYSTEM_FAILPOINT); - LOG_TRACE(log, "waiting for failpoint {}", query.fail_point_name); + LOG_TRACE(log, "Waiting for failpoint {}", query.fail_point_name); FailPointInjection::pauseFailPoint(query.fail_point_name); - LOG_TRACE(log, "finished failpoint {}", query.fail_point_name); + LOG_TRACE(log, "Finished waiting for failpoint {}", query.fail_point_name); break; } case Type::RESET_COVERAGE: diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index ff1a048029d..e8b55ea423b 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -352,7 +353,8 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind } writeCString(grid_symbols.bar, out); - writeReadableNumberTip(chunk); + if (readable_number_tip) + writeReadableNumberTipIfSingleValue(out, chunk, format_settings, color); writeCString("\n", out); } @@ -392,72 +394,6 @@ void PrettyBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port_kind } -static String highlightDigitGroups(String source) -{ - if (source.size() <= 4) - return source; - - bool is_regular_number = true; - size_t num_digits_before_decimal = 0; - for (auto c : source) - { - if (c == '-' || c == ' ') - continue; - if (c == '.') - break; - if (c >= '0' && c <= '9') - { - ++num_digits_before_decimal; - } - else - { - is_regular_number = false; - break; - } - } - - if (!is_regular_number || num_digits_before_decimal <= 4) - return source; - - String result; - size_t size = source.size(); - result.reserve(2 * size); - - bool before_decimal = true; - size_t digit_num = 0; - for (size_t i = 0; i < size; ++i) - { - auto c = source[i]; - if (before_decimal && c >= '0' && c <= '9') - { - ++digit_num; - size_t offset = num_digits_before_decimal - digit_num; - if (offset && offset % 3 == 0) - { - result += "\033[4m"; - result += c; - result += "\033[0m"; - } - else - { - result += c; - } - } - else if (c == '.') - { - before_decimal = false; - result += c; - } - else - { - result += c; - } - } - - return result; -} - - void PrettyBlockOutputFormat::writeValueWithPadding( const IColumn & column, const ISerialization & serialization, size_t row_num, size_t value_width, size_t pad_to_width, size_t cut_to_width, bool align_right, bool is_number) @@ -553,30 +489,6 @@ void PrettyBlockOutputFormat::writeSuffix() } } -void PrettyBlockOutputFormat::writeReadableNumberTip(const Chunk & chunk) -{ - const auto & columns = chunk.getColumns(); - auto is_single_number = readable_number_tip && chunk.getNumRows() == 1 && chunk.getNumColumns() == 1; - if (!is_single_number) - return; - - if (columns[0]->isNullAt(0)) - return; - - auto value = columns[0]->getFloat64(0); - auto threshold = format_settings.pretty.output_format_pretty_single_large_number_tip_threshold; - - if (threshold && isFinite(value) && abs(value) > threshold) - { - if (color) - writeCString("\033[90m", out); - writeCString(" -- ", out); - formatReadableQuantity(value, out, 2); - if (color) - writeCString("\033[0m", out); - } -} - void registerOutputFormatPretty(FormatFactory & factory) { registerPrettyFormatWithNoEscapesAndMonoBlock(factory, "Pretty"); diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 698efecd4b2..824a2fd2e6f 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -38,7 +38,6 @@ protected: virtual void writeChunk(const Chunk & chunk, PortKind port_kind); void writeMonoChunkIfNeeded(); void writeSuffix() override; - void writeReadableNumberTip(const Chunk & chunk); void onRowsReadBeforeUpdate() override { total_rows = getRowsReadBefore(); } @@ -57,8 +56,10 @@ protected: bool color; -private: +protected: bool readable_number_tip = false; + +private: bool mono_block; /// For mono_block == true only Chunk mono_chunk; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp index 57ec23e7e3b..1e4f784bc71 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -189,7 +190,8 @@ void PrettyCompactBlockOutputFormat::writeRow( } writeCString(grid_symbols.bar, out); - writeReadableNumberTip(chunk); + if (readable_number_tip) + writeReadableNumberTipIfSingleValue(out, chunk, format_settings, color); writeCString("\n", out); } diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp index 0a594b54b12..5b481099e41 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -102,7 +103,8 @@ void PrettySpaceBlockOutputFormat::writeChunk(const Chunk & chunk, PortKind port writeValueWithPadding( *columns[column], *serializations[column], row, cur_width, max_widths[column], cut_to_width, type.shouldAlignRightInPrettyFormats(), isNumber(type)); } - writeReadableNumberTip(chunk); + if (readable_number_tip) + writeReadableNumberTipIfSingleValue(out, chunk, format_settings, color); writeChar('\n', out); } diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp b/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp index 4852af9f0c8..7b0135b3ae4 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.cpp @@ -4,7 +4,10 @@ #include #include #include +#include #include +#include +#include namespace DB @@ -14,6 +17,8 @@ VerticalRowOutputFormat::VerticalRowOutputFormat( WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) : IRowOutputFormat(header_, out_), format_settings(format_settings_) { + color = format_settings.pretty.color == 1 || (format_settings.pretty.color == 2 && format_settings.is_writing_to_terminal); + const auto & sample = getPort(PortKind::Main).getHeader(); size_t columns = sample.columns(); @@ -31,6 +36,7 @@ VerticalRowOutputFormat::VerticalRowOutputFormat( } names_and_paddings.resize(columns); + is_number.resize(columns); for (size_t i = 0; i < columns; ++i) { WriteBufferFromString buf(names_and_paddings[i]); @@ -42,6 +48,7 @@ VerticalRowOutputFormat::VerticalRowOutputFormat( { size_t new_size = max_name_width - name_widths[i] + names_and_paddings[i].size(); names_and_paddings[i].resize(new_size, ' '); + is_number[i] = isNumber(removeNullable(recursiveRemoveLowCardinality(sample.getByPosition(i).type))); } } @@ -61,7 +68,26 @@ void VerticalRowOutputFormat::writeField(const IColumn & column, const ISerializ void VerticalRowOutputFormat::writeValue(const IColumn & column, const ISerialization & serialization, size_t row_num) const { - serialization.serializeText(column, row_num, out, format_settings); + if (color && format_settings.pretty.highlight_digit_groups && is_number[field_number]) + { + String serialized_value; + { + WriteBufferFromString buf(serialized_value); + serialization.serializeText(column, row_num, buf, format_settings); + } + + /// Highlight groups of thousands. + serialized_value = highlightDigitGroups(serialized_value); + out.write(serialized_value.data(), serialized_value.size()); + } + else + { + serialization.serializeText(column, row_num, out, format_settings); + } + + /// Write a tip. + if (is_number[field_number]) + writeReadableNumberTip(out, column, row_num, format_settings, color); } diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h index 5870c3503fc..6fe79adc9be 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -56,6 +56,9 @@ private: using NamesAndPaddings = std::vector; NamesAndPaddings names_and_paddings; + + std::vector is_number; + bool color; }; } diff --git a/tests/queries/0_stateless/03268_vertical_pretty_numbers.reference b/tests/queries/0_stateless/03268_vertical_pretty_numbers.reference new file mode 100644 index 00000000000..397e9145798 --- /dev/null +++ b/tests/queries/0_stateless/03268_vertical_pretty_numbers.reference @@ -0,0 +1,1532 @@ +Row 1: +────── +exp2(number): 1 +exp10(number): 1 +concat('test', number): test0 + +Row 2: +────── +exp2(number): 2 -- 2.00 +exp10(number): 10 -- 10.00 +concat('test', number): test1 + +Row 3: +────── +exp2(number): 4 -- 4.00 +exp10(number): 100 -- 100.00 +concat('test', number): test2 + +Row 4: +────── +exp2(number): 8 -- 8.00 +exp10(number): 1000 -- 1.00 thousand +concat('test', number): test3 + +Row 5: +────── +exp2(number): 16 -- 16.00 +exp10(number): 10000 -- 10.00 thousand +concat('test', number): test4 + +Row 6: +────── +exp2(number): 32 -- 32.00 +exp10(number): 100000 -- 100.00 thousand +concat('test', number): test5 + +Row 7: +────── +exp2(number): 64 -- 64.00 +exp10(number): 1000000 -- 1.00 million +concat('test', number): test6 + +Row 8: +────── +exp2(number): 128 -- 128.00 +exp10(number): 10000000 -- 10.00 million +concat('test', number): test7 + +Row 9: +─────── +exp2(number): 256 -- 256.00 +exp10(number): 100000000 -- 100.00 million +concat('test', number): test8 + +Row 10: +─────── +exp2(number): 512 -- 512.00 +exp10(number): 1000000000 -- 1.00 billion +concat('test', number): test9 + +Row 11: +─────── +exp2(number): 1024 -- 1.02 thousand +exp10(number): 10000000000 -- 10.00 billion +concat('test', number): test10 + +Row 12: +─────── +exp2(number): 2048 -- 2.05 thousand +exp10(number): 100000000000 -- 100.00 billion +concat('test', number): test11 + +Row 13: +─────── +exp2(number): 4096 -- 4.10 thousand +exp10(number): 1000000000000 -- 1.00 trillion +concat('test', number): test12 + +Row 14: +─────── +exp2(number): 8192 -- 8.19 thousand +exp10(number): 10000000000000 -- 10.00 trillion +concat('test', number): test13 + +Row 15: +─────── +exp2(number): 16384 -- 16.38 thousand +exp10(number): 100000000000000 -- 100.00 trillion +concat('test', number): test14 + +Row 16: +─────── +exp2(number): 32768 -- 32.77 thousand +exp10(number): 1000000000000000 -- 1.00 quadrillion +concat('test', number): test15 + +Row 17: +─────── +exp2(number): 65536 -- 65.54 thousand +exp10(number): 10000000000000000 -- 10.00 quadrillion +concat('test', number): test16 + +Row 18: +─────── +exp2(number): 131072 -- 131.07 thousand +exp10(number): 100000000000000000 -- 100.00 quadrillion +concat('test', number): test17 + +Row 19: +─────── +exp2(number): 262144 -- 262.14 thousand +exp10(number): 1000000000000000000 -- 1.00 quintillion +concat('test', number): test18 + +Row 20: +─────── +exp2(number): 524288 -- 524.29 thousand +exp10(number): 10000000000000000000 -- 10.00 quintillion +concat('test', number): test19 + +Row 21: +─────── +exp2(number): 1048576 -- 1.05 million +exp10(number): 100000000000000000000 -- 100.00 quintillion +concat('test', number): test20 + +Row 22: +─────── +exp2(number): 2097152 -- 2.10 million +exp10(number): 1e21 -- 1.00 sextillion +concat('test', number): test21 + +Row 23: +─────── +exp2(number): 4194304 -- 4.19 million +exp10(number): 1e22 -- 10.00 sextillion +concat('test', number): test22 + +Row 24: +─────── +exp2(number): 8388608 -- 8.39 million +exp10(number): 1e23 -- 100.00 sextillion +concat('test', number): test23 + +Row 25: +─────── +exp2(number): 16777216 -- 16.78 million +exp10(number): 1e24 -- 1.00 septillion +concat('test', number): test24 + +Row 26: +─────── +exp2(number): 33554432 -- 33.55 million +exp10(number): 1e25 -- 10.00 septillion +concat('test', number): test25 + +Row 27: +─────── +exp2(number): 67108864 -- 67.11 million +exp10(number): 1e26 -- 100.00 septillion +concat('test', number): test26 + +Row 28: +─────── +exp2(number): 134217728 -- 134.22 million +exp10(number): 1e27 -- 1.00 octillion +concat('test', number): test27 + +Row 29: +─────── +exp2(number): 268435456 -- 268.44 million +exp10(number): 1e28 -- 10.00 octillion +concat('test', number): test28 + +Row 30: +─────── +exp2(number): 536870912 -- 536.87 million +exp10(number): 1e29 -- 100.00 octillion +concat('test', number): test29 + +Row 31: +─────── +exp2(number): 1073741824 -- 1.07 billion +exp10(number): 1e30 -- 1.00 nonillion +concat('test', number): test30 + +Row 32: +─────── +exp2(number): 2147483648 -- 2.15 billion +exp10(number): 1e31 -- 10.00 nonillion +concat('test', number): test31 + +Row 33: +─────── +exp2(number): 4294967296 -- 4.29 billion +exp10(number): 1e32 -- 100.00 nonillion +concat('test', number): test32 + +Row 34: +─────── +exp2(number): 8589934592 -- 8.59 billion +exp10(number): 1e33 -- 1000.00 nonillion +concat('test', number): test33 + +Row 35: +─────── +exp2(number): 17179869184 -- 17.18 billion +exp10(number): 1e34 -- 10.00 decillion +concat('test', number): test34 + +Row 36: +─────── +exp2(number): 34359738368 -- 34.36 billion +exp10(number): 1e35 -- 100.00 decillion +concat('test', number): test35 + +Row 37: +─────── +exp2(number): 68719476736 -- 68.72 billion +exp10(number): 1e36 -- 1.00 undecillion +concat('test', number): test36 + +Row 38: +─────── +exp2(number): 137438953472 -- 137.44 billion +exp10(number): 1e37 -- 10.00 undecillion +concat('test', number): test37 + +Row 39: +─────── +exp2(number): 274877906944 -- 274.88 billion +exp10(number): 1e38 -- 100.00 undecillion +concat('test', number): test38 + +Row 40: +─────── +exp2(number): 549755813888 -- 549.76 billion +exp10(number): 1e39 -- 1000.00 undecillion +concat('test', number): test39 + +Row 41: +─────── +exp2(number): 1099511627776 -- 1.10 trillion +exp10(number): 1e40 -- 10.00 duodecillion +concat('test', number): test40 + +Row 42: +─────── +exp2(number): 2199023255552 -- 2.20 trillion +exp10(number): 1e41 -- 100.00 duodecillion +concat('test', number): test41 + +Row 43: +─────── +exp2(number): 4398046511104 -- 4.40 trillion +exp10(number): 1e42 -- 1.00 tredecillion +concat('test', number): test42 + +Row 44: +─────── +exp2(number): 8796093022208 -- 8.80 trillion +exp10(number): 1e43 -- 10.00 tredecillion +concat('test', number): test43 + +Row 45: +─────── +exp2(number): 17592186044416 -- 17.59 trillion +exp10(number): 1e44 -- 100.00 tredecillion +concat('test', number): test44 + +Row 46: +─────── +exp2(number): 35184372088832 -- 35.18 trillion +exp10(number): 1e45 -- 1000.00 tredecillion +concat('test', number): test45 + +Row 47: +─────── +exp2(number): 70368744177664 -- 70.37 trillion +exp10(number): 1e46 -- 10.00 quattuordecillion +concat('test', number): test46 + +Row 48: +─────── +exp2(number): 140737488355328 -- 140.74 trillion +exp10(number): 1e47 -- 100.00 quattuordecillion +concat('test', number): test47 + +Row 49: +─────── +exp2(number): 281474976710656 -- 281.47 trillion +exp10(number): 1e48 -- 1.00 quindecillion +concat('test', number): test48 + +Row 50: +─────── +exp2(number): 562949953421312 -- 562.95 trillion +exp10(number): 1e49 -- 10.00 quindecillion +concat('test', number): test49 + +Row 51: +─────── +exp2(number): 1125899906842624 -- 1.13 quadrillion +exp10(number): 1e50 -- 100.00 quindecillion +concat('test', number): test50 + +Row 52: +─────── +exp2(number): 2251799813685248 -- 2.25 quadrillion +exp10(number): 1e51 -- 1.00 sexdecillion +concat('test', number): test51 + +Row 53: +─────── +exp2(number): 4503599627370496 -- 4.50 quadrillion +exp10(number): 1e52 -- 10.00 sexdecillion +concat('test', number): test52 + +Row 54: +─────── +exp2(number): 9007199254740992 -- 9.01 quadrillion +exp10(number): 1e53 -- 100.00 sexdecillion +concat('test', number): test53 + +Row 55: +─────── +exp2(number): 18014398509481984 -- 18.01 quadrillion +exp10(number): 1e54 -- 1.00 septendecillion +concat('test', number): test54 + +Row 56: +─────── +exp2(number): 36028797018963970 -- 36.03 quadrillion +exp10(number): 1e55 -- 10.00 septendecillion +concat('test', number): test55 + +Row 57: +─────── +exp2(number): 72057594037927940 -- 72.06 quadrillion +exp10(number): 1e56 -- 100.00 septendecillion +concat('test', number): test56 + +Row 58: +─────── +exp2(number): 144115188075855870 -- 144.12 quadrillion +exp10(number): 1e57 -- 1.00 octodecillion +concat('test', number): test57 + +Row 59: +─────── +exp2(number): 288230376151711740 -- 288.23 quadrillion +exp10(number): 1e58 -- 10.00 octodecillion +concat('test', number): test58 + +Row 60: +─────── +exp2(number): 576460752303423500 -- 576.46 quadrillion +exp10(number): 1e59 -- 100.00 octodecillion +concat('test', number): test59 + +Row 61: +─────── +exp2(number): 1152921504606847000 -- 1.15 quintillion +exp10(number): 1e60 -- 1000.00 octodecillion +concat('test', number): test60 + +Row 62: +─────── +exp2(number): 2305843009213694000 -- 2.31 quintillion +exp10(number): 1e61 -- 10.00 novemdecillion +concat('test', number): test61 + +Row 63: +─────── +exp2(number): 4611686018427388000 -- 4.61 quintillion +exp10(number): 1e62 -- 100.00 novemdecillion +concat('test', number): test62 + +Row 64: +─────── +exp2(number): 9223372036854776000 -- 9.22 quintillion +exp10(number): 1e63 -- 1.00 vigintillion +concat('test', number): test63 +Row 1: +────── +exp2(number): 1 +exp10(number): 1 +concat('test', number): test0 + +Row 2: +────── +exp2(number): 2 -- 2.00 +exp10(number): 10 -- 10.00 +concat('test', number): test1 + +Row 3: +────── +exp2(number): 4 -- 4.00 +exp10(number): 100 -- 100.00 +concat('test', number): test2 + +Row 4: +────── +exp2(number): 8 -- 8.00 +exp10(number): 1000 -- 1.00 thousand +concat('test', number): test3 + +Row 5: +────── +exp2(number): 16 -- 16.00 +exp10(number): 10000 -- 10.00 thousand +concat('test', number): test4 + +Row 6: +────── +exp2(number): 32 -- 32.00 +exp10(number): 100000 -- 100.00 thousand +concat('test', number): test5 + +Row 7: +────── +exp2(number): 64 -- 64.00 +exp10(number): 1000000 -- 1.00 million +concat('test', number): test6 + +Row 8: +────── +exp2(number): 128 -- 128.00 +exp10(number): 10000000 -- 10.00 million +concat('test', number): test7 + +Row 9: +─────── +exp2(number): 256 -- 256.00 +exp10(number): 100000000 -- 100.00 million +concat('test', number): test8 + +Row 10: +─────── +exp2(number): 512 -- 512.00 +exp10(number): 1000000000 -- 1.00 billion +concat('test', number): test9 + +Row 11: +─────── +exp2(number): 1024 -- 1.02 thousand +exp10(number): 10000000000 -- 10.00 billion +concat('test', number): test10 + +Row 12: +─────── +exp2(number): 2048 -- 2.05 thousand +exp10(number): 100000000000 -- 100.00 billion +concat('test', number): test11 + +Row 13: +─────── +exp2(number): 4096 -- 4.10 thousand +exp10(number): 1000000000000 -- 1.00 trillion +concat('test', number): test12 + +Row 14: +─────── +exp2(number): 8192 -- 8.19 thousand +exp10(number): 10000000000000 -- 10.00 trillion +concat('test', number): test13 + +Row 15: +─────── +exp2(number): 16384 -- 16.38 thousand +exp10(number): 100000000000000 -- 100.00 trillion +concat('test', number): test14 + +Row 16: +─────── +exp2(number): 32768 -- 32.77 thousand +exp10(number): 1000000000000000 -- 1.00 quadrillion +concat('test', number): test15 + +Row 17: +─────── +exp2(number): 65536 -- 65.54 thousand +exp10(number): 10000000000000000 -- 10.00 quadrillion +concat('test', number): test16 + +Row 18: +─────── +exp2(number): 131072 -- 131.07 thousand +exp10(number): 100000000000000000 -- 100.00 quadrillion +concat('test', number): test17 + +Row 19: +─────── +exp2(number): 262144 -- 262.14 thousand +exp10(number): 1000000000000000000 -- 1.00 quintillion +concat('test', number): test18 + +Row 20: +─────── +exp2(number): 524288 -- 524.29 thousand +exp10(number): 10000000000000000000 -- 10.00 quintillion +concat('test', number): test19 + +Row 21: +─────── +exp2(number): 1048576 -- 1.05 million +exp10(number): 100000000000000000000 -- 100.00 quintillion +concat('test', number): test20 + +Row 22: +─────── +exp2(number): 2097152 -- 2.10 million +exp10(number): 1e21 -- 1.00 sextillion +concat('test', number): test21 + +Row 23: +─────── +exp2(number): 4194304 -- 4.19 million +exp10(number): 1e22 -- 10.00 sextillion +concat('test', number): test22 + +Row 24: +─────── +exp2(number): 8388608 -- 8.39 million +exp10(number): 1e23 -- 100.00 sextillion +concat('test', number): test23 + +Row 25: +─────── +exp2(number): 16777216 -- 16.78 million +exp10(number): 1e24 -- 1.00 septillion +concat('test', number): test24 + +Row 26: +─────── +exp2(number): 33554432 -- 33.55 million +exp10(number): 1e25 -- 10.00 septillion +concat('test', number): test25 + +Row 27: +─────── +exp2(number): 67108864 -- 67.11 million +exp10(number): 1e26 -- 100.00 septillion +concat('test', number): test26 + +Row 28: +─────── +exp2(number): 134217728 -- 134.22 million +exp10(number): 1e27 -- 1.00 octillion +concat('test', number): test27 + +Row 29: +─────── +exp2(number): 268435456 -- 268.44 million +exp10(number): 1e28 -- 10.00 octillion +concat('test', number): test28 + +Row 30: +─────── +exp2(number): 536870912 -- 536.87 million +exp10(number): 1e29 -- 100.00 octillion +concat('test', number): test29 + +Row 31: +─────── +exp2(number): 1073741824 -- 1.07 billion +exp10(number): 1e30 -- 1.00 nonillion +concat('test', number): test30 + +Row 32: +─────── +exp2(number): 2147483648 -- 2.15 billion +exp10(number): 1e31 -- 10.00 nonillion +concat('test', number): test31 + +Row 33: +─────── +exp2(number): 4294967296 -- 4.29 billion +exp10(number): 1e32 -- 100.00 nonillion +concat('test', number): test32 + +Row 34: +─────── +exp2(number): 8589934592 -- 8.59 billion +exp10(number): 1e33 -- 1000.00 nonillion +concat('test', number): test33 + +Row 35: +─────── +exp2(number): 17179869184 -- 17.18 billion +exp10(number): 1e34 -- 10.00 decillion +concat('test', number): test34 + +Row 36: +─────── +exp2(number): 34359738368 -- 34.36 billion +exp10(number): 1e35 -- 100.00 decillion +concat('test', number): test35 + +Row 37: +─────── +exp2(number): 68719476736 -- 68.72 billion +exp10(number): 1e36 -- 1.00 undecillion +concat('test', number): test36 + +Row 38: +─────── +exp2(number): 137438953472 -- 137.44 billion +exp10(number): 1e37 -- 10.00 undecillion +concat('test', number): test37 + +Row 39: +─────── +exp2(number): 274877906944 -- 274.88 billion +exp10(number): 1e38 -- 100.00 undecillion +concat('test', number): test38 + +Row 40: +─────── +exp2(number): 549755813888 -- 549.76 billion +exp10(number): 1e39 -- 1000.00 undecillion +concat('test', number): test39 + +Row 41: +─────── +exp2(number): 1099511627776 -- 1.10 trillion +exp10(number): 1e40 -- 10.00 duodecillion +concat('test', number): test40 + +Row 42: +─────── +exp2(number): 2199023255552 -- 2.20 trillion +exp10(number): 1e41 -- 100.00 duodecillion +concat('test', number): test41 + +Row 43: +─────── +exp2(number): 4398046511104 -- 4.40 trillion +exp10(number): 1e42 -- 1.00 tredecillion +concat('test', number): test42 + +Row 44: +─────── +exp2(number): 8796093022208 -- 8.80 trillion +exp10(number): 1e43 -- 10.00 tredecillion +concat('test', number): test43 + +Row 45: +─────── +exp2(number): 17592186044416 -- 17.59 trillion +exp10(number): 1e44 -- 100.00 tredecillion +concat('test', number): test44 + +Row 46: +─────── +exp2(number): 35184372088832 -- 35.18 trillion +exp10(number): 1e45 -- 1000.00 tredecillion +concat('test', number): test45 + +Row 47: +─────── +exp2(number): 70368744177664 -- 70.37 trillion +exp10(number): 1e46 -- 10.00 quattuordecillion +concat('test', number): test46 + +Row 48: +─────── +exp2(number): 140737488355328 -- 140.74 trillion +exp10(number): 1e47 -- 100.00 quattuordecillion +concat('test', number): test47 + +Row 49: +─────── +exp2(number): 281474976710656 -- 281.47 trillion +exp10(number): 1e48 -- 1.00 quindecillion +concat('test', number): test48 + +Row 50: +─────── +exp2(number): 562949953421312 -- 562.95 trillion +exp10(number): 1e49 -- 10.00 quindecillion +concat('test', number): test49 + +Row 51: +─────── +exp2(number): 1125899906842624 -- 1.13 quadrillion +exp10(number): 1e50 -- 100.00 quindecillion +concat('test', number): test50 + +Row 52: +─────── +exp2(number): 2251799813685248 -- 2.25 quadrillion +exp10(number): 1e51 -- 1.00 sexdecillion +concat('test', number): test51 + +Row 53: +─────── +exp2(number): 4503599627370496 -- 4.50 quadrillion +exp10(number): 1e52 -- 10.00 sexdecillion +concat('test', number): test52 + +Row 54: +─────── +exp2(number): 9007199254740992 -- 9.01 quadrillion +exp10(number): 1e53 -- 100.00 sexdecillion +concat('test', number): test53 + +Row 55: +─────── +exp2(number): 18014398509481984 -- 18.01 quadrillion +exp10(number): 1e54 -- 1.00 septendecillion +concat('test', number): test54 + +Row 56: +─────── +exp2(number): 36028797018963970 -- 36.03 quadrillion +exp10(number): 1e55 -- 10.00 septendecillion +concat('test', number): test55 + +Row 57: +─────── +exp2(number): 72057594037927940 -- 72.06 quadrillion +exp10(number): 1e56 -- 100.00 septendecillion +concat('test', number): test56 + +Row 58: +─────── +exp2(number): 144115188075855870 -- 144.12 quadrillion +exp10(number): 1e57 -- 1.00 octodecillion +concat('test', number): test57 + +Row 59: +─────── +exp2(number): 288230376151711740 -- 288.23 quadrillion +exp10(number): 1e58 -- 10.00 octodecillion +concat('test', number): test58 + +Row 60: +─────── +exp2(number): 576460752303423500 -- 576.46 quadrillion +exp10(number): 1e59 -- 100.00 octodecillion +concat('test', number): test59 + +Row 61: +─────── +exp2(number): 1152921504606847000 -- 1.15 quintillion +exp10(number): 1e60 -- 1000.00 octodecillion +concat('test', number): test60 + +Row 62: +─────── +exp2(number): 2305843009213694000 -- 2.31 quintillion +exp10(number): 1e61 -- 10.00 novemdecillion +concat('test', number): test61 + +Row 63: +─────── +exp2(number): 4611686018427388000 -- 4.61 quintillion +exp10(number): 1e62 -- 100.00 novemdecillion +concat('test', number): test62 + +Row 64: +─────── +exp2(number): 9223372036854776000 -- 9.22 quintillion +exp10(number): 1e63 -- 1.00 vigintillion +concat('test', number): test63 +Row 1: +────── +exp2(number): 1 +exp10(number): 1 +concat('test', number): test0 + +Row 2: +────── +exp2(number): 2 -- 2.00 +exp10(number): 10 -- 10.00 +concat('test', number): test1 + +Row 3: +────── +exp2(number): 4 -- 4.00 +exp10(number): 100 -- 100.00 +concat('test', number): test2 + +Row 4: +────── +exp2(number): 8 -- 8.00 +exp10(number): 1000 -- 1.00 thousand +concat('test', number): test3 + +Row 5: +────── +exp2(number): 16 -- 16.00 +exp10(number): 10000 -- 10.00 thousand +concat('test', number): test4 + +Row 6: +────── +exp2(number): 32 -- 32.00 +exp10(number): 100000 -- 100.00 thousand +concat('test', number): test5 + +Row 7: +────── +exp2(number): 64 -- 64.00 +exp10(number): 1000000 -- 1.00 million +concat('test', number): test6 + +Row 8: +────── +exp2(number): 128 -- 128.00 +exp10(number): 10000000 -- 10.00 million +concat('test', number): test7 + +Row 9: +─────── +exp2(number): 256 -- 256.00 +exp10(number): 100000000 -- 100.00 million +concat('test', number): test8 + +Row 10: +─────── +exp2(number): 512 -- 512.00 +exp10(number): 1000000000 -- 1.00 billion +concat('test', number): test9 + +Row 11: +─────── +exp2(number): 1024 -- 1.02 thousand +exp10(number): 10000000000 -- 10.00 billion +concat('test', number): test10 + +Row 12: +─────── +exp2(number): 2048 -- 2.05 thousand +exp10(number): 100000000000 -- 100.00 billion +concat('test', number): test11 + +Row 13: +─────── +exp2(number): 4096 -- 4.10 thousand +exp10(number): 1000000000000 -- 1.00 trillion +concat('test', number): test12 + +Row 14: +─────── +exp2(number): 8192 -- 8.19 thousand +exp10(number): 10000000000000 -- 10.00 trillion +concat('test', number): test13 + +Row 15: +─────── +exp2(number): 16384 -- 16.38 thousand +exp10(number): 100000000000000 -- 100.00 trillion +concat('test', number): test14 + +Row 16: +─────── +exp2(number): 32768 -- 32.77 thousand +exp10(number): 1000000000000000 -- 1.00 quadrillion +concat('test', number): test15 + +Row 17: +─────── +exp2(number): 65536 -- 65.54 thousand +exp10(number): 10000000000000000 -- 10.00 quadrillion +concat('test', number): test16 + +Row 18: +─────── +exp2(number): 131072 -- 131.07 thousand +exp10(number): 100000000000000000 -- 100.00 quadrillion +concat('test', number): test17 + +Row 19: +─────── +exp2(number): 262144 -- 262.14 thousand +exp10(number): 1000000000000000000 -- 1.00 quintillion +concat('test', number): test18 + +Row 20: +─────── +exp2(number): 524288 -- 524.29 thousand +exp10(number): 10000000000000000000 -- 10.00 quintillion +concat('test', number): test19 + +Row 21: +─────── +exp2(number): 1048576 -- 1.05 million +exp10(number): 100000000000000000000 -- 100.00 quintillion +concat('test', number): test20 + +Row 22: +─────── +exp2(number): 2097152 -- 2.10 million +exp10(number): 1e21 -- 1.00 sextillion +concat('test', number): test21 + +Row 23: +─────── +exp2(number): 4194304 -- 4.19 million +exp10(number): 1e22 -- 10.00 sextillion +concat('test', number): test22 + +Row 24: +─────── +exp2(number): 8388608 -- 8.39 million +exp10(number): 1e23 -- 100.00 sextillion +concat('test', number): test23 + +Row 25: +─────── +exp2(number): 16777216 -- 16.78 million +exp10(number): 1e24 -- 1.00 septillion +concat('test', number): test24 + +Row 26: +─────── +exp2(number): 33554432 -- 33.55 million +exp10(number): 1e25 -- 10.00 septillion +concat('test', number): test25 + +Row 27: +─────── +exp2(number): 67108864 -- 67.11 million +exp10(number): 1e26 -- 100.00 septillion +concat('test', number): test26 + +Row 28: +─────── +exp2(number): 134217728 -- 134.22 million +exp10(number): 1e27 -- 1.00 octillion +concat('test', number): test27 + +Row 29: +─────── +exp2(number): 268435456 -- 268.44 million +exp10(number): 1e28 -- 10.00 octillion +concat('test', number): test28 + +Row 30: +─────── +exp2(number): 536870912 -- 536.87 million +exp10(number): 1e29 -- 100.00 octillion +concat('test', number): test29 + +Row 31: +─────── +exp2(number): 1073741824 -- 1.07 billion +exp10(number): 1e30 -- 1.00 nonillion +concat('test', number): test30 + +Row 32: +─────── +exp2(number): 2147483648 -- 2.15 billion +exp10(number): 1e31 -- 10.00 nonillion +concat('test', number): test31 + +Row 33: +─────── +exp2(number): 4294967296 -- 4.29 billion +exp10(number): 1e32 -- 100.00 nonillion +concat('test', number): test32 + +Row 34: +─────── +exp2(number): 8589934592 -- 8.59 billion +exp10(number): 1e33 -- 1000.00 nonillion +concat('test', number): test33 + +Row 35: +─────── +exp2(number): 17179869184 -- 17.18 billion +exp10(number): 1e34 -- 10.00 decillion +concat('test', number): test34 + +Row 36: +─────── +exp2(number): 34359738368 -- 34.36 billion +exp10(number): 1e35 -- 100.00 decillion +concat('test', number): test35 + +Row 37: +─────── +exp2(number): 68719476736 -- 68.72 billion +exp10(number): 1e36 -- 1.00 undecillion +concat('test', number): test36 + +Row 38: +─────── +exp2(number): 137438953472 -- 137.44 billion +exp10(number): 1e37 -- 10.00 undecillion +concat('test', number): test37 + +Row 39: +─────── +exp2(number): 274877906944 -- 274.88 billion +exp10(number): 1e38 -- 100.00 undecillion +concat('test', number): test38 + +Row 40: +─────── +exp2(number): 549755813888 -- 549.76 billion +exp10(number): 1e39 -- 1000.00 undecillion +concat('test', number): test39 + +Row 41: +─────── +exp2(number): 1099511627776 -- 1.10 trillion +exp10(number): 1e40 -- 10.00 duodecillion +concat('test', number): test40 + +Row 42: +─────── +exp2(number): 2199023255552 -- 2.20 trillion +exp10(number): 1e41 -- 100.00 duodecillion +concat('test', number): test41 + +Row 43: +─────── +exp2(number): 4398046511104 -- 4.40 trillion +exp10(number): 1e42 -- 1.00 tredecillion +concat('test', number): test42 + +Row 44: +─────── +exp2(number): 8796093022208 -- 8.80 trillion +exp10(number): 1e43 -- 10.00 tredecillion +concat('test', number): test43 + +Row 45: +─────── +exp2(number): 17592186044416 -- 17.59 trillion +exp10(number): 1e44 -- 100.00 tredecillion +concat('test', number): test44 + +Row 46: +─────── +exp2(number): 35184372088832 -- 35.18 trillion +exp10(number): 1e45 -- 1000.00 tredecillion +concat('test', number): test45 + +Row 47: +─────── +exp2(number): 70368744177664 -- 70.37 trillion +exp10(number): 1e46 -- 10.00 quattuordecillion +concat('test', number): test46 + +Row 48: +─────── +exp2(number): 140737488355328 -- 140.74 trillion +exp10(number): 1e47 -- 100.00 quattuordecillion +concat('test', number): test47 + +Row 49: +─────── +exp2(number): 281474976710656 -- 281.47 trillion +exp10(number): 1e48 -- 1.00 quindecillion +concat('test', number): test48 + +Row 50: +─────── +exp2(number): 562949953421312 -- 562.95 trillion +exp10(number): 1e49 -- 10.00 quindecillion +concat('test', number): test49 + +Row 51: +─────── +exp2(number): 1125899906842624 -- 1.13 quadrillion +exp10(number): 1e50 -- 100.00 quindecillion +concat('test', number): test50 + +Row 52: +─────── +exp2(number): 2251799813685248 -- 2.25 quadrillion +exp10(number): 1e51 -- 1.00 sexdecillion +concat('test', number): test51 + +Row 53: +─────── +exp2(number): 4503599627370496 -- 4.50 quadrillion +exp10(number): 1e52 -- 10.00 sexdecillion +concat('test', number): test52 + +Row 54: +─────── +exp2(number): 9007199254740992 -- 9.01 quadrillion +exp10(number): 1e53 -- 100.00 sexdecillion +concat('test', number): test53 + +Row 55: +─────── +exp2(number): 18014398509481984 -- 18.01 quadrillion +exp10(number): 1e54 -- 1.00 septendecillion +concat('test', number): test54 + +Row 56: +─────── +exp2(number): 36028797018963970 -- 36.03 quadrillion +exp10(number): 1e55 -- 10.00 septendecillion +concat('test', number): test55 + +Row 57: +─────── +exp2(number): 72057594037927940 -- 72.06 quadrillion +exp10(number): 1e56 -- 100.00 septendecillion +concat('test', number): test56 + +Row 58: +─────── +exp2(number): 144115188075855870 -- 144.12 quadrillion +exp10(number): 1e57 -- 1.00 octodecillion +concat('test', number): test57 + +Row 59: +─────── +exp2(number): 288230376151711740 -- 288.23 quadrillion +exp10(number): 1e58 -- 10.00 octodecillion +concat('test', number): test58 + +Row 60: +─────── +exp2(number): 576460752303423500 -- 576.46 quadrillion +exp10(number): 1e59 -- 100.00 octodecillion +concat('test', number): test59 + +Row 61: +─────── +exp2(number): 1152921504606847000 -- 1.15 quintillion +exp10(number): 1e60 -- 1000.00 octodecillion +concat('test', number): test60 + +Row 62: +─────── +exp2(number): 2305843009213694000 -- 2.31 quintillion +exp10(number): 1e61 -- 10.00 novemdecillion +concat('test', number): test61 + +Row 63: +─────── +exp2(number): 4611686018427388000 -- 4.61 quintillion +exp10(number): 1e62 -- 100.00 novemdecillion +concat('test', number): test62 + +Row 64: +─────── +exp2(number): 9223372036854776000 -- 9.22 quintillion +exp10(number): 1e63 -- 1.00 vigintillion +concat('test', number): test63 +Row 1: +────── +exp2(number): 1 +exp10(number): 1 +concat('test', number): test0 + +Row 2: +────── +exp2(number): 2 +exp10(number): 10 +concat('test', number): test1 + +Row 3: +────── +exp2(number): 4 +exp10(number): 100 +concat('test', number): test2 + +Row 4: +────── +exp2(number): 8 +exp10(number): 1000 +concat('test', number): test3 + +Row 5: +────── +exp2(number): 16 +exp10(number): 10000 +concat('test', number): test4 + +Row 6: +────── +exp2(number): 32 +exp10(number): 100000 +concat('test', number): test5 + +Row 7: +────── +exp2(number): 64 +exp10(number): 1000000 +concat('test', number): test6 + +Row 8: +────── +exp2(number): 128 +exp10(number): 10000000 +concat('test', number): test7 + +Row 9: +─────── +exp2(number): 256 +exp10(number): 100000000 +concat('test', number): test8 + +Row 10: +─────── +exp2(number): 512 +exp10(number): 1000000000 +concat('test', number): test9 + +Row 11: +─────── +exp2(number): 1024 +exp10(number): 10000000000 +concat('test', number): test10 + +Row 12: +─────── +exp2(number): 2048 +exp10(number): 100000000000 +concat('test', number): test11 + +Row 13: +─────── +exp2(number): 4096 +exp10(number): 1000000000000 +concat('test', number): test12 + +Row 14: +─────── +exp2(number): 8192 +exp10(number): 10000000000000 +concat('test', number): test13 + +Row 15: +─────── +exp2(number): 16384 +exp10(number): 100000000000000 +concat('test', number): test14 + +Row 16: +─────── +exp2(number): 32768 +exp10(number): 1000000000000000 +concat('test', number): test15 + +Row 17: +─────── +exp2(number): 65536 +exp10(number): 10000000000000000 +concat('test', number): test16 + +Row 18: +─────── +exp2(number): 131072 +exp10(number): 100000000000000000 +concat('test', number): test17 + +Row 19: +─────── +exp2(number): 262144 +exp10(number): 1000000000000000000 +concat('test', number): test18 + +Row 20: +─────── +exp2(number): 524288 +exp10(number): 10000000000000000000 +concat('test', number): test19 + +Row 21: +─────── +exp2(number): 1048576 +exp10(number): 100000000000000000000 +concat('test', number): test20 + +Row 22: +─────── +exp2(number): 2097152 +exp10(number): 1e21 +concat('test', number): test21 + +Row 23: +─────── +exp2(number): 4194304 +exp10(number): 1e22 +concat('test', number): test22 + +Row 24: +─────── +exp2(number): 8388608 +exp10(number): 1e23 +concat('test', number): test23 + +Row 25: +─────── +exp2(number): 16777216 +exp10(number): 1e24 +concat('test', number): test24 + +Row 26: +─────── +exp2(number): 33554432 +exp10(number): 1e25 +concat('test', number): test25 + +Row 27: +─────── +exp2(number): 67108864 +exp10(number): 1e26 +concat('test', number): test26 + +Row 28: +─────── +exp2(number): 134217728 +exp10(number): 1e27 +concat('test', number): test27 + +Row 29: +─────── +exp2(number): 268435456 +exp10(number): 1e28 +concat('test', number): test28 + +Row 30: +─────── +exp2(number): 536870912 +exp10(number): 1e29 +concat('test', number): test29 + +Row 31: +─────── +exp2(number): 1073741824 +exp10(number): 1e30 +concat('test', number): test30 + +Row 32: +─────── +exp2(number): 2147483648 +exp10(number): 1e31 +concat('test', number): test31 + +Row 33: +─────── +exp2(number): 4294967296 +exp10(number): 1e32 +concat('test', number): test32 + +Row 34: +─────── +exp2(number): 8589934592 +exp10(number): 1e33 +concat('test', number): test33 + +Row 35: +─────── +exp2(number): 17179869184 +exp10(number): 1e34 +concat('test', number): test34 + +Row 36: +─────── +exp2(number): 34359738368 +exp10(number): 1e35 +concat('test', number): test35 + +Row 37: +─────── +exp2(number): 68719476736 +exp10(number): 1e36 +concat('test', number): test36 + +Row 38: +─────── +exp2(number): 137438953472 +exp10(number): 1e37 +concat('test', number): test37 + +Row 39: +─────── +exp2(number): 274877906944 +exp10(number): 1e38 +concat('test', number): test38 + +Row 40: +─────── +exp2(number): 549755813888 +exp10(number): 1e39 +concat('test', number): test39 + +Row 41: +─────── +exp2(number): 1099511627776 +exp10(number): 1e40 +concat('test', number): test40 + +Row 42: +─────── +exp2(number): 2199023255552 +exp10(number): 1e41 +concat('test', number): test41 + +Row 43: +─────── +exp2(number): 4398046511104 +exp10(number): 1e42 +concat('test', number): test42 + +Row 44: +─────── +exp2(number): 8796093022208 +exp10(number): 1e43 +concat('test', number): test43 + +Row 45: +─────── +exp2(number): 17592186044416 +exp10(number): 1e44 +concat('test', number): test44 + +Row 46: +─────── +exp2(number): 35184372088832 +exp10(number): 1e45 +concat('test', number): test45 + +Row 47: +─────── +exp2(number): 70368744177664 +exp10(number): 1e46 +concat('test', number): test46 + +Row 48: +─────── +exp2(number): 140737488355328 +exp10(number): 1e47 +concat('test', number): test47 + +Row 49: +─────── +exp2(number): 281474976710656 +exp10(number): 1e48 +concat('test', number): test48 + +Row 50: +─────── +exp2(number): 562949953421312 +exp10(number): 1e49 +concat('test', number): test49 + +Row 51: +─────── +exp2(number): 1125899906842624 +exp10(number): 1e50 +concat('test', number): test50 + +Row 52: +─────── +exp2(number): 2251799813685248 +exp10(number): 1e51 +concat('test', number): test51 + +Row 53: +─────── +exp2(number): 4503599627370496 +exp10(number): 1e52 +concat('test', number): test52 + +Row 54: +─────── +exp2(number): 9007199254740992 +exp10(number): 1e53 +concat('test', number): test53 + +Row 55: +─────── +exp2(number): 18014398509481984 +exp10(number): 1e54 +concat('test', number): test54 + +Row 56: +─────── +exp2(number): 36028797018963970 +exp10(number): 1e55 +concat('test', number): test55 + +Row 57: +─────── +exp2(number): 72057594037927940 +exp10(number): 1e56 +concat('test', number): test56 + +Row 58: +─────── +exp2(number): 144115188075855870 +exp10(number): 1e57 +concat('test', number): test57 + +Row 59: +─────── +exp2(number): 288230376151711740 +exp10(number): 1e58 +concat('test', number): test58 + +Row 60: +─────── +exp2(number): 576460752303423500 +exp10(number): 1e59 +concat('test', number): test59 + +Row 61: +─────── +exp2(number): 1152921504606847000 +exp10(number): 1e60 +concat('test', number): test60 + +Row 62: +─────── +exp2(number): 2305843009213694000 +exp10(number): 1e61 +concat('test', number): test61 + +Row 63: +─────── +exp2(number): 4611686018427388000 +exp10(number): 1e62 +concat('test', number): test62 + +Row 64: +─────── +exp2(number): 9223372036854776000 +exp10(number): 1e63 +concat('test', number): test63 diff --git a/tests/queries/0_stateless/03268_vertical_pretty_numbers.sql b/tests/queries/0_stateless/03268_vertical_pretty_numbers.sql new file mode 100644 index 00000000000..0462134ed63 --- /dev/null +++ b/tests/queries/0_stateless/03268_vertical_pretty_numbers.sql @@ -0,0 +1,11 @@ +SET output_format_pretty_color = 1, output_format_pretty_highlight_digit_groups = 1, output_format_pretty_single_large_number_tip_threshold = 1; +SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical; + +SET output_format_pretty_color = 0, output_format_pretty_highlight_digit_groups = 1, output_format_pretty_single_large_number_tip_threshold = 1; +SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical; + +SET output_format_pretty_color = 1, output_format_pretty_highlight_digit_groups = 0, output_format_pretty_single_large_number_tip_threshold = 1; +SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical; + +SET output_format_pretty_color = 0, output_format_pretty_highlight_digit_groups = 0, output_format_pretty_single_large_number_tip_threshold = 0; +SELECT exp2(number), exp10(number), 'test'||number FROM numbers(64) FORMAT Vertical; From a2220233b75a5da3fd5408f77a87df7c6c5e51d2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:25:49 +0100 Subject: [PATCH 40/91] Fix test --- .../0_stateless/02050_clickhouse_local_parsing_exception.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh index 7a92fa6fefe..65563837f55 100755 --- a/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh +++ b/tests/queries/0_stateless/02050_clickhouse_local_parsing_exception.sh @@ -4,5 +4,4 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_LOCAL --query="SELECT number FROM system.numbers INTO OUTFILE test.native.zst FORMAT Native" 2>&1 | grep -q "Code: 62. DB::Exception: Syntax error: failed at position 48 ('test'): test.native.zst FORMAT Native. Expected string literal." && echo 'OK' || echo 'FAIL' ||: - +$CLICKHOUSE_LOCAL --query="SELECT number FROM system.numbers INTO OUTFILE test.native.zst FORMAT Native" 2>&1 | grep -q "Code: 62. DB::Exception: Syntax error: failed at position 48 ('test'): test.native.zst FORMAT Native." && echo 'OK' || echo 'FAIL' ||: From 5ceb19453d108163880e9d7fdd06ec4858606c52 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:26:58 +0100 Subject: [PATCH 41/91] Fix style --- src/Formats/PrettyFormatHelpers.h | 3 +++ 1 file changed, 3 insertions(+) diff --git a/src/Formats/PrettyFormatHelpers.h b/src/Formats/PrettyFormatHelpers.h index 72ab5e3c2a0..b5d679c5a42 100644 --- a/src/Formats/PrettyFormatHelpers.h +++ b/src/Formats/PrettyFormatHelpers.h @@ -1,5 +1,8 @@ +#pragma once + #include + namespace DB { From dd5a573302a3e38e15b3bfbc8cafe75cdc22cc7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:50:13 +0100 Subject: [PATCH 42/91] Reset MergeTree to master --- src/Storages/MergeTree/DataPartsExchange.cpp | 2 +- .../MergeTree/FutureMergedMutatedPart.h | 1 + src/Storages/MergeTree/IDataPartStorage.h | 3 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 + .../MergeTree/IMergeTreeDataPartWriter.cpp | 29 +- .../MergeTree/IMergeTreeDataPartWriter.h | 8 + src/Storages/MergeTree/IMergeTreeReader.h | 1 + .../MergeTree/IMergedBlockOutputStream.cpp | 1 - .../MergeTree/IMergedBlockOutputStream.h | 11 +- src/Storages/MergeTree/KeyCondition.cpp | 25 ++ .../MergeTree/MergeFromLogEntryTask.cpp | 9 +- .../MergeTree/MergePlainMergeTreeTask.cpp | 37 +- .../MergeTree/MergeProjectionPartsTask.cpp | 3 + .../MergeSelectors/TrivialMergeSelector.cpp | 94 +++++ .../MergeSelectors/TrivialMergeSelector.h | 32 ++ .../MergeSelectors/registerMergeSelectors.cpp | 2 + src/Storages/MergeTree/MergeTask.cpp | 42 +- src/Storages/MergeTree/MergeTask.h | 9 + src/Storages/MergeTree/MergeTreeData.cpp | 160 ++++++- src/Storages/MergeTree/MergeTreeData.h | 6 +- .../MergeTree/MergeTreeDataFormatVersion.h | 4 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 34 +- .../MergeTree/MergeTreeDataMergerMutator.h | 2 + .../MergeTree/MergeTreeDataPartBuilder.cpp | 18 +- .../MergeTree/MergeTreeDataPartBuilder.h | 12 +- .../MergeTree/MergeTreeDataPartCompact.cpp | 35 +- .../MergeTree/MergeTreeDataPartCompact.h | 2 + .../MergeTree/MergeTreeDataPartType.h | 1 + .../MergeTree/MergeTreeDataPartWide.cpp | 50 ++- .../MergeTree/MergeTreeDataPartWide.h | 2 + .../MergeTreeDataPartWriterCompact.cpp | 56 +-- .../MergeTreeDataPartWriterCompact.h | 8 +- .../MergeTreeDataPartWriterOnDisk.cpp | 206 ++++----- .../MergeTree/MergeTreeDataPartWriterOnDisk.h | 40 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 81 ++-- .../MergeTree/MergeTreeDataPartWriterWide.h | 13 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 43 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 6 +- .../MergeTree/MergeTreeDataWriter.cpp | 15 +- .../MergeTree/MergeTreeIOSettings.cpp | 4 +- src/Storages/MergeTree/MergeTreeIOSettings.h | 5 +- .../MergeTree/MergeTreeIndexGranularity.cpp | 18 +- .../MergeTree/MergeTreeIndexGranularity.h | 2 +- .../MergeTreeIndexGranularityInfo.cpp | 8 + .../MergeTree/MergeTreeIndexGranularityInfo.h | 1 + .../MergeTreeIndexVectorSimilarity.cpp | 67 +-- .../MergeTreeIndexVectorSimilarity.h | 14 +- .../MergeTree/MergeTreeMarksLoader.cpp | 29 ++ src/Storages/MergeTree/MergeTreeMarksLoader.h | 13 +- .../MergeTree/MergeTreeMutationStatus.cpp | 4 +- src/Storages/MergeTree/MergeTreePartInfo.h | 7 + .../MergeTree/MergeTreePartsMover.cpp | 2 +- .../MergeTree/MergeTreePrefetchedReadPool.cpp | 36 +- .../MergeTree/MergeTreePrefetchedReadPool.h | 1 + src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- src/Storages/MergeTree/MergeTreeReadPool.cpp | 2 + src/Storages/MergeTree/MergeTreeReadPool.h | 1 + .../MergeTree/MergeTreeReadPoolBase.cpp | 105 +++-- .../MergeTree/MergeTreeReadPoolBase.h | 4 + .../MergeTree/MergeTreeReadPoolInOrder.cpp | 2 + .../MergeTree/MergeTreeReadPoolInOrder.h | 1 + .../MergeTreeReadPoolParallelReplicas.cpp | 2 + .../MergeTreeReadPoolParallelReplicas.h | 1 + ...rgeTreeReadPoolParallelReplicasInOrder.cpp | 2 + ...MergeTreeReadPoolParallelReplicasInOrder.h | 1 + src/Storages/MergeTree/MergeTreeReadTask.cpp | 25 +- src/Storages/MergeTree/MergeTreeReadTask.h | 10 +- .../MergeTree/MergeTreeReaderWide.cpp | 2 +- .../MergeTree/MergeTreeSelectAlgorithms.cpp | 5 +- .../MergeTree/MergeTreeSelectAlgorithms.h | 8 +- .../MergeTree/MergeTreeSelectProcessor.cpp | 4 +- .../MergeTree/MergeTreeSelectProcessor.h | 1 - src/Storages/MergeTree/MergeTreeSettings.cpp | 397 +++++++++--------- src/Storages/MergeTree/MergeTreeSink.cpp | 21 +- .../MergeTree/MergedBlockOutputStream.cpp | 2 + .../MergeTree/MergedBlockOutputStream.h | 1 + .../MergedColumnOnlyOutputStream.cpp | 5 +- .../MergeTree/MergedColumnOnlyOutputStream.h | 1 + .../MergeTree/MutateFromLogEntryTask.cpp | 4 + .../MergeTree/MutatePlainMergeTreeTask.cpp | 4 + src/Storages/MergeTree/MutateTask.cpp | 5 +- .../ReplicatedMergeTreeAttachThread.cpp | 90 +--- .../ReplicatedMergeTreeAttachThread.h | 2 - .../MergeTree/ReplicatedMergeTreeQueue.cpp | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 1 + .../ReplicatedMergeTreeRestartingThread.cpp | 92 ++++ .../ReplicatedMergeTreeRestartingThread.h | 4 + .../MergeTree/ReplicatedMergeTreeSink.cpp | 36 +- src/Storages/MergeTree/checkDataPart.cpp | 2 +- 90 files changed, 1398 insertions(+), 768 deletions(-) create mode 100644 src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp create mode 100644 src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.h diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index e13ec5a7515..1d79ae5aacb 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -908,7 +908,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( { part_storage_for_loading->commitTransaction(); - MergeTreeDataPartBuilder builder(data, part_name, volume, part_relative_path, part_dir); + MergeTreeDataPartBuilder builder(data, part_name, volume, part_relative_path, part_dir, getReadSettings()); new_data_part = builder.withPartFormatFromDisk().build(); new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr); diff --git a/src/Storages/MergeTree/FutureMergedMutatedPart.h b/src/Storages/MergeTree/FutureMergedMutatedPart.h index 09fb7b01678..ca607bb4e33 100644 --- a/src/Storages/MergeTree/FutureMergedMutatedPart.h +++ b/src/Storages/MergeTree/FutureMergedMutatedPart.h @@ -22,6 +22,7 @@ struct FutureMergedMutatedPart MergeTreeDataPartFormat part_format; MergeTreePartInfo part_info; MergeTreeData::DataPartsVector parts; + std::vector blocking_parts_to_remove; MergeType merge_type = MergeType::Regular; const MergeTreePartition & getPartition() const { return parts.front()->partition; } diff --git a/src/Storages/MergeTree/IDataPartStorage.h b/src/Storages/MergeTree/IDataPartStorage.h index a09c24c63ab..49d9fbf2291 100644 --- a/src/Storages/MergeTree/IDataPartStorage.h +++ b/src/Storages/MergeTree/IDataPartStorage.h @@ -1,5 +1,4 @@ #pragma once -#include #include #include #include @@ -16,7 +15,7 @@ namespace DB { - +struct ReadSettings; class ReadBufferFromFileBase; class WriteBufferFromFileBase; diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 20d7528d38a..41783ffddb0 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -833,7 +833,7 @@ MergeTreeDataPartBuilder IMergeTreeDataPart::getProjectionPartBuilder(const Stri { const char * projection_extension = is_temp_projection ? ".tmp_proj" : ".proj"; auto projection_storage = getDataPartStorage().getProjection(projection_name + projection_extension, !is_temp_projection); - MergeTreeDataPartBuilder builder(storage, projection_name, projection_storage); + MergeTreeDataPartBuilder builder(storage, projection_name, projection_storage, getReadSettings()); return builder.withPartInfo(MergeListElement::FAKE_RESULT_PART_FOR_PROJECTION).withParentPart(this); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 378832d32a1..b41a1d840e1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -180,6 +180,9 @@ public: void loadRowsCountFileForUnexpectedPart(); + /// Loads marks and saves them into mark cache for specified columns. + virtual void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const = 0; + String getMarksFileExtension() const { return index_granularity_info.mark_type.getFileExtension(); } /// Generate the new name for this part according to `new_part_info` and min/max dates from the old name. diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp index a9f188338e1..dbfdbbdea88 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.cpp @@ -91,6 +91,13 @@ Columns IMergeTreeDataPartWriter::releaseIndexColumns() return result; } +PlainMarksByName IMergeTreeDataPartWriter::releaseCachedMarks() +{ + PlainMarksByName res; + std::swap(cached_marks, res); + return res; +} + SerializationPtr IMergeTreeDataPartWriter::getSerialization(const String & column_name) const { auto it = serializations.find(column_name); @@ -178,24 +185,9 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( const MergeTreeIndexGranularity & computed_index_granularity) { if (part_type == MergeTreeDataPartType::Compact) - return createMergeTreeDataPartCompactWriter( - data_part_name_, - logger_name_, - serializations_, - data_part_storage_, - index_granularity_info_, - storage_settings_, - columns_list, - column_positions, - metadata_snapshot, - virtual_columns, - indices_to_recalc, - stats_to_recalc_, - marks_file_extension_, - default_codec_, - writer_settings, - computed_index_granularity); - + return createMergeTreeDataPartCompactWriter(data_part_name_, logger_name_, serializations_, data_part_storage_, + index_granularity_info_, storage_settings_, columns_list, column_positions, metadata_snapshot, virtual_columns, indices_to_recalc, stats_to_recalc_, + marks_file_extension_, default_codec_, writer_settings, computed_index_granularity); if (part_type == MergeTreeDataPartType::Wide) return createMergeTreeDataPartWideWriter( data_part_name_, @@ -213,7 +205,6 @@ MergeTreeDataPartWriterPtr createMergeTreeDataPartWriter( default_codec_, writer_settings, computed_index_granularity); - throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown part type: {}", part_type.toString()); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h index eb51a1b2922..d1c76505d7c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPartWriter.h +++ b/src/Storages/MergeTree/IMergeTreeDataPartWriter.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB @@ -45,7 +46,12 @@ public: virtual void finish(bool sync) = 0; + virtual size_t getNumberOfOpenStreams() const = 0; + Columns releaseIndexColumns(); + + PlainMarksByName releaseCachedMarks(); + const MergeTreeIndexGranularity & getIndexGranularity() const { return index_granularity; } protected: @@ -69,6 +75,8 @@ protected: MutableDataPartStoragePtr data_part_storage; MutableColumns index_columns; MergeTreeIndexGranularity index_granularity; + /// Marks that will be saved to cache on finish. + PlainMarksByName cached_marks; }; using MergeTreeDataPartWriterPtr = std::unique_ptr; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index d799ce57b40..c68617d3995 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -18,6 +18,7 @@ public: using ValueSizeMap = std::map; using VirtualFields = std::unordered_map; using DeserializeBinaryBulkStateMap = std::map; + using FileStreams = std::map>; IMergeTreeReader( MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_, diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 209b274ee6a..eb904a8e2ef 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -4,7 +4,6 @@ #include #include - namespace DB { diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.h b/src/Storages/MergeTree/IMergedBlockOutputStream.h index f67cf66ee50..7dd6d720170 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.h +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.h @@ -7,7 +7,6 @@ #include #include - namespace DB { @@ -35,6 +34,16 @@ public: return writer->getIndexGranularity(); } + PlainMarksByName releaseCachedMarks() + { + return writer->releaseCachedMarks(); + } + + size_t getNumberOfOpenStreams() const + { + return writer->getNumberOfOpenStreams(); + } + protected: /// Remove all columns marked expired in data_part. Also, clears checksums diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 1506dc38946..17723d341fb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -1446,6 +1447,30 @@ public: IFunctionBase::Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override { + if (const auto * adaptor = typeid_cast(func.get())) + { + if (dynamic_cast(adaptor->getFunction().get()) && kind == Kind::RIGHT_CONST) + { + auto time_zone = extractTimeZoneNameFromColumn(const_arg.column.get(), const_arg.name); + + const IDataType * type_ptr = &type; + if (const auto * low_cardinality_type = typeid_cast(type_ptr)) + type_ptr = low_cardinality_type->getDictionaryType().get(); + + if (type_ptr->isNullable()) + type_ptr = static_cast(*type_ptr).getNestedType().get(); + + DataTypePtr type_with_time_zone; + if (typeid_cast(type_ptr)) + type_with_time_zone = std::make_shared(time_zone); + else if (const auto * dt64 = typeid_cast(type_ptr)) + type_with_time_zone = std::make_shared(dt64->getScale(), time_zone); + else + return {}; /// In case we will have other types with time zone + + return func->getMonotonicityForRange(*type_with_time_zone, left, right); + } + } return func->getMonotonicityForRange(type, left, right); } diff --git a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp index 56d7133dfc3..d7e807c689f 100644 --- a/src/Storages/MergeTree/MergeFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MergeFromLogEntryTask.cpp @@ -335,6 +335,10 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() future_merged_part, task_context); + storage.writePartLog( + PartLogElement::MERGE_PARTS_START, {}, 0, + entry.new_part_name, part, parts, merge_mutate_entry.get(), {}); + transaction_ptr = std::make_unique(storage, NO_TRANSACTION_RAW); merge_task = storage.merger_mutator.mergePartsToTemporaryPart( @@ -352,7 +356,6 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() storage.merging_params, NO_TRANSACTION_PTR); - /// Adjust priority for (auto & item : future_merged_part->parts) priority.value += item->getBytesOnDisk(); @@ -368,6 +371,7 @@ ReplicatedMergeMutateTaskBase::PrepareResult MergeFromLogEntryTask::prepare() bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWriter write_part_log) { part = merge_task->getFuture().get(); + auto cached_marks = merge_task->releaseCachedMarks(); storage.merger_mutator.renameMergedTemporaryPart(part, parts, NO_TRANSACTION_PTR, *transaction_ptr); /// Why we reset task here? Because it holds shared pointer to part and tryRemovePartImmediately will @@ -441,6 +445,9 @@ bool MergeFromLogEntryTask::finalize(ReplicatedMergeMutateTaskBase::PartLogWrite finish_callback = [storage_ptr = &storage]() { storage_ptr->merge_selecting_task->schedule(); }; ProfileEvents::increment(ProfileEvents::ReplicatedPartMerges); + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + addMarksToCache(*part, cached_marks, mark_cache); + write_part_log({}); StorageReplicatedMergeTree::incrementMergedPartsProfileEvent(part->getType()); diff --git a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp index be44177847c..6aca58faf47 100644 --- a/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MergePlainMergeTreeTask.cpp @@ -92,6 +92,10 @@ void MergePlainMergeTreeTask::prepare() future_part, task_context); + storage.writePartLog( + PartLogElement::MERGE_PARTS_START, {}, 0, + future_part->name, new_part, future_part->parts, merge_list_entry.get(), {}); + write_part_log = [this] (const ExecutionStatus & execution_status) { auto profile_counters_snapshot = std::make_shared(profile_counters.getPartiallyAtomicSnapshot()); @@ -121,19 +125,19 @@ void MergePlainMergeTreeTask::prepare() }; merge_task = storage.merger_mutator.mergePartsToTemporaryPart( - future_part, - metadata_snapshot, - merge_list_entry.get(), - {} /* projection_merge_list_element */, - table_lock_holder, - time(nullptr), - task_context, - merge_mutate_entry->tagger->reserved_space, - deduplicate, - deduplicate_by_columns, - cleanup, - storage.merging_params, - txn); + future_part, + metadata_snapshot, + merge_list_entry.get(), + {} /* projection_merge_list_element */, + table_lock_holder, + time(nullptr), + task_context, + merge_mutate_entry->tagger->reserved_space, + deduplicate, + deduplicate_by_columns, + cleanup, + storage.merging_params, + txn); } @@ -148,6 +152,12 @@ void MergePlainMergeTreeTask::finish() ThreadFuzzer::maybeInjectSleep(); ThreadFuzzer::maybeInjectMemoryLimitException(); + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + { + auto marks = merge_task->releaseCachedMarks(); + addMarksToCache(*new_part, marks, mark_cache); + } + write_part_log({}); StorageMergeTree::incrementMergedPartsProfileEvent(new_part->getType()); transfer_profile_counters_to_initial_query(); @@ -159,7 +169,6 @@ void MergePlainMergeTreeTask::finish() ThreadFuzzer::maybeInjectSleep(); ThreadFuzzer::maybeInjectMemoryLimitException(); } - } ContextMutablePtr MergePlainMergeTreeTask::createTaskContext() const diff --git a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp index 4e1bb2f11a7..34cd925a8c6 100644 --- a/src/Storages/MergeTree/MergeProjectionPartsTask.cpp +++ b/src/Storages/MergeTree/MergeProjectionPartsTask.cpp @@ -83,6 +83,9 @@ bool MergeProjectionPartsTask::executeStep() ".tmp_proj"); next_level_parts.push_back(executeHere(tmp_part_merge_task)); + /// FIXME (alesapin) we should use some temporary storage for this, + /// not commit each subprojection part + next_level_parts.back()->getDataPartStorage().commitTransaction(); next_level_parts.back()->is_temp = true; } diff --git a/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp b/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp new file mode 100644 index 00000000000..cd1fa7b01cd --- /dev/null +++ b/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.cpp @@ -0,0 +1,94 @@ +#include +#include + +#include +#include + +#include + + +namespace DB +{ + +void registerTrivialMergeSelector(MergeSelectorFactory & factory) +{ + factory.registerPublicSelector("Trivial", MergeSelectorAlgorithm::TRIVIAL, [](const std::any &) + { + return std::make_shared(); + }); +} + +TrivialMergeSelector::PartsRange TrivialMergeSelector::select( + const PartsRanges & parts_ranges, + size_t max_total_size_to_merge) +{ + size_t num_partitions = parts_ranges.size(); + if (num_partitions == 0) + return {}; + + /// Sort partitions from the largest to smallest in the number of parts. + std::vector sorted_partition_indices; + sorted_partition_indices.reserve(num_partitions); + for (size_t i = 0; i < num_partitions; ++i) + if (parts_ranges[i].size() >= settings.num_parts_to_merge) + sorted_partition_indices.emplace_back(i); + + if (sorted_partition_indices.empty()) + return {}; + + std::sort(sorted_partition_indices.begin(), sorted_partition_indices.end(), + [&](size_t i, size_t j){ return parts_ranges[i].size() > parts_ranges[j].size(); }); + + size_t partition_idx = 0; + size_t left = 0; + size_t right = 0; + + std::vector candidates; + while (candidates.size() < settings.num_ranges_to_choose) + { + const PartsRange & partition = parts_ranges[partition_idx]; + + if (1 + right - left == settings.num_parts_to_merge) + { + ++right; + + size_t total_size = 0; + for (size_t i = left; i < right; ++i) + total_size += partition[i].size; + + if (!max_total_size_to_merge || total_size <= max_total_size_to_merge) + { + candidates.emplace_back(partition.data() + left, partition.data() + right); + if (candidates.size() == settings.num_ranges_to_choose) + break; + } + + left = right; + } + + if (partition.size() - left < settings.num_parts_to_merge) + { + ++partition_idx; + if (partition_idx == sorted_partition_indices.size()) + break; + + left = 0; + right = 0; + } + + ++right; + + if (right < partition.size() && partition[right].level < partition[left].level) + left = right; + } + + if (candidates.empty()) + return {}; + + if (candidates.size() == 1) + return candidates[0]; + + return candidates[thread_local_rng() % candidates.size()]; +} + +} diff --git a/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.h b/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.h new file mode 100644 index 00000000000..6d989aea0fb --- /dev/null +++ b/src/Storages/MergeTree/MergeSelectors/TrivialMergeSelector.h @@ -0,0 +1,32 @@ +#pragma once + +#include + + +namespace DB +{ + +/** Go through partitions starting from the largest (in the number of parts). + * Go through parts from left to right. + * Find the first range of N parts where their level is not decreasing. + * Then continue finding these ranges and find up to M of these ranges. + * Choose a random one from them. + */ +class TrivialMergeSelector : public IMergeSelector +{ +public: + struct Settings + { + size_t num_parts_to_merge = 10; + size_t num_ranges_to_choose = 100; + }; + + PartsRange select( + const PartsRanges & parts_ranges, + size_t max_total_size_to_merge) override; + +private: + const Settings settings; +}; + +} diff --git a/src/Storages/MergeTree/MergeSelectors/registerMergeSelectors.cpp b/src/Storages/MergeTree/MergeSelectors/registerMergeSelectors.cpp index 61f941adc36..6a3c1ef4b2b 100644 --- a/src/Storages/MergeTree/MergeSelectors/registerMergeSelectors.cpp +++ b/src/Storages/MergeTree/MergeSelectors/registerMergeSelectors.cpp @@ -7,6 +7,7 @@ namespace DB void registerSimpleMergeSelector(MergeSelectorFactory & factory); void registerStochasticSimpleMergeSelector(MergeSelectorFactory & factory); +void registerTrivialMergeSelector(MergeSelectorFactory & factory); void registerAllMergeSelector(MergeSelectorFactory & factory); void registerTTLDeleteMergeSelector(MergeSelectorFactory & factory); void registerTTLRecompressMergeSelector(MergeSelectorFactory & factory); @@ -17,6 +18,7 @@ void registerMergeSelectors() registerSimpleMergeSelector(factory); registerStochasticSimpleMergeSelector(factory); + registerTrivialMergeSelector(factory); registerAllMergeSelector(factory); registerTTLDeleteMergeSelector(factory); registerTTLRecompressMergeSelector(factory); diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index b03fb1b12cf..08066113375 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -40,10 +40,22 @@ #include #include +#ifndef NDEBUG + #include +#endif + +#ifdef CLICKHOUSE_CLOUD + #include + #include + #include + #include +#endif + namespace ProfileEvents { extern const Event Merge; + extern const Event MergeSourceParts; extern const Event MergedColumns; extern const Event GatheredColumns; extern const Event MergeTotalMilliseconds; @@ -81,6 +93,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_columns_to_activate; extern const MergeTreeSettingsUInt64 vertical_merge_algorithm_min_rows_to_activate; extern const MergeTreeSettingsBool vertical_merge_remote_filesystem_prefetch; + extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -295,6 +308,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const { ProfileEvents::increment(ProfileEvents::Merge); + ProfileEvents::increment(ProfileEvents::MergeSourceParts, global_ctx->future_part->parts.size()); String local_tmp_prefix; if (global_ctx->need_prefix) @@ -335,13 +349,13 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const if (global_ctx->parent_part) { auto data_part_storage = global_ctx->parent_part->getDataPartStorage().getProjection(local_tmp_part_basename, /* use parent transaction */ false); - builder.emplace(*global_ctx->data, global_ctx->future_part->name, data_part_storage); + builder.emplace(*global_ctx->data, global_ctx->future_part->name, data_part_storage, getReadSettings()); builder->withParentPart(global_ctx->parent_part); } else { auto local_single_disk_volume = std::make_shared("volume_" + global_ctx->future_part->name, global_ctx->disk, 0); - builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename)); + builder.emplace(global_ctx->data->getDataPartBuilder(global_ctx->future_part->name, local_single_disk_volume, local_tmp_part_basename, getReadSettings())); builder->withPartStorageType(global_ctx->future_part->part_format.storage_type); } @@ -533,6 +547,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const } } + bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); + global_ctx->to = std::make_shared( global_ctx->new_data_part, global_ctx->metadata_snapshot, @@ -542,6 +558,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const ctx->compression_codec, global_ctx->txn ? global_ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, + save_marks_in_cache, ctx->blocks_are_granules_size, global_ctx->context->getWriteSettings()); @@ -1072,6 +1089,8 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const ctx->executor = std::make_unique(ctx->column_parts_pipeline); NamesAndTypesList columns_list = {*ctx->it_name_and_type}; + bool save_marks_in_cache = (*global_ctx->data->getSettings())[MergeTreeSetting::prewarm_mark_cache] && global_ctx->context->getMarkCache(); + ctx->column_to = std::make_unique( global_ctx->new_data_part, global_ctx->metadata_snapshot, @@ -1080,6 +1099,7 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const column_pipepline.indexes_to_recalc, getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, + save_marks_in_cache, global_ctx->to->getIndexGranularity()); ctx->column_elems_written = 0; @@ -1117,6 +1137,10 @@ void MergeTask::VerticalMergeStage::finalizeVerticalMergeForOneColumn() const auto changed_checksums = ctx->column_to->fillChecksums(global_ctx->new_data_part, global_ctx->checksums_gathered_columns); global_ctx->checksums_gathered_columns.add(std::move(changed_checksums)); + auto cached_marks = ctx->column_to->releaseCachedMarks(); + for (auto & [name, marks] : cached_marks) + global_ctx->cached_marks.emplace(name, std::move(marks)); + ctx->delayed_streams.emplace_back(std::move(ctx->column_to)); while (ctx->delayed_streams.size() > ctx->max_delayed_streams) @@ -1263,6 +1287,10 @@ bool MergeTask::MergeProjectionsStage::finalizeProjectionsAndWholeMerge() const else global_ctx->to->finalizePart(global_ctx->new_data_part, ctx->need_sync, &global_ctx->storage_columns, &global_ctx->checksums_gathered_columns); + auto cached_marks = global_ctx->to->releaseCachedMarks(); + for (auto & [name, marks] : cached_marks) + global_ctx->cached_marks.emplace(name, std::move(marks)); + global_ctx->new_data_part->getDataPartStorage().precommitTransaction(); global_ctx->promise.set_value(global_ctx->new_data_part); @@ -1385,7 +1413,7 @@ bool MergeTask::execute() } -/// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream +/// Apply merge strategy (Ordinary, Collapsing, Aggregating, etc) to the stream class MergePartsStep : public ITransformingStep { public: @@ -1421,7 +1449,7 @@ public: /// that is going in insertion order. ProcessorPtr merged_transform; - const auto &header = pipeline.getHeader(); + const auto & header = pipeline.getHeader(); const auto input_streams_count = pipeline.getNumStreams(); WriteBuffer * rows_sources_write_buf = nullptr; @@ -1690,7 +1718,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const sort_description, partition_key_columns, global_ctx->merging_params, - (is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources temporaty file is used only for vertical merge + (is_vertical_merge ? RowsSourcesTemporaryFile::FILE_ID : ""), /// rows_sources' temporary file is used only for vertical merge (*data_settings)[MergeTreeSetting::merge_max_block_size], (*data_settings)[MergeTreeSetting::merge_max_block_size_bytes], ctx->blocks_are_granules_size, @@ -1759,6 +1787,10 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); auto builder = merge_parts_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + // Merges are not using concurrency control now. Queries and merges running together could lead to CPU overcommit. + // TODO(serxa): Enable concurrency control for merges. This should be done after CPU scheduler introduction. + builder->setConcurrencyControl(false); + global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index 5a4fb1ec0b8..53792165987 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -132,6 +133,13 @@ public: return nullptr; } + PlainMarksByName releaseCachedMarks() const + { + PlainMarksByName res; + std::swap(global_ctx->cached_marks, res); + return res; + } + bool execute(); private: @@ -209,6 +217,7 @@ private: std::promise promise{}; IMergedBlockOutputStream::WrittenOffsetColumns written_offset_columns{}; + PlainMarksByName cached_marks; MergeTreeTransactionPtr txn; bool need_prefix; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 72a41fcf2c1..b2f35d0a309 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -154,6 +155,7 @@ namespace namespace DB { + namespace Setting { extern const SettingsBool allow_drop_detached; @@ -229,6 +231,12 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString storage_policy; extern const MergeTreeSettingsFloat zero_copy_concurrent_part_removal_max_postpone_ratio; extern const MergeTreeSettingsUInt64 zero_copy_concurrent_part_removal_max_split_times; + extern const MergeTreeSettingsBool prewarm_mark_cache; +} + +namespace ServerSetting +{ + extern const ServerSettingsDouble mark_cache_prewarm_ratio; } namespace ErrorCodes @@ -261,6 +269,7 @@ namespace ErrorCodes extern const int SUPPORT_IS_DISABLED; extern const int TOO_MANY_SIMULTANEOUS_QUERIES; extern const int INCORRECT_QUERY; + extern const int INVALID_SETTING_VALUE; extern const int CANNOT_RESTORE_TABLE; extern const int ZERO_COPY_REPLICATION_ERROR; extern const int NOT_INITIALIZED; @@ -759,6 +768,16 @@ void MergeTreeData::checkProperties( } } + /// If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs. + /// SET allow_experimental_vector_similarity_index = 1; + /// CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; + /// INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000); + /// WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100; + /// As a workaround, force enabled adaptive index granularity for now (it is the default anyways). + if (new_metadata.secondary_indices.hasType("vector_similarity") && (*getSettings())[MergeTreeSetting::index_granularity_bytes] == 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, + "Experimental vector similarity index can only be used with MergeTree setting 'index_granularity_bytes' != 0"); + if (!new_metadata.projections.empty()) { std::unordered_set projections_names; @@ -1423,7 +1442,7 @@ void MergeTreeData::loadUnexpectedDataPart(UnexpectedPartLoadState & state) try { - state.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + state.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartInfo(part_info) .withPartFormatFromDisk() .build(); @@ -1438,7 +1457,7 @@ void MergeTreeData::loadUnexpectedDataPart(UnexpectedPartLoadState & state) /// Build a fake part and mark it as broken in case of filesystem error. /// If the error impacts part directory instead of single files, /// an exception will be thrown during detach and silently ignored. - state.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + state.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); @@ -1472,7 +1491,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( /// Build a fake part and mark it as broken in case of filesystem error. /// If the error impacts part directory instead of single files, /// an exception will be thrown during detach and silently ignored. - res.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + res.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); @@ -1493,7 +1512,7 @@ MergeTreeData::LoadPartResult MergeTreeData::loadDataPart( try { - res.part = getDataPartBuilder(part_name, single_disk_volume, part_name) + res.part = getDataPartBuilder(part_name, single_disk_volume, part_name, getReadSettings()) .withPartInfo(part_info) .withPartFormatFromDisk() .build(); @@ -2324,6 +2343,60 @@ void MergeTreeData::stopOutdatedAndUnexpectedDataPartsLoadingTask() } } +void MergeTreeData::prewarmMarkCacheIfNeeded(ThreadPool & pool) +{ + if (!(*getSettings())[MergeTreeSetting::prewarm_mark_cache]) + return; + + prewarmMarkCache(pool); +} + +void MergeTreeData::prewarmMarkCache(ThreadPool & pool) +{ + auto * mark_cache = getContext()->getMarkCache().get(); + if (!mark_cache) + return; + + auto metadata_snaphost = getInMemoryMetadataPtr(); + auto column_names = getColumnsToPrewarmMarks(*getSettings(), metadata_snaphost->getColumns().getAllPhysical()); + + if (column_names.empty()) + return; + + Stopwatch watch; + LOG_TRACE(log, "Prewarming mark cache"); + + auto data_parts = getDataPartsVectorForInternalUsage(); + + /// Prewarm mark cache firstly for the most fresh parts according + /// to time columns in partition key (if exists) and by modification time. + + auto to_tuple = [](const auto & part) + { + return std::make_tuple(part->getMinMaxDate().second, part->getMinMaxTime().second, part->modification_time); + }; + + std::sort(data_parts.begin(), data_parts.end(), [&to_tuple](const auto & lhs, const auto & rhs) + { + return to_tuple(lhs) > to_tuple(rhs); + }); + + ThreadPoolCallbackRunnerLocal runner(pool, "PrewarmMarks"); + double ratio_to_prewarm = getContext()->getServerSettings()[ServerSetting::mark_cache_prewarm_ratio]; + + for (const auto & part : data_parts) + { + if (mark_cache->sizeInBytes() >= mark_cache->maxSizeInBytes() * ratio_to_prewarm) + break; + + runner([&] { part->loadMarksToCache(column_names, mark_cache); }); + } + + runner.waitForAllToFinishAndRethrowFirstError(); + watch.stop(); + LOG_TRACE(log, "Prewarmed mark cache in {} seconds", watch.elapsedSeconds()); +} + /// Is the part directory old. /// True if its modification time and the modification time of all files inside it is less then threshold. /// (Only files on the first level of nesting are considered). @@ -2655,6 +2728,10 @@ void MergeTreeData::removePartsFinally(const MergeTreeData::DataPartsVector & pa for (const auto & part : parts) { part_log_elem.partition_id = part->info.partition_id; + { + WriteBufferFromString out(part_log_elem.partition); + part->partition.serializeText(part->storage, out, {}); + } part_log_elem.part_name = part->name; part_log_elem.bytes_compressed_on_disk = part->getBytesOnDisk(); part_log_elem.bytes_uncompressed = part->getBytesUncompressedOnDisk(); @@ -3310,6 +3387,16 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental vector similarity index is disabled (turn on setting 'allow_experimental_vector_similarity_index')"); + /// If adaptive index granularity is disabled, certain vector search queries with PREWHERE run into LOGICAL_ERRORs. + /// SET allow_experimental_vector_similarity_index = 1; + /// CREATE TABLE tab (`id` Int32, `vec` Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance') GRANULARITY 100000000) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity_bytes = 0; + /// INSERT INTO tab SELECT number, [toFloat32(number), 0.] FROM numbers(10000); + /// WITH [1., 0.] AS reference_vec SELECT id, L2Distance(vec, reference_vec) FROM tab PREWHERE toLowCardinality(10) ORDER BY L2Distance(vec, reference_vec) ASC LIMIT 100; + /// As a workaround, force enabled adaptive index granularity for now (it is the default anyways). + if (AlterCommands::hasVectorSimilarityIndex(new_metadata) && (*getSettings())[MergeTreeSetting::index_granularity_bytes] == 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, + "Experimental vector similarity index can only be used with MergeTree setting 'index_granularity_bytes' != 0"); + for (const auto & disk : getDisks()) if (!disk->supportsHardLinks() && !commands.isSettingsAlter() && !commands.isCommentAlter()) throw Exception( @@ -3622,6 +3709,9 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context const auto & new_changes = new_metadata.settings_changes->as().changes; local_context->checkMergeTreeSettingsConstraints(*settings_from_storage, new_changes); + bool found_disk_setting = false; + bool found_storage_policy_setting = false; + for (const auto & changed_setting : new_changes) { const auto & setting_name = changed_setting.name; @@ -3645,9 +3735,22 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context } if (setting_name == "storage_policy") + { checkStoragePolicy(local_context->getStoragePolicy(new_value.safeGet())); + found_storage_policy_setting = true; + } + else if (setting_name == "disk") + { + checkStoragePolicy(local_context->getStoragePolicyFromDisk(new_value.safeGet())); + found_disk_setting = true; + } } + if (found_storage_policy_setting && found_disk_setting) + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "MergeTree settings `storage_policy` and `disk` cannot be specified at the same time"); + /// Check if it is safe to reset the settings for (const auto & current_setting : current_changes) { @@ -3732,9 +3835,9 @@ MergeTreeDataPartFormat MergeTreeData::choosePartFormatOnDisk(size_t bytes_uncom } MergeTreeDataPartBuilder MergeTreeData::getDataPartBuilder( - const String & name, const VolumePtr & volume, const String & part_dir) const + const String & name, const VolumePtr & volume, const String & part_dir, const ReadSettings & read_settings_) const { - return MergeTreeDataPartBuilder(*this, name, volume, relative_data_path, part_dir); + return MergeTreeDataPartBuilder(*this, name, volume, relative_data_path, part_dir, read_settings_); } void MergeTreeData::changeSettings( @@ -3746,12 +3849,16 @@ void MergeTreeData::changeSettings( bool has_storage_policy_changed = false; const auto & new_changes = new_settings->as().changes; + StoragePolicyPtr new_storage_policy = nullptr; for (const auto & change : new_changes) { - if (change.name == "storage_policy") + if (change.name == "disk" || change.name == "storage_policy") { - StoragePolicyPtr new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet()); + if (change.name == "disk") + new_storage_policy = getContext()->getStoragePolicyFromDisk(change.value.safeGet()); + else + new_storage_policy = getContext()->getStoragePolicy(change.value.safeGet()); StoragePolicyPtr old_storage_policy = getStoragePolicy(); /// StoragePolicy of different version or name is guaranteed to have different pointer @@ -5812,7 +5919,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(cons /// Load this part from the directory `temp_part_dir`. auto load_part = [&] { - MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, parent_part_dir, part_dir_name); + MergeTreeDataPartBuilder builder(*this, part_name, single_disk_volume, parent_part_dir, part_dir_name, getReadSettings()); builder.withPartFormatFromDisk(); part = std::move(builder).build(); part->version.setCreationTID(Tx::PrehistoricTID, nullptr); @@ -5827,7 +5934,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::loadPartRestoredFromBackup(cons if (!part) { /// Make a fake data part only to copy its files to /detached/. - part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, parent_part_dir, part_dir_name} + part = MergeTreeDataPartBuilder{*this, part_name, single_disk_volume, parent_part_dir, part_dir_name, getReadSettings()} .withPartStorageType(MergeTreeDataPartStorageType::Full) .withPartType(MergeTreeDataPartType::Wide) .build(); @@ -6326,6 +6433,12 @@ DetachedPartsInfo MergeTreeData::getDetachedParts() const for (const auto & disk : getDisks()) { + /// While it is possible to have detached parts on readonly/write-once disks + /// (if they were produced on another machine, where it wasn't readonly) + /// to avoid wasting resources for slow disks, avoid trying to enumerate them. + if (disk->isReadOnly() || disk->isWriteOnce()) + continue; + String detached_path = fs::path(relative_data_path) / DETACHED_DIR_NAME; /// Note: we don't care about TOCTOU issue here. @@ -6473,7 +6586,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeData::tryLoadPartsToAttach(const LOG_DEBUG(log, "Checking part {}", new_name); auto single_disk_volume = std::make_shared("volume_" + old_name, disk); - auto part = getDataPartBuilder(old_name, single_disk_volume, source_dir / new_name) + auto part = getDataPartBuilder(old_name, single_disk_volume, source_dir / new_name, getReadSettings()) .withPartFormatFromDisk() .build(); @@ -7528,7 +7641,7 @@ std::pair MergeTreeData::cloneAn std::string(fs::path(dst_part_storage->getFullRootPath()) / tmp_dst_part_name), with_copy); - auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage) + auto dst_data_part = MergeTreeDataPartBuilder(*this, dst_part_name, dst_part_storage, getReadSettings()) .withPartFormatFromDisk() .build(); @@ -7874,7 +7987,8 @@ try part_log_elem.event_type = type; - if (part_log_elem.event_type == PartLogElement::MERGE_PARTS) + if (part_log_elem.event_type == PartLogElement::MERGE_PARTS + || part_log_elem.event_type == PartLogElement::MERGE_PARTS_START) { if (merge_entry) { @@ -7899,6 +8013,20 @@ try part_log_elem.table_name = table_id.table_name; part_log_elem.table_uuid = table_id.uuid; part_log_elem.partition_id = MergeTreePartInfo::fromPartName(new_part_name, format_version).partition_id; + + { + const DataPart * result_or_source_data_part = nullptr; + if (result_part) + result_or_source_data_part = result_part.get(); + else if (!source_parts.empty()) + result_or_source_data_part = source_parts.at(0).get(); + if (result_or_source_data_part) + { + WriteBufferFromString out(part_log_elem.partition); + result_or_source_data_part->partition.serializeText(*this, out, {}); + } + } + part_log_elem.part_name = new_part_name; if (result_part) @@ -7928,10 +8056,6 @@ try { part_log_elem.profile_counters = profile_counters; } - else - { - LOG_WARNING(log, "Profile counters are not set"); - } part_log->add(std::move(part_log_elem)); } @@ -8786,7 +8910,7 @@ std::pair MergeTreeData::createE VolumePtr data_part_volume = createVolumeFromReservation(reservation, volume); auto tmp_dir_holder = getTemporaryPartDirectoryHolder(EMPTY_PART_TMP_PREFIX + new_part_name); - auto new_data_part = getDataPartBuilder(new_part_name, data_part_volume, EMPTY_PART_TMP_PREFIX + new_part_name) + auto new_data_part = getDataPartBuilder(new_part_name, data_part_volume, EMPTY_PART_TMP_PREFIX + new_part_name, getReadSettings()) .withBytesAndRowsOnDisk(0, 0) .withPartInfo(new_part_info) .build(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 7a9730e8627..fe360907875 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -241,7 +241,7 @@ public: MergeTreeDataPartFormat choosePartFormat(size_t bytes_uncompressed, size_t rows_count) const; MergeTreeDataPartFormat choosePartFormatOnDisk(size_t bytes_uncompressed, size_t rows_count) const; - MergeTreeDataPartBuilder getDataPartBuilder(const String & name, const VolumePtr & volume, const String & part_dir) const; + MergeTreeDataPartBuilder getDataPartBuilder(const String & name, const VolumePtr & volume, const String & part_dir, const ReadSettings & read_settings_) const; /// Auxiliary object to add a set of parts into the working set in two steps: /// * First, as PreActive parts (the parts are ready, but not yet in the active set). @@ -506,6 +506,10 @@ public: /// Load the set of data parts from disk. Call once - immediately after the object is created. void loadDataParts(bool skip_sanity_checks, std::optional> expected_parts); + /// Prewarm mark cache for the most recent data parts. + void prewarmMarkCache(ThreadPool & pool); + void prewarmMarkCacheIfNeeded(ThreadPool & pool); + String getLogName() const { return log.loadName(); } Int64 getMaxBlockNumber() const; diff --git a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h index 0a84f08ea71..a61938a993c 100644 --- a/src/Storages/MergeTree/MergeTreeDataFormatVersion.h +++ b/src/Storages/MergeTree/MergeTreeDataFormatVersion.h @@ -8,7 +8,7 @@ namespace DB STRONG_TYPEDEF(UInt32, MergeTreeDataFormatVersion) -const MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; -const MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; +static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_OLD_FORMAT_VERSION {0}; +static constexpr MergeTreeDataFormatVersion MERGE_TREE_DATA_MIN_FORMAT_VERSION_WITH_CUSTOM_PARTITIONING {1}; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 8b3c7bdf3fb..176b5c00b0a 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -48,6 +48,16 @@ namespace CurrentMetrics { extern const Metric BackgroundMergesAndMutationsPoolTask; } +namespace ProfileEvents +{ + + extern const Event MergerMutatorsGetPartsForMergeElapsedMicroseconds; + extern const Event MergerMutatorPrepareRangesForMergeElapsedMicroseconds; + extern const Event MergerMutatorSelectPartsForMergeElapsedMicroseconds; + extern const Event MergerMutatorRangesForMergeCount; + extern const Event MergerMutatorPartsInRangesForMergeCount; + extern const Event MergerMutatorSelectRangePartsCount; +} namespace DB { @@ -70,6 +80,8 @@ namespace MergeTreeSetting extern const MergeTreeSettingsBool ttl_only_drop_parts; extern const MergeTreeSettingsUInt64 parts_to_throw_insert; extern const MergeTreeSettingsMergeSelectorAlgorithm merge_selector_algorithm; + extern const MergeTreeSettingsBool merge_selector_enable_heuristic_to_remove_small_parts_at_right; + extern const MergeTreeSettingsFloat merge_selector_base; } namespace ErrorCodes @@ -213,6 +225,7 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart { PartitionIdsHint res; MergeTreeData::DataPartsVector data_parts = getDataPartsToSelectMergeFrom(txn); + if (data_parts.empty()) return res; @@ -270,6 +283,8 @@ MergeTreeDataMergerMutator::PartitionIdsHint MergeTreeDataMergerMutator::getPart MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectMergeFrom( const MergeTreeTransactionPtr & txn, const PartitionIdsHint * partitions_hint) const { + + Stopwatch get_data_parts_for_merge_timer; auto res = getDataPartsToSelectMergeFrom(txn); if (!partitions_hint) return res; @@ -278,6 +293,8 @@ MergeTreeData::DataPartsVector MergeTreeDataMergerMutator::getDataPartsToSelectM { return !partitions_hint->contains(part->info.partition_id); }); + + ProfileEvents::increment(ProfileEvents::MergerMutatorsGetPartsForMergeElapsedMicroseconds, get_data_parts_for_merge_timer.elapsedMicroseconds()); return res; } @@ -355,6 +372,7 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo const MergeTreeTransactionPtr & txn, PreformattedMessage & out_disable_reason) const { + Stopwatch ranges_for_merge_timer; MergeSelectingInfo res; res.current_time = std::time(nullptr); @@ -455,6 +473,10 @@ MergeTreeDataMergerMutator::MergeSelectingInfo MergeTreeDataMergerMutator::getPo prev_part = ∂ } + ProfileEvents::increment(ProfileEvents::MergerMutatorPartsInRangesForMergeCount, res.parts_selected_precondition); + ProfileEvents::increment(ProfileEvents::MergerMutatorRangesForMergeCount, res.parts_ranges.size()); + ProfileEvents::increment(ProfileEvents::MergerMutatorPrepareRangesForMergeElapsedMicroseconds, ranges_for_merge_timer.elapsedMicroseconds()); + return res; } @@ -469,6 +491,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( PreformattedMessage & out_disable_reason, bool dry_run) { + Stopwatch select_parts_from_ranges_timer; const auto data_settings = data.getSettings(); IMergeSelector::PartsRange parts_to_merge; @@ -540,6 +563,9 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( /// Override value from table settings simple_merge_settings.window_size = (*data_settings)[MergeTreeSetting::merge_selector_window_size]; simple_merge_settings.max_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::max_parts_to_merge_at_once]; + simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSetting::merge_selector_enable_heuristic_to_remove_small_parts_at_right]; + simple_merge_settings.base = (*data_settings)[MergeTreeSetting::merge_selector_base]; + if (!(*data_settings)[MergeTreeSetting::min_age_to_force_merge_on_partition_only]) simple_merge_settings.min_age_to_force_merge = (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds]; @@ -565,7 +591,8 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( if (parts_to_merge.empty()) { - out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors)"); + ProfileEvents::increment(ProfileEvents::MergerMutatorSelectPartsForMergeElapsedMicroseconds, select_parts_from_ranges_timer.elapsedMicroseconds()); + out_disable_reason = PreformattedMessage::create("Did not find any parts to merge (with usual merge selectors) in {}ms", select_parts_from_ranges_timer.elapsedMicroseconds() / 1000); return SelectPartsDecision::CANNOT_SELECT; } } @@ -578,8 +605,11 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( parts.push_back(part); } - LOG_DEBUG(log, "Selected {} parts from {} to {}", parts.size(), parts.front()->name, parts.back()->name); + LOG_DEBUG(log, "Selected {} parts from {} to {} in {}ms", parts.size(), parts.front()->name, parts.back()->name, select_parts_from_ranges_timer.elapsedMicroseconds() / 1000); + ProfileEvents::increment(ProfileEvents::MergerMutatorSelectRangePartsCount, parts.size()); + future_part->assign(std::move(parts)); + ProfileEvents::increment(ProfileEvents::MergerMutatorSelectPartsForMergeElapsedMicroseconds, select_parts_from_ranges_timer.elapsedMicroseconds()); return SelectPartsDecision::SELECTED; } diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h index 71fcb93f369..6d209b9f931 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.h +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.h @@ -106,9 +106,11 @@ public: PreformattedMessage & out_disable_reason, bool dry_run = false); + /// Actually the most fresh partition with biggest modification_time String getBestPartitionToOptimizeEntire(const PartitionsInfo & partitions_info) const; /// Useful to quickly get a list of partitions that contain parts that we may want to merge + /// The result is limited by top_number_of_partitions_to_consider_for_merge PartitionIdsHint getPartitionsThatMayBeMerged( size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge_callback, diff --git a/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp b/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp index 37f578b0c25..6ec4bc31d90 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartBuilder.cpp @@ -14,20 +14,22 @@ namespace ErrorCodes } MergeTreeDataPartBuilder::MergeTreeDataPartBuilder( - const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_) + const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_, const ReadSettings & read_settings_) : data(data_) , name(std::move(name_)) , volume(std::move(volume_)) , root_path(std::move(root_path_)) , part_dir(std::move(part_dir_)) + , read_settings(read_settings_) { } MergeTreeDataPartBuilder::MergeTreeDataPartBuilder( - const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_) + const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_, const ReadSettings & read_settings_) : data(data_) , name(std::move(name_)) , part_storage(std::move(part_storage_)) + , read_settings(read_settings_) { } @@ -73,7 +75,8 @@ MutableDataPartStoragePtr MergeTreeDataPartBuilder::getPartStorageByType( MergeTreeDataPartStorageType storage_type_, const VolumePtr & volume_, const String & root_path_, - const String & part_dir_) + const String & part_dir_, + const ReadSettings &) /// Unused here, but used in private repo. { if (!volume_) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create part storage, because volume is not specified"); @@ -112,7 +115,7 @@ MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartType(MergeTreeDataP MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartStorageType(MergeTreeDataPartStorageType storage_type_) { - part_storage = getPartStorageByType(storage_type_, volume, root_path, part_dir); + part_storage = getPartStorageByType(storage_type_, volume, root_path, part_dir, read_settings); return *this; } @@ -126,7 +129,8 @@ MergeTreeDataPartBuilder::PartStorageAndMarkType MergeTreeDataPartBuilder::getPartStorageAndMarkType( const VolumePtr & volume_, const String & root_path_, - const String & part_dir_) + const String & part_dir_, + const ReadSettings & read_settings_) { auto disk = volume_->getDisk(); auto part_relative_path = fs::path(root_path_) / part_dir_; @@ -138,7 +142,7 @@ MergeTreeDataPartBuilder::getPartStorageAndMarkType( if (MarkType::isMarkFileExtension(ext)) { - auto storage = getPartStorageByType(MergeTreeDataPartStorageType::Full, volume_, root_path_, part_dir_); + auto storage = getPartStorageByType(MergeTreeDataPartStorageType::Full, volume_, root_path_, part_dir_, read_settings_); return {std::move(storage), MarkType(ext)}; } } @@ -156,7 +160,7 @@ MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartFormatFromDisk() MergeTreeDataPartBuilder & MergeTreeDataPartBuilder::withPartFormatFromVolume() { assert(volume); - auto [storage, mark_type] = getPartStorageAndMarkType(volume, root_path, part_dir); + auto [storage, mark_type] = getPartStorageAndMarkType(volume, root_path, part_dir, read_settings); if (!storage || !mark_type) { diff --git a/src/Storages/MergeTree/MergeTreeDataPartBuilder.h b/src/Storages/MergeTree/MergeTreeDataPartBuilder.h index 0f54ff0a631..bce881a1970 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartBuilder.h +++ b/src/Storages/MergeTree/MergeTreeDataPartBuilder.h @@ -21,8 +21,8 @@ using VolumePtr = std::shared_ptr; class MergeTreeDataPartBuilder { public: - MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_); - MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_); + MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, VolumePtr volume_, String root_path_, String part_dir_, const ReadSettings & read_settings_); + MergeTreeDataPartBuilder(const MergeTreeData & data_, String name_, MutableDataPartStoragePtr part_storage_, const ReadSettings & read_settings_); std::shared_ptr build(); @@ -42,7 +42,8 @@ public: static PartStorageAndMarkType getPartStorageAndMarkType( const VolumePtr & volume_, const String & root_path_, - const String & part_dir_); + const String & part_dir_, + const ReadSettings & read_settings); private: Self & withPartFormatFromVolume(); @@ -52,7 +53,8 @@ private: MergeTreeDataPartStorageType storage_type_, const VolumePtr & volume_, const String & root_path_, - const String & part_dir_); + const String & part_dir_, + const ReadSettings & read_settings); const MergeTreeData & data; const String name; @@ -64,6 +66,8 @@ private: std::optional part_type; MutableDataPartStoragePtr part_storage; const IMergeTreeDataPart * parent_part = nullptr; + + const ReadSettings read_settings; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp index fd46b3b9540..14c2da82de1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.cpp @@ -136,6 +136,32 @@ void MergeTreeDataPartCompact::loadIndexGranularity() loadIndexGranularityImpl(index_granularity, index_granularity_info, columns.size(), getDataPartStorage()); } +void MergeTreeDataPartCompact::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const +{ + if (column_names.empty() || !mark_cache) + return; + + auto context = storage.getContext(); + auto read_settings = context->getReadSettings(); + auto * load_marks_threadpool = read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + auto info_for_read = std::make_shared(shared_from_this(), std::make_shared()); + + LOG_TEST(getLogger("MergeTreeDataPartCompact"), "Loading marks into mark cache for columns {} of part {}", toString(column_names), name); + + MergeTreeMarksLoader loader( + info_for_read, + mark_cache, + index_granularity_info.getMarksFilePath(DATA_FILE_NAME), + index_granularity.getMarksCount(), + index_granularity_info, + /*save_marks_in_cache=*/ true, + read_settings, + load_marks_threadpool, + columns.size()); + + loader.loadMarks(); +} + bool MergeTreeDataPartCompact::hasColumnFiles(const NameAndTypePair & column) const { if (!getColumnPosition(column.getNameInStorage())) @@ -230,7 +256,14 @@ bool MergeTreeDataPartCompact::isStoredOnRemoteDiskWithZeroCopySupport() const MergeTreeDataPartCompact::~MergeTreeDataPartCompact() { - removeIfNeeded(); + try + { + removeIfNeeded(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } } diff --git a/src/Storages/MergeTree/MergeTreeDataPartCompact.h b/src/Storages/MergeTree/MergeTreeDataPartCompact.h index 9512485c54e..8e279571578 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartCompact.h @@ -54,6 +54,8 @@ public: std::optional getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } + void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const override; + ~MergeTreeDataPartCompact() override; protected: diff --git a/src/Storages/MergeTree/MergeTreeDataPartType.h b/src/Storages/MergeTree/MergeTreeDataPartType.h index 8177809d41e..a59ccc2fab1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartType.h +++ b/src/Storages/MergeTree/MergeTreeDataPartType.h @@ -45,6 +45,7 @@ public: enum Value { Full, + Packed, Unknown, }; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 9bbf0ad9739..c515d645253 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -182,6 +182,47 @@ void MergeTreeDataPartWide::loadIndexGranularity() loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), *any_column_filename); } +void MergeTreeDataPartWide::loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const +{ + if (column_names.empty() || !mark_cache) + return; + + std::vector> loaders; + + auto context = storage.getContext(); + auto read_settings = context->getReadSettings(); + auto * load_marks_threadpool = read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; + auto info_for_read = std::make_shared(shared_from_this(), std::make_shared()); + + LOG_TEST(getLogger("MergeTreeDataPartWide"), "Loading marks into mark cache for columns {} of part {}", toString(column_names), name); + + for (const auto & column_name : column_names) + { + auto serialization = getSerialization(column_name); + serialization->enumerateStreams([&](const auto & subpath) + { + auto stream_name = getStreamNameForColumn(column_name, subpath, checksums); + if (!stream_name) + return; + + loaders.emplace_back(std::make_unique( + info_for_read, + mark_cache, + index_granularity_info.getMarksFilePath(*stream_name), + index_granularity.getMarksCount(), + index_granularity_info, + /*save_marks_in_cache=*/ true, + read_settings, + load_marks_threadpool, + /*num_columns_in_mark=*/ 1)); + + loaders.back()->startAsyncLoad(); + }); + } + + for (auto & loader : loaders) + loader->loadMarks(); +} bool MergeTreeDataPartWide::isStoredOnRemoteDisk() const { @@ -200,7 +241,14 @@ bool MergeTreeDataPartWide::isStoredOnRemoteDiskWithZeroCopySupport() const MergeTreeDataPartWide::~MergeTreeDataPartWide() { - removeIfNeeded(); + try + { + removeIfNeeded(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } void MergeTreeDataPartWide::doCheckConsistency(bool require_part_metadata) const diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.h b/src/Storages/MergeTree/MergeTreeDataPartWide.h index 42893f47573..022a5fb746c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.h @@ -51,6 +51,8 @@ public: std::optional getColumnModificationTime(const String & column_name) const override; + void loadMarksToCache(const Names & column_names, MarkCache * mark_cache) const override; + protected: static void loadIndexGranularityImpl( MergeTreeIndexGranularity & index_granularity_, MergeTreeIndexGranularityInfo & index_granularity_info_, diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index a859172023f..c8d11ced683 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -1,5 +1,6 @@ #include #include +#include "Formats/MarkInCompressedFile.h" namespace DB { @@ -54,26 +55,15 @@ MergeTreeDataPartWriterCompact::MergeTreeDataPartWriterCompact( marks_source_hashing = std::make_unique(*marks_compressor); } + if (settings.save_marks_in_cache) + { + cached_marks[MergeTreeDataPartCompact::DATA_FILE_NAME] = std::make_unique(); + } + for (const auto & column : columns_list) { auto compression = getCodecDescOrDefault(column.name, default_codec); - addStreams(column, nullptr, compression); - } -} - -void MergeTreeDataPartWriterCompact::initDynamicStreamsIfNeeded(const Block & block) -{ - if (is_dynamic_streams_initialized) - return; - - is_dynamic_streams_initialized = true; - for (const auto & column : columns_list) - { - if (column.type->hasDynamicSubcolumns()) - { - auto compression = getCodecDescOrDefault(column.name, default_codec); - addStreams(column, block.getByName(column.name).column, compression); - } + MergeTreeDataPartWriterCompact::addStreams(column, nullptr, compression); } } @@ -175,20 +165,25 @@ void writeColumnSingleGranule( void MergeTreeDataPartWriterCompact::write(const Block & block, const IColumn::Permutation * permutation) { - /// On first block of data initialize streams for dynamic subcolumns. - initDynamicStreamsIfNeeded(block); + Block result_block = block; + + /// During serialization columns with dynamic subcolumns (like JSON/Dynamic) must have the same dynamic structure. + /// But it may happen that they don't (for example during ALTER MODIFY COLUMN from some type to JSON/Dynamic). + /// In this case we use dynamic structure of the column from the first written block and adjust columns from + /// the next blocks so they match this dynamic structure. + initOrAdjustDynamicStructureIfNeeded(result_block); /// Fill index granularity for this block /// if it's unknown (in case of insert data or horizontal merge, /// but not in case of vertical merge) if (compute_granularity) { - size_t index_granularity_for_block = computeIndexGranularity(block); + size_t index_granularity_for_block = computeIndexGranularity(result_block); assert(index_granularity_for_block >= 1); - fillIndexGranularity(index_granularity_for_block, block.rows()); + fillIndexGranularity(index_granularity_for_block, result_block.rows()); } - Block result_block = permuteBlockIfNeeded(block, permutation); + result_block = permuteBlockIfNeeded(result_block, permutation); if (!header) header = result_block.cloneEmpty(); @@ -255,9 +250,12 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G return &result_stream->hashing_buf; }; + MarkInCompressedFile mark{plain_hashing.count(), static_cast(0)}; + writeBinaryLittleEndian(mark.offset_in_compressed_file, marks_out); + writeBinaryLittleEndian(mark.offset_in_decompressed_block, marks_out); - writeBinaryLittleEndian(plain_hashing.count(), marks_out); - writeBinaryLittleEndian(static_cast(0), marks_out); + if (!cached_marks.empty()) + cached_marks.begin()->second->push_back(mark); writeColumnSingleGranule( block.getByName(name_and_type->name), getSerialization(name_and_type->name), @@ -296,11 +294,17 @@ void MergeTreeDataPartWriterCompact::fillDataChecksums(MergeTreeDataPartChecksum if (with_final_mark && data_written) { + MarkInCompressedFile mark{plain_hashing.count(), 0}; + for (size_t i = 0; i < columns_list.size(); ++i) { - writeBinaryLittleEndian(plain_hashing.count(), marks_out); - writeBinaryLittleEndian(static_cast(0), marks_out); + writeBinaryLittleEndian(mark.offset_in_compressed_file, marks_out); + writeBinaryLittleEndian(mark.offset_in_decompressed_block, marks_out); + + if (!cached_marks.empty()) + cached_marks.begin()->second->push_back(mark); } + writeBinaryLittleEndian(static_cast(0), marks_out); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h index b440a37222d..b3e2e78491d 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.h @@ -32,6 +32,8 @@ public: void fillChecksums(MergeTreeDataPartChecksums & checksums, NameSet & checksums_to_remove) override; void finish(bool sync) override; + size_t getNumberOfOpenStreams() const override { return 1; } + private: /// Finish serialization of the data. Flush rows in buffer to disk, compute checksums. void fillDataChecksums(MergeTreeDataPartChecksums & checksums); @@ -48,9 +50,7 @@ private: void addToChecksums(MergeTreeDataPartChecksums & checksums); - void addStreams(const NameAndTypePair & name_and_type, const ColumnPtr & column, const ASTPtr & effective_codec_desc); - - void initDynamicStreamsIfNeeded(const Block & block); + void addStreams(const NameAndTypePair & name_and_type, const ColumnPtr & column, const ASTPtr & effective_codec_desc) override; Block header; @@ -104,8 +104,6 @@ private: /// then finally to 'marks_file'. std::unique_ptr marks_compressor; std::unique_ptr marks_source_hashing; - - bool is_dynamic_streams_initialized = false; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp index 89db8174636..c483d47fed7 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.cpp @@ -3,26 +3,16 @@ #include #include #include -#include #include - namespace ProfileEvents { - extern const Event MergeTreeDataWriterSkipIndicesCalculationMicroseconds; - extern const Event MergeTreeDataWriterStatisticsCalculationMicroseconds; -} - -namespace CurrentMetrics -{ - extern const Metric CompressionThread; - extern const Metric CompressionThreadActive; - extern const Metric CompressionThreadScheduled; +extern const Event MergeTreeDataWriterSkipIndicesCalculationMicroseconds; +extern const Event MergeTreeDataWriterStatisticsCalculationMicroseconds; } namespace DB { - namespace MergeTreeSetting { extern const MergeTreeSettingsUInt64 index_granularity; @@ -35,53 +25,57 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() +template +void MergeTreeDataPartWriterOnDisk::Stream::preFinalize() { /// Here the main goal is to do preFinalize calls for plain_file and marks_file /// Before that all hashing and compression buffers have to be finalized /// Otherwise some data might stuck in the buffers above plain_file and marks_file /// Also the order is important - compressed_hashing->finalize(); - compressor->finalize(); - plain_hashing->finalize(); + compressed_hashing.finalize(); + compressor.finalize(); + plain_hashing.finalize(); - if (marks_hashing) + if constexpr (!only_plain_file) { if (compress_marks) { - marks_compressed_hashing->finalize(); - marks_compressor->finalize(); + marks_compressed_hashing.finalize(); + marks_compressor.finalize(); } - marks_hashing->finalize(); + marks_hashing.finalize(); } plain_file->preFinalize(); - if (marks_file) + if constexpr (!only_plain_file) marks_file->preFinalize(); is_prefinalized = true; } -void MergeTreeDataPartWriterOnDisk::Stream::finalize() +template +void MergeTreeDataPartWriterOnDisk::Stream::finalize() { if (!is_prefinalized) preFinalize(); plain_file->finalize(); - if (marks_file) + if constexpr (!only_plain_file) marks_file->finalize(); } -void MergeTreeDataPartWriterOnDisk::Stream::sync() const +template +void MergeTreeDataPartWriterOnDisk::Stream::sync() const { plain_file->sync(); - if (marks_file) + if constexpr (!only_plain_file) marks_file->sync(); } -MergeTreeDataPartWriterOnDisk::Stream::Stream( +template<> +MergeTreeDataPartWriterOnDisk::Stream::Stream( const String & escaped_column_name_, const MutableDataPartStoragePtr & data_part_storage, const String & data_path_, @@ -96,45 +90,20 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( escaped_column_name(escaped_column_name_), data_file_extension{data_file_extension_}, marks_file_extension{marks_file_extension_}, + plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), + plain_hashing(*plain_file), + compressor(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), + compressed_hashing(compressor), + marks_file(data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings)), + marks_hashing(*marks_file), + marks_compressor(marks_hashing, marks_compression_codec_, marks_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), + marks_compressed_hashing(marks_compressor), compress_marks(MarkType(marks_file_extension).compressed) { - plain_file = data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings); - plain_hashing.emplace(*plain_file); - - if (query_write_settings.max_compression_threads > 1) - { - compression_thread_pool.emplace( - CurrentMetrics::CompressionThread, CurrentMetrics::CompressionThreadActive, CurrentMetrics::CompressionThreadScheduled, - query_write_settings.max_compression_threads); - - compressor = std::make_unique( - *plain_hashing, - compression_codec_, - max_compress_block_size_, - query_write_settings.max_compression_threads, - *compression_thread_pool); - - is_compressor_parallel = true; - } - else - { - compressor = std::make_unique( - *plain_hashing, - compression_codec_, - max_compress_block_size_, - query_write_settings.use_adaptive_write_buffer, - query_write_settings.adaptive_write_buffer_initial_size); - } - - compressed_hashing.emplace(*compressor); - - marks_file = data_part_storage->writeFile(marks_path_ + marks_file_extension, 4096, query_write_settings); - marks_hashing.emplace(*marks_file); - marks_compressor.emplace(*marks_hashing, marks_compression_codec_, marks_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size); - marks_compressed_hashing.emplace(*marks_compressor); } -MergeTreeDataPartWriterOnDisk::Stream::Stream( +template<> +MergeTreeDataPartWriterOnDisk::Stream::Stream( const String & escaped_column_name_, const MutableDataPartStoragePtr & data_part_storage, const String & data_path_, @@ -146,33 +115,34 @@ MergeTreeDataPartWriterOnDisk::Stream::Stream( data_file_extension{data_file_extension_}, plain_file(data_part_storage->writeFile(data_path_ + data_file_extension, max_compress_block_size_, query_write_settings)), plain_hashing(*plain_file), - compressor(std::make_unique(*plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size)), - compressed_hashing(*compressor), + compressor(plain_hashing, compression_codec_, max_compress_block_size_, query_write_settings.use_adaptive_write_buffer, query_write_settings.adaptive_write_buffer_initial_size), + compressed_hashing(compressor), compress_marks(false) { } -void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) +template +void MergeTreeDataPartWriterOnDisk::Stream::addToChecksums(MergeTreeData::DataPart::Checksums & checksums) { String name = escaped_column_name; checksums.files[name + data_file_extension].is_compressed = true; - checksums.files[name + data_file_extension].uncompressed_size = compressed_hashing->count(); - checksums.files[name + data_file_extension].uncompressed_hash = compressed_hashing->getHash(); - checksums.files[name + data_file_extension].file_size = plain_hashing->count(); - checksums.files[name + data_file_extension].file_hash = plain_hashing->getHash(); + checksums.files[name + data_file_extension].uncompressed_size = compressed_hashing.count(); + checksums.files[name + data_file_extension].uncompressed_hash = compressed_hashing.getHash(); + checksums.files[name + data_file_extension].file_size = plain_hashing.count(); + checksums.files[name + data_file_extension].file_hash = plain_hashing.getHash(); - if (marks_hashing) + if constexpr (!only_plain_file) { if (compress_marks) { checksums.files[name + marks_file_extension].is_compressed = true; - checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing->count(); - checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing->getHash(); + checksums.files[name + marks_file_extension].uncompressed_size = marks_compressed_hashing.count(); + checksums.files[name + marks_file_extension].uncompressed_hash = marks_compressed_hashing.getHash(); } - checksums.files[name + marks_file_extension].file_size = marks_hashing->count(); - checksums.files[name + marks_file_extension].file_hash = marks_hashing->getHash(); + checksums.files[name + marks_file_extension].file_size = marks_hashing.count(); + checksums.files[name + marks_file_extension].file_hash = marks_hashing.getHash(); } } @@ -209,8 +179,8 @@ MergeTreeDataPartWriterOnDisk::MergeTreeDataPartWriterOnDisk( throw Exception(ErrorCodes::LOGICAL_ERROR, "Can't take information about index granularity from blocks, when non empty index_granularity array specified"); - if (!getDataPartStorage().exists()) - getDataPartStorage().createDirectories(); + /// We don't need to check if it exists or not, createDirectories doesn't throw + getDataPartStorage().createDirectories(); if (settings.rewrite_primary_key) initPrimaryIndex(); @@ -306,12 +276,12 @@ void MergeTreeDataPartWriterOnDisk::initStatistics() for (const auto & stat_ptr : stats) { String stats_name = stat_ptr->getFileName(); - stats_streams.emplace_back(std::make_unique( - stats_name, - data_part_storage, - stats_name, STATS_FILE_SUFFIX, - default_codec, settings.max_compress_block_size, - settings.query_write_settings)); + stats_streams.emplace_back(std::make_unique>( + stats_name, + data_part_storage, + stats_name, STATS_FILE_SUFFIX, + default_codec, settings.max_compress_block_size, + settings.query_write_settings)); } } @@ -328,14 +298,14 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices() { String stream_name = skip_index->getFileName(); skip_indices_streams.emplace_back( - std::make_unique( - stream_name, - data_part_storage, - stream_name, skip_index->getSerializedFileExtension(), - stream_name, marks_file_extension, - default_codec, settings.max_compress_block_size, - marks_compression_codec, settings.marks_compress_block_size, - settings.query_write_settings)); + std::make_unique>( + stream_name, + data_part_storage, + stream_name, skip_index->getSerializedFileExtension(), + stream_name, marks_file_extension, + default_codec, settings.max_compress_block_size, + marks_compression_codec, settings.marks_compress_block_size, + settings.query_write_settings)); GinIndexStorePtr store = nullptr; if (typeid_cast(&*skip_index) != nullptr) @@ -411,7 +381,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { const auto index_helper = skip_indices[i]; auto & stream = *skip_indices_streams[i]; - WriteBuffer & marks_out = stream.compress_marks ? *stream.marks_compressed_hashing : *stream.marks_hashing; + WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; GinIndexStorePtr store; if (typeid_cast(&*index_helper) != nullptr) @@ -427,7 +397,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { if (skip_index_accumulated_marks[i] == index_helper->index.granularity) { - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(*stream.compressed_hashing); + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing); skip_index_accumulated_marks[i] = 0; } @@ -435,11 +405,11 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block { skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store, settings); - if (stream.compressed_hashing->offset() >= settings.min_compress_block_size) - stream.compressed_hashing->next(); + if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) + stream.compressed_hashing.next(); - writeBinaryLittleEndian(stream.plain_hashing->count(), marks_out); - writeBinaryLittleEndian(stream.compressed_hashing->offset(), marks_out); + writeBinaryLittleEndian(stream.plain_hashing.count(), marks_out); + writeBinaryLittleEndian(stream.compressed_hashing.offset(), marks_out); /// Actually this numbers is redundant, but we have to store them /// to be compatible with the normal .mrk2 file format @@ -519,7 +489,7 @@ void MergeTreeDataPartWriterOnDisk::fillSkipIndicesChecksums(MergeTreeData::Data { auto & stream = *skip_indices_streams[i]; if (!skip_indices_aggregators[i]->empty()) - skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(*stream.compressed_hashing); + skip_indices_aggregators[i]->getGranuleAndReset()->serializeBinary(stream.compressed_hashing); /// Register additional files written only by the full-text index. Required because otherwise DROP TABLE complains about unknown /// files. Note that the provided actual checksums are bogus. The problem is that at this point the file writes happened already and @@ -559,7 +529,7 @@ void MergeTreeDataPartWriterOnDisk::fillStatisticsChecksums(MergeTreeData::DataP for (size_t i = 0; i < stats.size(); i++) { auto & stream = *stats_streams[i]; - stats[i]->serialize(*stream.compressed_hashing); + stats[i]->serialize(stream.compressed_hashing); stream.preFinalize(); stream.addToChecksums(checksums); } @@ -594,4 +564,46 @@ Names MergeTreeDataPartWriterOnDisk::getSkipIndicesColumns() const return Names(skip_indexes_column_names_set.begin(), skip_indexes_column_names_set.end()); } +void MergeTreeDataPartWriterOnDisk::initOrAdjustDynamicStructureIfNeeded(Block & block) +{ + if (!is_dynamic_streams_initialized) + { + for (const auto & column : columns_list) + { + if (column.type->hasDynamicSubcolumns()) + { + /// Create all streams for dynamic subcolumns using dynamic structure from block. + auto compression = getCodecDescOrDefault(column.name, default_codec); + addStreams(column, block.getByName(column.name).column, compression); + } + } + is_dynamic_streams_initialized = true; + block_sample = block.cloneEmpty(); + } + else + { + size_t size = block.columns(); + for (size_t i = 0; i != size; ++i) + { + auto & column = block.getByPosition(i); + const auto & sample_column = block_sample.getByPosition(i); + /// Check if the dynamic structure of this column is different from the sample column. + if (column.type->hasDynamicSubcolumns() && !column.column->dynamicStructureEquals(*sample_column.column)) + { + /// We need to change the dynamic structure of the column so it matches the sample column. + /// To do it, we create empty column of this type, take dynamic structure from sample column + /// and insert data into it. Resulting column will have required dynamic structure and the content + /// of the column in current block. + auto new_column = sample_column.type->createColumn(); + new_column->takeDynamicStructureFromSourceColumns({sample_column.column}); + new_column->insertRangeFrom(*column.column, 0, column.column->size()); + column.column = std::move(new_column); + } + } + } +} + +template struct MergeTreeDataPartWriterOnDisk::Stream; +template struct MergeTreeDataPartWriterOnDisk::Stream; + } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h index cb46785ccbd..49d654c15e1 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterOnDisk.h @@ -8,6 +8,7 @@ #include #include #include +#include namespace DB { @@ -27,7 +28,7 @@ struct Granule /// this granule can be continuation of the previous one. bool mark_on_start; /// if true: When this granule will be written to disk all rows for corresponding mark will - /// be written. It doesn't mean that rows_to_write == index_granularity.getMarkRows(mark_number), + /// be wrtten. It doesn't mean that rows_to_write == index_granularity.getMarkRows(mark_number), /// We may have a lot of small blocks between two marks and this may be the last one. bool is_complete; }; @@ -44,6 +45,7 @@ public: /// Helper class, which holds chain of buffers to write data file with marks. /// It is used to write: one column, skip index or all columns (in compact format). + template struct Stream { Stream( @@ -74,32 +76,30 @@ public: /// compressed_hashing -> compressor -> plain_hashing -> plain_file std::unique_ptr plain_file; - std::optional plain_hashing; - /// This could be either CompressedWriteBuffer or ParallelCompressedWriteBuffer - bool is_compressor_parallel = false; - std::unique_ptr compressor; - std::optional compressed_hashing; + HashingWriteBuffer plain_hashing; + CompressedWriteBuffer compressor; + HashingWriteBuffer compressed_hashing; /// marks_compressed_hashing -> marks_compressor -> marks_hashing -> marks_file std::unique_ptr marks_file; - std::optional marks_hashing; - std::optional marks_compressor; - std::optional marks_compressed_hashing; + std::conditional_t marks_hashing; + std::conditional_t marks_compressor; + std::conditional_t marks_compressed_hashing; bool compress_marks; bool is_prefinalized = false; - /// Thread pool for parallel compression. - std::optional compression_thread_pool; - void preFinalize(); + void finalize(); + void sync() const; void addToChecksums(MergeTreeDataPartChecksums & checksums); }; - using StreamPtr = std::unique_ptr; + using StreamPtr = std::unique_ptr>; + using StatisticStreamPtr = std::unique_ptr>; MergeTreeDataPartWriterOnDisk( const String & data_part_name_, @@ -154,10 +154,18 @@ protected: /// Get unique non ordered skip indices column. Names getSkipIndicesColumns() const; + virtual void addStreams(const NameAndTypePair & name_and_type, const ColumnPtr & column, const ASTPtr & effective_codec_desc) = 0; + + /// On first block create all required streams for columns with dynamic subcolumns and remember the block sample. + /// On each next block check if dynamic structure of the columns equals to the dynamic structure of the same + /// columns in the sample block. If for some column dynamic structure is different, adjust it so it matches + /// the structure from the sample. + void initOrAdjustDynamicStructureIfNeeded(Block & block); + const MergeTreeIndices skip_indices; const ColumnsStatistics stats; - std::vector stats_streams; + std::vector stats_streams; const String marks_file_extension; const CompressionCodecPtr default_codec; @@ -188,6 +196,10 @@ protected: size_t current_mark = 0; GinIndexStoreFactory::GinIndexStores gin_index_stores; + + bool is_dynamic_streams_initialized = false; + Block block_sample; + private: void initSkipIndices(); void initPrimaryIndex(); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 860722ba870..7c9724b1b75 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -6,10 +6,11 @@ #include #include #include +#include +#include #include #include - namespace DB { @@ -106,27 +107,16 @@ MergeTreeDataPartWriterWide::MergeTreeDataPartWriterWide( indices_to_recalc_, stats_to_recalc_, marks_file_extension_, default_codec_, settings_, index_granularity_) { + if (settings.save_marks_in_cache) + { + auto columns_vec = getColumnsToPrewarmMarks(*storage_settings, columns_list); + columns_to_load_marks = NameSet(columns_vec.begin(), columns_vec.end()); + } + for (const auto & column : columns_list) { auto compression = getCodecDescOrDefault(column.name, default_codec); - addStreams(column, nullptr, compression); - } -} - -void MergeTreeDataPartWriterWide::initDynamicStreamsIfNeeded(const DB::Block & block) -{ - if (is_dynamic_streams_initialized) - return; - - is_dynamic_streams_initialized = true; - block_sample = block.cloneEmpty(); - for (const auto & column : columns_list) - { - if (column.type->hasDynamicSubcolumns()) - { - auto compression = getCodecDescOrDefault(column.name, default_codec); - addStreams(column, block_sample.getByName(column.name).column, compression); - } + MergeTreeDataPartWriterWide::addStreams(column, nullptr, compression); } } @@ -188,7 +178,7 @@ void MergeTreeDataPartWriterWide::addStreams( query_write_settings.use_adaptive_write_buffer = settings.use_adaptive_write_buffer_for_dynamic_subcolumns && ISerialization::isDynamicSubcolumn(substream_path, substream_path.size()); query_write_settings.adaptive_write_buffer_initial_size = settings.adaptive_write_buffer_initial_size; - column_streams[stream_name] = std::make_unique( + column_streams[stream_name] = std::make_unique>( stream_name, data_part_storage, stream_name, DATA_FILE_EXTENSION, @@ -199,6 +189,9 @@ void MergeTreeDataPartWriterWide::addStreams( settings.marks_compress_block_size, query_write_settings); + if (columns_to_load_marks.contains(name_and_type.name)) + cached_marks.emplace(stream_name, std::make_unique()); + full_name_to_stream_name.emplace(full_stream_name, stream_name); stream_name_to_full_name.emplace(stream_name, full_stream_name); }; @@ -231,7 +224,7 @@ ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGett if (is_offsets && offset_columns.contains(stream_name)) return nullptr; - return &column_streams.at(stream_name)->compressed_hashing.value(); + return &column_streams.at(stream_name)->compressed_hashing; }; } @@ -267,15 +260,20 @@ void MergeTreeDataPartWriterWide::shiftCurrentMark(const Granules & granules_wri void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Permutation * permutation) { - /// On first block of data initialize streams for dynamic subcolumns. - initDynamicStreamsIfNeeded(block); + Block block_to_write = block; + + /// During serialization columns with dynamic subcolumns (like JSON/Dynamic) must have the same dynamic structure. + /// But it may happen that they don't (for example during ALTER MODIFY COLUMN from some type to JSON/Dynamic). + /// In this case we use dynamic structure of the column from the first written block and adjust columns from + /// the next blocks so they match this dynamic structure. + initOrAdjustDynamicStructureIfNeeded(block_to_write); /// Fill index granularity for this block /// if it's unknown (in case of insert data or horizontal merge, /// but not in case of vertical part of vertical merge) if (compute_granularity) { - size_t index_granularity_for_block = computeIndexGranularity(block); + size_t index_granularity_for_block = computeIndexGranularity(block_to_write); if (rows_written_in_last_mark > 0) { size_t rows_left_in_last_mark = index_granularity.getMarkRows(getCurrentMark()) - rows_written_in_last_mark; @@ -293,11 +291,9 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm } } - fillIndexGranularity(index_granularity_for_block, block.rows()); + fillIndexGranularity(index_granularity_for_block, block_to_write.rows()); } - Block block_to_write = block; - auto granules_to_write = getGranulesToWrite(index_granularity, block_to_write.rows(), getCurrentMark(), rows_written_in_last_mark); auto offset_columns = written_offset_columns ? *written_offset_columns : WrittenOffsetColumns{}; @@ -363,12 +359,16 @@ void MergeTreeDataPartWriterWide::writeSingleMark( void MergeTreeDataPartWriterWide::flushMarkToFile(const StreamNameAndMark & stream_with_mark, size_t rows_in_mark) { auto & stream = *column_streams[stream_with_mark.stream_name]; - WriteBuffer & marks_out = stream.compress_marks ? *stream.marks_compressed_hashing : *stream.marks_hashing; + WriteBuffer & marks_out = stream.compress_marks ? stream.marks_compressed_hashing : stream.marks_hashing; writeBinaryLittleEndian(stream_with_mark.mark.offset_in_compressed_file, marks_out); writeBinaryLittleEndian(stream_with_mark.mark.offset_in_decompressed_block, marks_out); + if (settings.can_use_adaptive_granularity) writeBinaryLittleEndian(rows_in_mark, marks_out); + + if (auto it = cached_marks.find(stream_with_mark.stream_name); it != cached_marks.end()) + it->second->push_back(stream_with_mark.mark); } StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( @@ -400,22 +400,15 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( auto & stream = *column_streams[stream_name]; /// There could already be enough data to compress into the new block. - auto push_mark = [&] - { - StreamNameAndMark stream_with_mark; - stream_with_mark.stream_name = stream_name; - stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing->count(); - stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing->offset(); - result.push_back(stream_with_mark); - }; + if (stream.compressed_hashing.offset() >= min_compress_block_size) + stream.compressed_hashing.next(); - if (stream.compressed_hashing->offset() >= min_compress_block_size) - { + StreamNameAndMark stream_with_mark; + stream_with_mark.stream_name = stream_name; + stream_with_mark.mark.offset_in_compressed_file = stream.plain_hashing.count(); + stream_with_mark.mark.offset_in_decompressed_block = stream.compressed_hashing.offset(); - stream.compressed_hashing->next(); - } - - push_mark(); + result.push_back(stream_with_mark); }, name_and_type.type, column_sample); return result; @@ -446,7 +439,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( if (is_offsets && offset_columns.contains(stream_name)) return; - column_streams.at(stream_name)->compressed_hashing->nextIfAtEnd(); + column_streams.at(stream_name)->compressed_hashing.nextIfAtEnd(); }, name_and_type.type, column.getPtr()); } @@ -750,7 +743,6 @@ void MergeTreeDataPartWriterWide::fillChecksums(MergeTreeDataPartChecksums & che fillPrimaryIndexChecksums(checksums); fillSkipIndicesChecksums(checksums); - fillStatisticsChecksums(checksums); } @@ -764,7 +756,6 @@ void MergeTreeDataPartWriterWide::finish(bool sync) finishPrimaryIndexSerialization(sync); finishSkipIndicesSerialization(sync); - finishStatisticsSerialization(sync); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h index ab86ed27c7e..19304b28c6c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.h @@ -43,6 +43,8 @@ public: void finish(bool sync) final; + size_t getNumberOfOpenStreams() const override { return column_streams.size(); } + private: /// Finish serialization of data: write final mark if required and compute checksums /// Also validate written data in debug mode @@ -91,9 +93,7 @@ private: void addStreams( const NameAndTypePair & name_and_type, const ColumnPtr & column, - const ASTPtr & effective_codec_desc); - - void initDynamicStreamsIfNeeded(const Block & block); + const ASTPtr & effective_codec_desc) override; /// Method for self check (used in debug-build only). Checks that written /// data and corresponding marks are consistent. Otherwise throws logical @@ -136,13 +136,12 @@ private: using MarksForColumns = std::unordered_map; MarksForColumns last_non_written_marks; + /// Set of columns to put marks in cache during write. + NameSet columns_to_load_marks; + /// How many rows we have already written in the current mark. /// More than zero when incoming blocks are smaller then their granularity. size_t rows_written_in_last_mark = 0; - - Block block_sample; - - bool is_dynamic_streams_initialized = false; }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 13918ae8e91..1b3c58000e7 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -71,10 +71,7 @@ namespace Setting extern const SettingsString force_data_skipping_indices; extern const SettingsBool force_index_by_date; extern const SettingsSeconds lock_acquire_timeout; - extern const SettingsUInt64 max_parser_backtracks; - extern const SettingsUInt64 max_parser_depth; extern const SettingsInt64 max_partitions_to_read; - extern const SettingsUInt64 max_query_size; extern const SettingsUInt64 max_threads_for_indexes; extern const SettingsNonZeroUInt64 max_parallel_replicas; extern const SettingsUInt64 merge_tree_coarse_index_granularity; @@ -640,20 +637,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (use_skip_indexes && settings[Setting::force_data_skipping_indices].changed) { - const auto & indices = settings[Setting::force_data_skipping_indices].toString(); - - Strings forced_indices; - { - Tokens tokens(indices.data(), indices.data() + indices.size(), settings[Setting::max_query_size]); - IParser::Pos pos( - tokens, static_cast(settings[Setting::max_parser_depth]), static_cast(settings[Setting::max_parser_backtracks])); - Expected expected; - if (!parseIdentifiersOrStringLiterals(pos, expected, forced_indices)) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse force_data_skipping_indices ('{}')", indices); - } + const auto & indices_str = settings[Setting::force_data_skipping_indices].toString(); + auto forced_indices = parseIdentifiersOrStringLiterals(indices_str, settings); if (forced_indices.empty()) - throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices); + throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "No indices parsed from force_data_skipping_indices ('{}')", indices_str); std::unordered_set useful_indices_names; for (const auto & useful_index : skip_indexes.useful_indices) @@ -1022,11 +1010,7 @@ size_t MergeTreeDataSelectExecutor::roundRowsOrBytesToMarks( /// Same as roundRowsOrBytesToMarks() but do not return more then max_marks size_t MergeTreeDataSelectExecutor::minMarksForConcurrentRead( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity, - size_t max_marks) + size_t rows_setting, size_t bytes_setting, size_t rows_granularity, size_t bytes_granularity, size_t min_marks, size_t max_marks) { size_t marks = 1; @@ -1035,18 +1019,17 @@ size_t MergeTreeDataSelectExecutor::minMarksForConcurrentRead( else if (rows_setting) marks = (rows_setting + rows_granularity - 1) / rows_granularity; - if (bytes_granularity == 0) - return marks; - - /// Overflow - if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow - return max_marks; - if (bytes_setting) - return std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); - return marks; + if (bytes_granularity) + { + /// Overflow + if (bytes_setting + bytes_granularity <= bytes_setting) /// overflow + marks = max_marks; + else if (bytes_setting) + marks = std::max(marks, (bytes_setting + bytes_granularity - 1) / bytes_granularity); + } + return std::max(marks, min_marks); } - /// Calculates a set of mark ranges, that could possibly contain keys, required by condition. /// In other words, it removes subranges from whole range, that definitely could not contain required keys. /// If @exact_ranges is not null, fill it with ranges containing marks of fully matched records. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 70536b7aa54..d16d9243c14 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -153,11 +153,7 @@ public: /// The same as roundRowsOrBytesToMarks, but return no more than max_marks. static size_t minMarksForConcurrentRead( - size_t rows_setting, - size_t bytes_setting, - size_t rows_granularity, - size_t bytes_granularity, - size_t max_marks); + size_t rows_setting, size_t bytes_setting, size_t rows_granularity, size_t bytes_granularity, size_t min_marks, size_t max_marks); /// If possible, construct optional key condition from predicates containing _part_offset column. static void buildKeyConditionFromPartOffset( diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 67fef759ed4..6d19f45e2c4 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -73,6 +73,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsFloat min_free_disk_ratio_to_perform_insert; extern const MergeTreeSettingsBool optimize_row_order; extern const MergeTreeSettingsFloat ratio_of_defaults_for_sparse_serialization; + extern const MergeTreeSettingsBool prewarm_mark_cache; } namespace ErrorCodes @@ -609,7 +610,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( } } - auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir) + auto new_data_part = data.getDataPartBuilder(part_name, data_part_volume, part_dir, getReadSettings()) .withPartFormat(data.choosePartFormat(expected_size, block.rows())) .withPartInfo(new_part_info) .build(); @@ -684,6 +685,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); + bool save_marks_in_cache = (*data_settings)[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); auto out = std::make_unique( new_data_part, @@ -693,8 +695,9 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPartImpl( statistics, compression_codec, context->getCurrentTransaction() ? context->getCurrentTransaction()->tid : Tx::PrehistoricTID, - false, - false, + /*reset_columns=*/ false, + save_marks_in_cache, + /*blocks_are_granules_size=*/ false, context->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); @@ -829,6 +832,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( /// This effectively chooses minimal compression method: /// either default lz4 or compression method with zero thresholds on absolute and relative part size. auto compression_codec = data.getContext()->chooseCompressionCodec(0, 0); + bool save_marks_in_cache = (*data.getSettings())[MergeTreeSetting::prewarm_mark_cache] && data.getContext()->getMarkCache(); auto out = std::make_unique( new_data_part, @@ -839,7 +843,10 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( ColumnsStatistics{}, compression_codec, Tx::PrehistoricTID, - false, false, data.getContext()->getWriteSettings()); + /*reset_columns=*/ false, + save_marks_in_cache, + /*blocks_are_granules_size=*/ false, + data.getContext()->getWriteSettings()); out->writeWithPermutation(block, perm_ptr); auto finalizer = out->finalizePartAsync(new_data_part, false); diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.cpp b/src/Storages/MergeTree/MergeTreeIOSettings.cpp index 6705d75af41..bacfbbd5720 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeIOSettings.cpp @@ -26,7 +26,6 @@ namespace MergeTreeSetting extern const MergeTreeSettingsString primary_key_compression_codec; extern const MergeTreeSettingsBool use_adaptive_write_buffer_for_dynamic_subcolumns; extern const MergeTreeSettingsBool use_compact_variant_discriminators_serialization; - extern const MergeTreeSettingsUInt64 max_compression_threads; } MergeTreeWriterSettings::MergeTreeWriterSettings( @@ -35,6 +34,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( const MergeTreeSettingsPtr & storage_settings, bool can_use_adaptive_granularity_, bool rewrite_primary_key_, + bool save_marks_in_cache_, bool blocks_are_granules_size_) : min_compress_block_size( (*storage_settings)[MergeTreeSetting::min_compress_block_size] ? (*storage_settings)[MergeTreeSetting::min_compress_block_size] : global_settings[Setting::min_compress_block_size]) @@ -47,6 +47,7 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( , primary_key_compress_block_size((*storage_settings)[MergeTreeSetting::primary_key_compress_block_size]) , can_use_adaptive_granularity(can_use_adaptive_granularity_) , rewrite_primary_key(rewrite_primary_key_) + , save_marks_in_cache(save_marks_in_cache_) , blocks_are_granules_size(blocks_are_granules_size_) , query_write_settings(query_write_settings_) , low_cardinality_max_dictionary_size(global_settings[Setting::low_cardinality_max_dictionary_size]) @@ -55,7 +56,6 @@ MergeTreeWriterSettings::MergeTreeWriterSettings( , use_adaptive_write_buffer_for_dynamic_subcolumns((*storage_settings)[MergeTreeSetting::use_adaptive_write_buffer_for_dynamic_subcolumns]) , adaptive_write_buffer_initial_size((*storage_settings)[MergeTreeSetting::adaptive_write_buffer_initial_size]) { - query_write_settings.max_compression_threads = (*storage_settings)[MergeTreeSetting::max_compression_threads]; } } diff --git a/src/Storages/MergeTree/MergeTreeIOSettings.h b/src/Storages/MergeTree/MergeTreeIOSettings.h index fcc72815d8f..4d1d2533729 100644 --- a/src/Storages/MergeTree/MergeTreeIOSettings.h +++ b/src/Storages/MergeTree/MergeTreeIOSettings.h @@ -2,6 +2,7 @@ #include #include #include +#include #include @@ -60,7 +61,8 @@ struct MergeTreeWriterSettings const MergeTreeSettingsPtr & storage_settings, bool can_use_adaptive_granularity_, bool rewrite_primary_key_, - bool blocks_are_granules_size_ = false); + bool save_marks_in_cache_, + bool blocks_are_granules_size_); size_t min_compress_block_size; size_t max_compress_block_size; @@ -74,6 +76,7 @@ struct MergeTreeWriterSettings bool can_use_adaptive_granularity; bool rewrite_primary_key; + bool save_marks_in_cache; bool blocks_are_granules_size; WriteSettings query_write_settings; diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 467d2567df1..d69a00643f0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -96,29 +96,13 @@ size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t num return to_mark - from_mark; } -size_t MergeTreeIndexGranularity::countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows, size_t min_marks_to_read) const +size_t MergeTreeIndexGranularity::countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const { size_t rows_before_mark = getMarkStartingRow(from_mark); size_t last_row_pos = rows_before_mark + offset_in_rows + number_of_rows; auto it = std::upper_bound(marks_rows_partial_sums.begin(), marks_rows_partial_sums.end(), last_row_pos); size_t to_mark = it - marks_rows_partial_sums.begin(); - /// This is a heuristic to respect min_marks_to_read which is ignored by MergeTreeReadPool in case of remote disk. - /// See comment in IMergeTreeSelectAlgorithm. - if (min_marks_to_read) - { - // check overflow - size_t min_marks_to_read_2 = 0; - bool overflow = common::mulOverflow(min_marks_to_read, 2, min_marks_to_read_2); - - size_t to_mark_overwrite = 0; - if (!overflow) - overflow = common::addOverflow(from_mark, min_marks_to_read_2, to_mark_overwrite); - - if (!overflow && to_mark_overwrite < to_mark) - to_mark = to_mark_overwrite; - } - return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows; } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/src/Storages/MergeTree/MergeTreeIndexGranularity.h index 78a1423ad7e..f66e721ec1e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -37,7 +37,7 @@ public: /// |-----|---------------------------|----|----| /// ^------------------------^-----------^ //// from_mark offset_in_rows number_of_rows - size_t countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows, size_t min_marks_to_read) const; + size_t countRowsForRows(size_t from_mark, size_t number_of_rows, size_t offset_in_rows) const; /// Total marks size_t getMarksCount() const; diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp index 2af7abc17f9..9211ab51ad5 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.cpp @@ -108,6 +108,14 @@ std::optional MergeTreeIndexGranularityInfo::getMarksTypeFromFilesyste return {}; } +MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo( + MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_) + : mark_type(mark_type_) + , fixed_index_granularity(index_granularity_) + , index_granularity_bytes(index_granularity_bytes_) +{ +} + MergeTreeIndexGranularityInfo::MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MergeTreeDataPartType type_) : MergeTreeIndexGranularityInfo(storage, {storage.canUseAdaptiveGranularity(), (*storage.getSettings())[MergeTreeSetting::compress_marks], type_.getValue()}) { diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h index 87445c99ade..b302d6b1a4b 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularityInfo.h @@ -49,6 +49,7 @@ public: MergeTreeIndexGranularityInfo(const MergeTreeData & storage, MarkType mark_type_); MergeTreeIndexGranularityInfo(MergeTreeDataPartType type_, bool is_adaptive_, size_t index_granularity_, size_t index_granularity_bytes_); + MergeTreeIndexGranularityInfo(MarkType mark_type_, size_t index_granularity_, size_t index_granularity_bytes_); void changeGranularityIfRequired(const IDataPartStorage & data_part_storage); diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp index c269a0a23ae..f95b840e223 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -41,10 +42,16 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int INCORRECT_NUMBER_OF_COLUMNS; extern const int INCORRECT_QUERY; + extern const int INVALID_SETTING_VALUE; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; } +namespace Setting +{ + extern const SettingsUInt64 hnsw_candidate_list_size_for_search; +} + namespace { @@ -104,7 +111,7 @@ USearchIndexWithSerialization::USearchIndexWithSerialization( { USearchIndex::metric_t metric(dimensions, metric_kind, scalar_kind); - unum::usearch::index_dense_config_t config(usearch_hnsw_params.m, usearch_hnsw_params.ef_construction, usearch_hnsw_params.ef_search); + unum::usearch::index_dense_config_t config(usearch_hnsw_params.connectivity, usearch_hnsw_params.expansion_add, default_expansion_search); config.enable_key_lookups = false; /// we don't do row-to-vector lookups auto result = USearchIndex::make(metric, config); @@ -338,10 +345,11 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ throw Exception(ErrorCodes::INCORRECT_DATA, "Index granularity is too big: more than {} rows per index granule.", std::numeric_limits::max()); if (index_sample_block.columns() > 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected block with single column"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected that index is build over a single column"); - const String & index_column_name = index_sample_block.getByPosition(0).name; - const ColumnPtr & index_column = block.getByName(index_column_name).column; + const auto & index_column_name = index_sample_block.getByPosition(0).name; + + const auto & index_column = block.getByName(index_column_name).column; ColumnPtr column_cut = index_column->cut(*pos, rows_read); const auto * column_array = typeid_cast(column_cut.get()); @@ -375,8 +383,7 @@ void MergeTreeIndexAggregatorVectorSimilarity::update(const Block & block, size_ if (index->size() + rows > std::numeric_limits::max()) throw Exception(ErrorCodes::INCORRECT_DATA, "Size of vector similarity index would exceed 4 billion entries"); - DataTypePtr data_type = block.getDataTypes()[0]; - const auto * data_type_array = typeid_cast(data_type.get()); + const auto * data_type_array = typeid_cast(block.getByName(index_column_name).type.get()); if (!data_type_array) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected data type Array(Float*)"); const TypeIndex nested_type_index = data_type_array->getNestedType()->getTypeId(); @@ -399,7 +406,11 @@ MergeTreeIndexConditionVectorSimilarity::MergeTreeIndexConditionVectorSimilarity ContextPtr context) : vector_similarity_condition(query, context) , metric_kind(metric_kind_) + , expansion_search(context->getSettingsRef()[Setting::hnsw_candidate_list_size_for_search]) { + if (expansion_search == 0) + throw Exception(ErrorCodes::INVALID_SETTING_VALUE, "Setting 'hnsw_candidate_list_size_for_search' must not be 0"); + } bool MergeTreeIndexConditionVectorSimilarity::mayBeTrueOnGranule(MergeTreeIndexGranulePtr) const @@ -430,13 +441,17 @@ std::vector MergeTreeIndexConditionVectorSimilarity::calculateApproximat const USearchIndexWithSerializationPtr index = granule->index; if (vector_similarity_condition.getDimensions() != index->dimensions()) - throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) " - "does not match the dimension in the index ({})", + throw Exception(ErrorCodes::INCORRECT_QUERY, "The dimension of the space in the request ({}) does not match the dimension in the index ({})", vector_similarity_condition.getDimensions(), index->dimensions()); const std::vector reference_vector = vector_similarity_condition.getReferenceVector(); - auto search_result = index->search(reference_vector.data(), limit); + /// We want to run the search with the user-provided value for setting hnsw_candidate_list_size_for_search (aka. expansion_search). + /// The way to do this in USearch is to call index_dense_gt::change_expansion_search. Unfortunately, this introduces a need to + /// synchronize index access, see https://github.com/unum-cloud/usearch/issues/500. As a workaround, we extended USearch' search method + /// to accept a custom expansion_add setting. The config value is only used on the fly, i.e. not persisted in the index. + + auto search_result = index->search(reference_vector.data(), limit, USearchIndex::any_thread(), false, expansion_search); if (!search_result) throw Exception(ErrorCodes::INCORRECT_DATA, "Could not search in vector similarity index. Error: {}", String(search_result.error.release())); @@ -501,13 +516,12 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) UsearchHnswParams usearch_hnsw_params; /// Optional parameters: - const bool has_six_args = (index.arguments.size() == 6); - if (has_six_args) + const bool has_five_args = (index.arguments.size() == 5); + if (has_five_args) { scalar_kind = quantizationToScalarKind.at(index.arguments[2].safeGet()); - usearch_hnsw_params = {.m = index.arguments[3].safeGet(), - .ef_construction = index.arguments[4].safeGet(), - .ef_search = index.arguments[5].safeGet()}; + usearch_hnsw_params = {.connectivity = index.arguments[3].safeGet(), + .expansion_add = index.arguments[4].safeGet()}; } return std::make_shared(index, metric_kind, scalar_kind, usearch_hnsw_params); @@ -516,25 +530,23 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index) void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* attach */) { const bool has_two_args = (index.arguments.size() == 2); - const bool has_six_args = (index.arguments.size() == 6); + const bool has_five_args = (index.arguments.size() == 5); /// Check number and type of arguments - if (!has_two_args && !has_six_args) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or six arguments"); + if (!has_two_args && !has_five_args) + throw Exception(ErrorCodes::INCORRECT_QUERY, "Vector similarity index must have two or five arguments"); if (index.arguments[0].getType() != Field::Types::String) throw Exception(ErrorCodes::INCORRECT_QUERY, "First argument of vector similarity index (method) must be of type String"); if (index.arguments[1].getType() != Field::Types::String) throw Exception(ErrorCodes::INCORRECT_QUERY, "Second argument of vector similarity index (metric) must be of type String"); - if (has_six_args) + if (has_five_args) { if (index.arguments[2].getType() != Field::Types::String) throw Exception(ErrorCodes::INCORRECT_QUERY, "Third argument of vector similarity index (quantization) must be of type String"); if (index.arguments[3].getType() != Field::Types::UInt64) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Fourth argument of vector similarity index (M) must be of type UInt64"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Fourth argument of vector similarity index (hnsw_max_connections_per_layer) must be of type UInt64"); if (index.arguments[4].getType() != Field::Types::UInt64) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Fifth argument of vector similarity index (ef_construction) must be of type UInt64"); - if (index.arguments[5].getType() != Field::Types::UInt64) - throw Exception(ErrorCodes::INCORRECT_QUERY, "Sixth argument of vector similarity index (ef_search) must be of type UInt64"); + throw Exception(ErrorCodes::INCORRECT_QUERY, "Fifth argument of vector similarity index (hnsw_candidate_list_size_for_construction) must be of type UInt64"); } /// Check that passed arguments are supported @@ -542,18 +554,17 @@ void vectorSimilarityIndexValidator(const IndexDescription & index, bool /* atta throw Exception(ErrorCodes::INCORRECT_DATA, "First argument (method) of vector similarity index is not supported. Supported methods are: {}", joinByComma(methods)); if (!distanceFunctionToMetricKind.contains(index.arguments[1].safeGet())) throw Exception(ErrorCodes::INCORRECT_DATA, "Second argument (distance function) of vector similarity index is not supported. Supported distance function are: {}", joinByComma(distanceFunctionToMetricKind)); - if (has_six_args) + if (has_five_args) { if (!quantizationToScalarKind.contains(index.arguments[2].safeGet())) throw Exception(ErrorCodes::INCORRECT_DATA, "Third argument (quantization) of vector similarity index is not supported. Supported quantizations are: {}", joinByComma(quantizationToScalarKind)); /// Call Usearch's own parameter validation method for HNSW-specific parameters - UInt64 m = index.arguments[3].safeGet(); - UInt64 ef_construction = index.arguments[4].safeGet(); - UInt64 ef_search = index.arguments[5].safeGet(); - - unum::usearch::index_dense_config_t config(m, ef_construction, ef_search); + UInt64 connectivity = index.arguments[3].safeGet(); + UInt64 expansion_add = index.arguments[4].safeGet(); + UInt64 expansion_search = default_expansion_search; + unum::usearch::index_dense_config_t config(connectivity, expansion_add, expansion_search); if (auto error = config.validate(); error) throw Exception(ErrorCodes::INCORRECT_DATA, "Invalid parameters passed to vector similarity index. Error: {}", String(error.release())); } diff --git a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h index b77473e7c2b..9a81e168393 100644 --- a/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h +++ b/src/Storages/MergeTree/MergeTreeIndexVectorSimilarity.h @@ -11,11 +11,18 @@ namespace DB { +/// Defaults for HNSW parameters. Instead of using the default parameters provided by USearch (default_connectivity(), +/// default_expansion_add(), default_expansion_search()), we experimentally came up with our own default parameters. They provide better +/// trade-offs with regards to index construction time, search precision and queries-per-second (speed). +static constexpr size_t default_connectivity = 32; +static constexpr size_t default_expansion_add = 128; +static constexpr size_t default_expansion_search = 256; + +/// Parameters for HNSW index construction. struct UsearchHnswParams { - size_t m = unum::usearch::default_connectivity(); - size_t ef_construction = unum::usearch::default_expansion_add(); - size_t ef_search = unum::usearch::default_expansion_search(); + size_t connectivity = default_connectivity; + size_t expansion_add = default_expansion_add; }; using USearchIndex = unum::usearch::index_dense_t; @@ -142,6 +149,7 @@ public: private: const VectorSimilarityCondition vector_similarity_condition; const unum::usearch::metric_kind_t metric_kind; + const size_t expansion_search; }; diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp index 168134a329f..a271af578cc 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.cpp +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.cpp @@ -3,10 +3,12 @@ #include #include #include +#include #include #include #include #include +#include #include @@ -21,6 +23,11 @@ namespace ProfileEvents namespace DB { +namespace MergeTreeSetting +{ + extern const MergeTreeSettingsString columns_to_prewarm_mark_cache; +} + namespace ErrorCodes { extern const int CANNOT_READ_ALL_DATA; @@ -211,6 +218,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksSync() if (mark_cache) { auto key = MarkCache::hash(fs::path(data_part_storage->getFullPath()) / mrk_path); + if (save_marks_in_cache) { auto callback = [this] { return loadMarksImpl(); }; @@ -249,4 +257,25 @@ std::future MergeTreeMarksLoader::loadMarksAsync() "LoadMarksThread"); } +void addMarksToCache(const IMergeTreeDataPart & part, const PlainMarksByName & cached_marks, MarkCache * mark_cache) +{ + MemoryTrackerBlockerInThread temporarily_disable_memory_tracker; + + for (const auto & [stream_name, marks] : cached_marks) + { + auto mark_path = part.index_granularity_info.getMarksFilePath(stream_name); + auto key = MarkCache::hash(fs::path(part.getDataPartStorage().getFullPath()) / mark_path); + mark_cache->set(key, std::make_shared(*marks)); + } +} + +Names getColumnsToPrewarmMarks(const MergeTreeSettings & settings, const NamesAndTypesList & columns_list) +{ + auto columns_str = settings[MergeTreeSetting::columns_to_prewarm_mark_cache].toString(); + if (columns_str.empty()) + return columns_list.getNames(); + + return parseIdentifiersOrStringLiterals(columns_str, Context::getGlobalContextInstance()->getSettingsRef()); +} + } diff --git a/src/Storages/MergeTree/MergeTreeMarksLoader.h b/src/Storages/MergeTree/MergeTreeMarksLoader.h index 2aa4474e1c5..e031700d6a7 100644 --- a/src/Storages/MergeTree/MergeTreeMarksLoader.h +++ b/src/Storages/MergeTree/MergeTreeMarksLoader.h @@ -1,9 +1,8 @@ #pragma once #include -#include -#include #include +#include namespace DB @@ -11,6 +10,7 @@ namespace DB struct MergeTreeIndexGranularityInfo; using MarksPtr = MarkCache::MappedPtr; +struct ReadSettings; class Threadpool; /// Class that helps to get marks by indexes. @@ -77,4 +77,13 @@ private: using MergeTreeMarksLoaderPtr = std::shared_ptr; +class IMergeTreeDataPart; +struct MergeTreeSettings; + +/// Adds computed marks for part to the marks cache. +void addMarksToCache(const IMergeTreeDataPart & part, const PlainMarksByName & cached_marks, MarkCache * mark_cache); + +/// Returns the list of columns suitable for prewarming of mark cache according to settings. +Names getColumnsToPrewarmMarks(const MergeTreeSettings & settings, const NamesAndTypesList & columns_list); + } diff --git a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp index 6553054774e..e0214d6a79d 100644 --- a/src/Storages/MergeTree/MergeTreeMutationStatus.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationStatus.cpp @@ -26,11 +26,11 @@ void checkMutationStatus(std::optional & status, const throw Exception( ErrorCodes::UNFINISHED, "Exception happened during execution of mutation{} '{}' with part '{}' reason: '{}'. This error maybe retryable or not. " - "In case of unretryable error, mutation can be killed with KILL MUTATION query", + "In case of unretryable error, mutation can be killed with KILL MUTATION query \n\n{}\n", mutation_ids.size() > 1 ? "s" : "", boost::algorithm::join(mutation_ids, ", "), status->latest_failed_part, - status->latest_fail_reason); + status->latest_fail_reason, StackTrace().toString()); } } diff --git a/src/Storages/MergeTree/MergeTreePartInfo.h b/src/Storages/MergeTree/MergeTreePartInfo.h index f128722b03b..28b043fcf20 100644 --- a/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/src/Storages/MergeTree/MergeTreePartInfo.h @@ -46,6 +46,13 @@ struct MergeTreePartInfo < std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); } + bool operator>(const MergeTreePartInfo & rhs) const + { + return std::forward_as_tuple(partition_id, min_block, max_block, level, mutation) + > std::forward_as_tuple(rhs.partition_id, rhs.min_block, rhs.max_block, rhs.level, rhs.mutation); + } + + bool operator==(const MergeTreePartInfo & rhs) const { return !(*this != rhs); diff --git a/src/Storages/MergeTree/MergeTreePartsMover.cpp b/src/Storages/MergeTree/MergeTreePartsMover.cpp index 48a4a37f444..e9c9f2b4b06 100644 --- a/src/Storages/MergeTree/MergeTreePartsMover.cpp +++ b/src/Storages/MergeTree/MergeTreePartsMover.cpp @@ -280,7 +280,7 @@ MergeTreePartsMover::TemporaryClonedPart MergeTreePartsMover::clonePart(const Me cloned_part_storage = part->makeCloneOnDisk(disk, MergeTreeData::MOVING_DIR_NAME, read_settings, write_settings, cancellation_hook); } - MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage); + MergeTreeDataPartBuilder builder(*data, part->name, cloned_part_storage, getReadSettings()); cloned_part.part = std::move(builder).withPartFormatFromDisk().build(); LOG_TRACE(log, "Part {} was cloned to {}", part->name, cloned_part.part->getDataPartStorage().getFullPath()); diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp index a99172c4acd..4e5389f2869 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.cpp @@ -1,6 +1,6 @@ +#include #include #include -#include #include #include #include @@ -8,13 +8,13 @@ #include #include #include -#include #include +#include #include #include -#include #include -#include +#include +#include namespace ProfileEvents @@ -102,6 +102,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), @@ -113,9 +114,12 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool( reader_settings_, column_names_, settings_, + params_, context_) , prefetch_threadpool(getContext()->getPrefetchThreadpool()) - , log(getLogger("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) + , log(getLogger( + "MergeTreePrefetchedReadPool(" + + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")")) { /// Tasks creation might also create a lost of readers - check they do not /// do any time consuming operations in ctor. @@ -304,25 +308,11 @@ MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::stealTask(size_t thread, Merge MergeTreeReadTaskPtr MergeTreePrefetchedReadPool::createTask(ThreadTask & task, MergeTreeReadTask * previous_task) { if (task.isValidReadersFuture()) - { - auto size_predictor = task.read_info->shared_size_predictor - ? std::make_unique(*task.read_info->shared_size_predictor) - : nullptr; - - return std::make_unique(task.read_info, task.readers_future->get(), task.ranges, std::move(size_predictor)); - } + return MergeTreeReadPoolBase::createTask(task.read_info, task.readers_future->get(), task.ranges); return MergeTreeReadPoolBase::createTask(task.read_info, task.ranges, previous_task); } -size_t getApproximateSizeOfGranule(const IMergeTreeDataPart & part, const Names & columns_to_read) -{ - ColumnSize columns_size{}; - for (const auto & col_name : columns_to_read) - columns_size.add(part.getColumnSize(col_name)); - return columns_size.data_compressed / part.getMarksCount(); -} - void MergeTreePrefetchedReadPool::fillPerPartStatistics() { per_part_statistics.clear(); @@ -338,11 +328,7 @@ void MergeTreePrefetchedReadPool::fillPerPartStatistics() for (const auto & range : parts_ranges[i].ranges) part_stat.sum_marks += range.end - range.begin; - const auto & columns = settings[Setting::merge_tree_determine_task_size_by_prewhere_columns] && prewhere_info - ? prewhere_info->prewhere_actions.getRequiredColumnsNames() - : column_names; - - part_stat.approx_size_of_mark = getApproximateSizeOfGranule(*read_info.data_part, columns); + part_stat.approx_size_of_mark = read_info.approx_size_of_mark; auto update_stat_for_column = [&](const auto & column_name) { diff --git a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h index 1a709250937..b94d4ea113a 100644 --- a/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h +++ b/src/Storages/MergeTree/MergeTreePrefetchedReadPool.h @@ -27,6 +27,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_); String getName() const override { return "PrefetchedReadPool"; } diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.h b/src/Storages/MergeTree/MergeTreeRangeReader.h index 7acc8cd88b4..13ce14e02ec 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -35,7 +35,7 @@ struct PrewhereExprStep bool remove_filter_column = false; bool need_filter = false; - /// Some PREWHERE steps should be executed without conversions. + /// Some PREWHERE steps should be executed without conversions (e.g. early mutation steps) /// A step without alter conversion cannot be executed after step with alter conversions. bool perform_alter_conversions = false; }; diff --git a/src/Storages/MergeTree/MergeTreeReadPool.cpp b/src/Storages/MergeTree/MergeTreeReadPool.cpp index 1e4922757f4..d266ad55824 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -45,6 +45,7 @@ MergeTreeReadPool::MergeTreeReadPool( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), @@ -56,6 +57,7 @@ MergeTreeReadPool::MergeTreeReadPool( reader_settings_, column_names_, settings_, + params_, context_) , backoff_settings{context_->getSettingsRef()} , backoff_state{pool_settings.threads} diff --git a/src/Storages/MergeTree/MergeTreeReadPool.h b/src/Storages/MergeTree/MergeTreeReadPool.h index c51dca315f9..a0425f0951c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPool.h +++ b/src/Storages/MergeTree/MergeTreeReadPool.h @@ -34,6 +34,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_); ~MergeTreeReadPool() override = default; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp index 6ce1726398a..15a87f463b4 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.cpp @@ -10,6 +10,7 @@ namespace Setting { extern const SettingsBool merge_tree_determine_task_size_by_prewhere_columns; extern const SettingsUInt64 merge_tree_min_bytes_per_task_for_remote_reading; + extern const SettingsUInt64 merge_tree_min_read_task_size; } namespace ErrorCodes @@ -27,6 +28,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & pool_settings_, + const MergeTreeReadTask::BlockSizeParams & block_size_params_, const ContextPtr & context_) : WithContext(context_) , parts_ranges(std::move(parts_)) @@ -38,6 +40,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( , reader_settings(reader_settings_) , column_names(column_names_) , pool_settings(pool_settings_) + , block_size_params(block_size_params_) , owned_mark_cache(context_->getGlobalContext()->getMarkCache()) , owned_uncompressed_cache(pool_settings_.use_uncompressed_cache ? context_->getGlobalContext()->getUncompressedCache() : nullptr) , header(storage_snapshot->getSampleBlockForColumns(column_names)) @@ -46,7 +49,7 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase( fillPerPartInfos(context_->getSettingsRef()); } -static size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & columns_to_read) +static size_t getSizeOfColumns(const IMergeTreeDataPart & part, const Names & columns_to_read) { ColumnSize columns_size{}; for (const auto & col_name : columns_to_read) @@ -55,44 +58,67 @@ static size_t getApproxSizeOfPart(const IMergeTreeDataPart & part, const Names & return columns_size.data_compressed ? columns_size.data_compressed : part.getBytesOnDisk(); } -static size_t calculateMinMarksPerTask( +/// Columns from different prewhere steps are read independently, so it makes sense to use the heaviest set of columns among them as an estimation. +static Names +getHeaviestSetOfColumnsAmongPrewhereSteps(const IMergeTreeDataPart & part, const std::vector & prewhere_steps_columns) +{ + const auto it = std::ranges::max_element( + prewhere_steps_columns, + [&](const auto & lhs, const auto & rhs) + { return getSizeOfColumns(part, lhs.getNames()) < getSizeOfColumns(part, rhs.getNames()); }); + return it->getNames(); +} + +static std::pair // (min_marks_per_task, avg_mark_bytes) +calculateMinMarksPerTask( const RangesInDataPart & part, const Names & columns_to_read, - PrewhereInfoPtr prewhere_info, + const std::vector & prewhere_steps_columns, const MergeTreeReadPoolBase::PoolSettings & pool_settings, const Settings & settings) { - size_t min_marks_per_task = pool_settings.min_marks_for_concurrent_read; - const size_t part_marks_count = part.getMarksCount(); - if (part_marks_count && part.data_part->isStoredOnRemoteDisk()) + size_t min_marks_per_task + = std::max(settings[Setting::merge_tree_min_read_task_size], pool_settings.min_marks_for_concurrent_read); + size_t avg_mark_bytes = 0; + /// It is important to obtain marks count from the part itself instead of calling `part.getMarksCount()`, + /// because `part` will report number of marks selected from this part by the query. + const size_t part_marks_count = part.data_part->getMarksCount(); + if (part_marks_count) { - /// We assume that most of the time prewhere does it's job good meaning that lion's share of the rows is filtered out. - /// Which means in turn that for most of the rows we will read only the columns from prewhere clause. - /// So it makes sense to use only them for the estimation. - const auto & columns = settings[Setting::merge_tree_determine_task_size_by_prewhere_columns] && prewhere_info - ? prewhere_info->prewhere_actions.getRequiredColumnsNames() - : columns_to_read; - const size_t part_compressed_bytes = getApproxSizeOfPart(*part.data_part, columns); - - const auto avg_mark_bytes = std::max(part_compressed_bytes / part_marks_count, 1); - const auto min_bytes_per_task = settings[Setting::merge_tree_min_bytes_per_task_for_remote_reading]; - /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. - /// We also create at least two tasks per thread to have something to steal from a slow thread. - const auto heuristic_min_marks - = std::min(pool_settings.sum_marks / pool_settings.threads / 2, min_bytes_per_task / avg_mark_bytes); - if (heuristic_min_marks > min_marks_per_task) + if (part.data_part->isStoredOnRemoteDisk()) { - LOG_TEST( - &Poco::Logger::get("MergeTreeReadPoolBase"), - "Increasing min_marks_per_task from {} to {} based on columns size heuristic", - min_marks_per_task, - heuristic_min_marks); - min_marks_per_task = heuristic_min_marks; + /// We assume that most of the time prewhere does it's job good meaning that lion's share of the rows is filtered out. + /// Which means in turn that for most of the rows we will read only the columns from prewhere clause. + /// So it makes sense to use only them for the estimation. + const auto & columns = settings[Setting::merge_tree_determine_task_size_by_prewhere_columns] && !prewhere_steps_columns.empty() + ? getHeaviestSetOfColumnsAmongPrewhereSteps(*part.data_part, prewhere_steps_columns) + : columns_to_read; + const size_t part_compressed_bytes = getSizeOfColumns(*part.data_part, columns); + + avg_mark_bytes = std::max(part_compressed_bytes / part_marks_count, 1); + const auto min_bytes_per_task = settings[Setting::merge_tree_min_bytes_per_task_for_remote_reading]; + /// We're taking min here because number of tasks shouldn't be too low - it will make task stealing impossible. + /// We also create at least two tasks per thread to have something to steal from a slow thread. + const auto heuristic_min_marks + = std::min(pool_settings.sum_marks / pool_settings.threads / 2, min_bytes_per_task / avg_mark_bytes); + if (heuristic_min_marks > min_marks_per_task) + { + LOG_TEST( + &Poco::Logger::get("MergeTreeReadPoolBase"), + "Increasing min_marks_per_task from {} to {} based on columns size heuristic", + min_marks_per_task, + heuristic_min_marks); + min_marks_per_task = heuristic_min_marks; + } + } + else + { + avg_mark_bytes = std::max(getSizeOfColumns(*part.data_part, columns_to_read) / part_marks_count, 1); } } LOG_TEST(&Poco::Logger::get("MergeTreeReadPoolBase"), "Will use min_marks_per_task={}", min_marks_per_task); - return min_marks_per_task; + return {min_marks_per_task, avg_mark_bytes}; } void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings) @@ -159,8 +185,8 @@ void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings) } is_part_on_remote_disk.push_back(part_with_ranges.data_part->isStoredOnRemoteDisk()); - read_task_info.min_marks_per_task - = calculateMinMarksPerTask(part_with_ranges, column_names, prewhere_info, pool_settings, settings); + std::tie(read_task_info.min_marks_per_task, read_task_info.approx_size_of_mark) + = calculateMinMarksPerTask(part_with_ranges, column_names, read_task_info.task_columns.pre_columns, pool_settings, settings); per_part_infos.push_back(std::make_shared(std::move(read_task_info))); } } @@ -182,15 +208,20 @@ std::vector MergeTreeReadPoolBase::getPerPartSumMarks() const return per_part_sum_marks; } -MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( - MergeTreeReadTaskInfoPtr read_info, - MarkRanges ranges, - MergeTreeReadTask * previous_task) const +MergeTreeReadTaskPtr +MergeTreeReadPoolBase::createTask(MergeTreeReadTaskInfoPtr read_info, MergeTreeReadTask::Readers task_readers, MarkRanges ranges) const { auto task_size_predictor = read_info->shared_size_predictor ? std::make_unique(*read_info->shared_size_predictor) : nullptr; /// make a copy + return std::make_unique( + read_info, std::move(task_readers), std::move(ranges), block_size_params, std::move(task_size_predictor)); +} + +MergeTreeReadTaskPtr +MergeTreeReadPoolBase::createTask(MergeTreeReadTaskInfoPtr read_info, MarkRanges ranges, MergeTreeReadTask * previous_task) const +{ auto get_part_name = [](const auto & task_info) -> String { const auto & data_part = task_info.data_part; @@ -229,11 +260,7 @@ MergeTreeReadTaskPtr MergeTreeReadPoolBase::createTask( task_readers = previous_task->releaseReaders(); } - return std::make_unique( - read_info, - std::move(task_readers), - std::move(ranges), - std::move(task_size_predictor)); + return createTask(read_info, std::move(task_readers), std::move(ranges)); } MergeTreeReadTask::Extras MergeTreeReadPoolBase::getExtras() const diff --git a/src/Storages/MergeTree/MergeTreeReadPoolBase.h b/src/Storages/MergeTree/MergeTreeReadPoolBase.h index 7f9106d476e..19b26156433 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolBase.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolBase.h @@ -33,6 +33,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_); Block getHeader() const override { return header; } @@ -48,6 +49,7 @@ protected: const MergeTreeReaderSettings reader_settings; const Names column_names; const PoolSettings pool_settings; + const MergeTreeReadTask::BlockSizeParams block_size_params; const MarkCachePtr owned_mark_cache; const UncompressedCachePtr owned_uncompressed_cache; const Block header; @@ -55,6 +57,8 @@ protected: void fillPerPartInfos(const Settings & settings); std::vector getPerPartSumMarks() const; + MergeTreeReadTaskPtr createTask(MergeTreeReadTaskInfoPtr read_info, MergeTreeReadTask::Readers task_readers, MarkRanges ranges) const; + MergeTreeReadTaskPtr createTask( MergeTreeReadTaskInfoPtr read_info, MarkRanges ranges, diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp index 60f127acdae..c4244ecd982 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.cpp @@ -20,6 +20,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), @@ -31,6 +32,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder( reader_settings_, column_names_, settings_, + params_, context_) , has_limit_below_one_block(has_limit_below_one_block_) , read_type(read_type_) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h index a3668acb170..41f3ab1061c 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolInOrder.h @@ -19,6 +19,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_); String getName() const override { return "ReadPoolInOrder"; } diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp index 075c0b1042b..8f06fc312c2 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.cpp @@ -112,6 +112,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), @@ -123,6 +124,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas( reader_settings_, column_names_, settings_, + params_, context_) , extension(std::move(extension_)) , coordination_mode(CoordinationMode::Default) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h index b9f2e133c4a..63816340eb1 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicas.h @@ -19,6 +19,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_); ~MergeTreeReadPoolParallelReplicas() override = default; diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp index 8ff2a4f31ee..f13da426c45 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.cpp @@ -26,6 +26,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_) : MergeTreeReadPoolBase( std::move(parts_), @@ -37,6 +38,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd reader_settings_, column_names_, settings_, + params_, context_) , extension(std::move(extension_)) , mode(mode_) diff --git a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h index 98a4d95768a..a05dc54b529 100644 --- a/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h +++ b/src/Storages/MergeTree/MergeTreeReadPoolParallelReplicasInOrder.h @@ -20,6 +20,7 @@ public: const MergeTreeReaderSettings & reader_settings_, const Names & column_names_, const PoolSettings & settings_, + const MergeTreeReadTask::BlockSizeParams & params_, const ContextPtr & context_); String getName() const override { return "ReadPoolParallelReplicasInOrder"; } diff --git a/src/Storages/MergeTree/MergeTreeReadTask.cpp b/src/Storages/MergeTree/MergeTreeReadTask.cpp index dd057dc9984..72fddb93a6d 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.cpp +++ b/src/Storages/MergeTree/MergeTreeReadTask.cpp @@ -26,10 +26,12 @@ MergeTreeReadTask::MergeTreeReadTask( MergeTreeReadTaskInfoPtr info_, Readers readers_, MarkRanges mark_ranges_, + const BlockSizeParams & block_size_params_, MergeTreeBlockSizePredictorPtr size_predictor_) : info(std::move(info_)) , readers(std::move(readers_)) , mark_ranges(std::move(mark_ranges_)) + , block_size_params(block_size_params_) , size_predictor(std::move(size_predictor_)) { } @@ -112,30 +114,31 @@ void MergeTreeReadTask::initializeRangeReaders(const PrewhereExprInfo & prewhere range_readers = createRangeReaders(readers, prewhere_actions); } -UInt64 MergeTreeReadTask::estimateNumRows(const BlockSizeParams & params) const +UInt64 MergeTreeReadTask::estimateNumRows() const { if (!size_predictor) { - if (params.preferred_block_size_bytes) + if (block_size_params.preferred_block_size_bytes) throw Exception(ErrorCodes::LOGICAL_ERROR, "Size predictor is not set, it might lead to a performance degradation"); - return static_cast(params.max_block_size_rows); + return static_cast(block_size_params.max_block_size_rows); } /// Calculates number of rows will be read using preferred_block_size_bytes. /// Can't be less than avg_index_granularity. - size_t rows_to_read = size_predictor->estimateNumRows(params.preferred_block_size_bytes); + size_t rows_to_read = size_predictor->estimateNumRows(block_size_params.preferred_block_size_bytes); if (!rows_to_read) return rows_to_read; auto total_row_in_current_granule = range_readers.main.numRowsInCurrentGranule(); rows_to_read = std::max(total_row_in_current_granule, rows_to_read); - if (params.preferred_max_column_in_block_size_bytes) + if (block_size_params.preferred_max_column_in_block_size_bytes) { /// Calculates number of rows will be read using preferred_max_column_in_block_size_bytes. - auto rows_to_read_for_max_size_column = size_predictor->estimateNumRowsForMaxSizeColumn(params.preferred_max_column_in_block_size_bytes); + auto rows_to_read_for_max_size_column + = size_predictor->estimateNumRowsForMaxSizeColumn(block_size_params.preferred_max_column_in_block_size_bytes); - double filtration_ratio = std::max(params.min_filtration_ratio, 1.0 - size_predictor->filtered_rows_ratio); + double filtration_ratio = std::max(block_size_params.min_filtration_ratio, 1.0 - size_predictor->filtered_rows_ratio); auto rows_to_read_for_max_size_column_with_filtration = static_cast(rows_to_read_for_max_size_column / filtration_ratio); @@ -148,16 +151,16 @@ UInt64 MergeTreeReadTask::estimateNumRows(const BlockSizeParams & params) const return rows_to_read; const auto & index_granularity = info->data_part->index_granularity; - return index_granularity.countRowsForRows(range_readers.main.currentMark(), rows_to_read, range_readers.main.numReadRowsInCurrentGranule(), params.min_marks_to_read); + return index_granularity.countRowsForRows(range_readers.main.currentMark(), rows_to_read, range_readers.main.numReadRowsInCurrentGranule()); } -MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read(const BlockSizeParams & params) +MergeTreeReadTask::BlockAndProgress MergeTreeReadTask::read() { if (size_predictor) size_predictor->startBlock(); - UInt64 recommended_rows = estimateNumRows(params); - UInt64 rows_to_read = std::max(static_cast(1), std::min(params.max_block_size_rows, recommended_rows)); + UInt64 recommended_rows = estimateNumRows(); + UInt64 rows_to_read = std::max(static_cast(1), std::min(block_size_params.max_block_size_rows, recommended_rows)); auto read_result = range_readers.main.read(rows_to_read, mark_ranges); diff --git a/src/Storages/MergeTree/MergeTreeReadTask.h b/src/Storages/MergeTree/MergeTreeReadTask.h index 748babb5b4c..2853cc39c51 100644 --- a/src/Storages/MergeTree/MergeTreeReadTask.h +++ b/src/Storages/MergeTree/MergeTreeReadTask.h @@ -70,6 +70,7 @@ struct MergeTreeReadTaskInfo VirtualFields const_virtual_fields; /// The amount of data to read per task based on size of the queried columns. size_t min_marks_per_task = 0; + size_t approx_size_of_mark = 0; }; using MergeTreeReadTaskInfoPtr = std::shared_ptr; @@ -110,7 +111,6 @@ public: UInt64 max_block_size_rows = DEFAULT_BLOCK_SIZE; UInt64 preferred_block_size_bytes = 1000000; UInt64 preferred_max_column_in_block_size_bytes = 0; - UInt64 min_marks_to_read = 0; double min_filtration_ratio = 0.00001; }; @@ -127,12 +127,12 @@ public: MergeTreeReadTaskInfoPtr info_, Readers readers_, MarkRanges mark_ranges_, - + const BlockSizeParams & block_size_params_, MergeTreeBlockSizePredictorPtr size_predictor_); void initializeRangeReaders(const PrewhereExprInfo & prewhere_actions); - BlockAndProgress read(const BlockSizeParams & params); + BlockAndProgress read(); bool isFinished() const { return mark_ranges.empty() && range_readers.main.isCurrentRangeFinished(); } const MergeTreeReadTaskInfo & getInfo() const { return *info; } @@ -145,7 +145,7 @@ public: static RangeReaders createRangeReaders(const Readers & readers, const PrewhereExprInfo & prewhere_actions); private: - UInt64 estimateNumRows(const BlockSizeParams & params) const; + UInt64 estimateNumRows() const; /// Shared information required for reading. MergeTreeReadTaskInfoPtr info; @@ -160,6 +160,8 @@ private: /// Ranges to read from data_part MarkRanges mark_ranges; + BlockSizeParams block_size_params; + /// Used to satistfy preferred_block_size_bytes limitation MergeTreeBlockSizePredictorPtr size_predictor; }; diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 898bf5a2933..77231d8d392 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -262,7 +262,7 @@ MergeTreeReaderWide::FileStreams::iterator MergeTreeReaderWide::addStream(const /*num_columns_in_mark=*/ 1); auto stream_settings = settings; - stream_settings.is_low_cardinality_dictionary = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; + stream_settings.is_low_cardinality_dictionary = ISerialization::isLowCardinalityDictionarySubcolumn(substream_path); auto create_stream = [&]() { diff --git a/src/Storages/MergeTree/MergeTreeSelectAlgorithms.cpp b/src/Storages/MergeTree/MergeTreeSelectAlgorithms.cpp index bf97d269dc6..213eab52ad8 100644 --- a/src/Storages/MergeTree/MergeTreeSelectAlgorithms.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectAlgorithms.cpp @@ -30,7 +30,8 @@ MergeTreeReadTaskPtr MergeTreeInReverseOrderSelectAlgorithm::getNewTask(IMergeTr return pool.getTask(part_idx, previous_task); } -MergeTreeReadTask::BlockAndProgress MergeTreeInReverseOrderSelectAlgorithm::readFromTask(MergeTreeReadTask & task, const BlockSizeParams & params) +MergeTreeReadTask::BlockAndProgress +MergeTreeInReverseOrderSelectAlgorithm::readFromTask(MergeTreeReadTask & task) { MergeTreeReadTask::BlockAndProgress res; @@ -42,7 +43,7 @@ MergeTreeReadTask::BlockAndProgress MergeTreeInReverseOrderSelectAlgorithm::read } while (!task.isFinished()) - chunks.push_back(task.read(params)); + chunks.push_back(task.read()); if (chunks.empty()) return {}; diff --git a/src/Storages/MergeTree/MergeTreeSelectAlgorithms.h b/src/Storages/MergeTree/MergeTreeSelectAlgorithms.h index afc8032bb99..eeaefb0dc4f 100644 --- a/src/Storages/MergeTree/MergeTreeSelectAlgorithms.h +++ b/src/Storages/MergeTree/MergeTreeSelectAlgorithms.h @@ -21,7 +21,7 @@ public: virtual bool needNewTask(const MergeTreeReadTask & task) const = 0; virtual MergeTreeReadTaskPtr getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) = 0; - virtual BlockAndProgress readFromTask(MergeTreeReadTask & task, const BlockSizeParams & params) = 0; + virtual BlockAndProgress readFromTask(MergeTreeReadTask & task) = 0; }; using MergeTreeSelectAlgorithmPtr = std::unique_ptr; @@ -35,7 +35,7 @@ public: bool needNewTask(const MergeTreeReadTask & task) const override { return task.isFinished(); } MergeTreeReadTaskPtr getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) override { return pool.getTask(thread_idx, previous_task); } - BlockAndProgress readFromTask(MergeTreeReadTask & task, const BlockSizeParams & params) override { return task.read(params); } + BlockAndProgress readFromTask(MergeTreeReadTask & task) override { return task.read(); } private: const size_t thread_idx; @@ -50,7 +50,7 @@ public: bool needNewTask(const MergeTreeReadTask & task) const override { return task.isFinished(); } MergeTreeReadTaskPtr getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) override; - MergeTreeReadTask::BlockAndProgress readFromTask(MergeTreeReadTask & task, const BlockSizeParams & params) override { return task.read(params); } + MergeTreeReadTask::BlockAndProgress readFromTask(MergeTreeReadTask & task) override { return task.read(); } private: const size_t part_idx; @@ -65,7 +65,7 @@ public: bool needNewTask(const MergeTreeReadTask & task) const override { return chunks.empty() && task.isFinished(); } MergeTreeReadTaskPtr getNewTask(IMergeTreeReadPool & pool, MergeTreeReadTask * previous_task) override; - BlockAndProgress readFromTask(MergeTreeReadTask & task, const BlockSizeParams & params) override; + BlockAndProgress readFromTask(MergeTreeReadTask & task) override; private: const size_t part_idx; diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp index 76bcf41d6d8..5efd33ce09a 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.cpp @@ -86,7 +86,6 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( MergeTreeSelectAlgorithmPtr algorithm_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, - const MergeTreeReadTask::BlockSizeParams & block_size_params_, const MergeTreeReaderSettings & reader_settings_) : pool(std::move(pool_)) , algorithm(std::move(algorithm_)) @@ -94,7 +93,6 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor( , actions_settings(actions_settings_) , prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps)) , reader_settings(reader_settings_) - , block_size_params(block_size_params_) , result_header(transformHeader(pool->getHeader(), prewhere_info)) { if (reader_settings.apply_deleted_mask) @@ -190,7 +188,7 @@ ChunkAndProgress MergeTreeSelectProcessor::read() if (!task->getMainRangeReader().isInitialized()) initializeRangeReaders(); - auto res = algorithm->readFromTask(*task, block_size_params); + auto res = algorithm->readFromTask(*task); if (res.row_count) { diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 8a9e3580a9f..33069a78e33 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -57,7 +57,6 @@ public: MergeTreeSelectAlgorithmPtr algorithm_, const PrewhereInfoPtr & prewhere_info_, const ExpressionActionsSettings & actions_settings_, - const MergeTreeReadTask::BlockSizeParams & block_size_params_, const MergeTreeReaderSettings & reader_settings_); String getName() const; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 4e7d0c0a721..33910d1048d 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -29,218 +30,220 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +// clang-format off + /** These settings represent fine tunes for internal details of MergeTree storages * and should not be changed by the user without a reason. */ - -#define MERGE_TREE_SETTINGS(M, ALIAS) \ - M(UInt64, min_compress_block_size, 0, "When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used.", 0) \ - M(UInt64, max_compress_block_size, 0, "Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used.", 0) \ - M(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ - M(UInt64, max_digestion_size_per_segment, 256_MiB, "Max number of bytes to digest per segment to build GIN index.", 0) \ +#define MERGE_TREE_SETTINGS(DECLARE, ALIAS) \ + DECLARE(UInt64, min_compress_block_size, 0, "When granule is written, compress the data in buffer if the size of pending uncompressed data is larger or equal than the specified threshold. If this setting is not set, the corresponding global setting is used.", 0) \ + DECLARE(UInt64, max_compress_block_size, 0, "Compress the pending uncompressed data in buffer if its size is larger or equal than the specified threshold. Block of data will be compressed even if the current granule is not finished. If this setting is not set, the corresponding global setting is used.", 0) \ + DECLARE(UInt64, index_granularity, 8192, "How many rows correspond to one primary key value.", 0) \ + DECLARE(UInt64, max_digestion_size_per_segment, 256_MiB, "Max number of bytes to digest per segment to build GIN index.", 0) \ \ /** Data storing format settings. */ \ - M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ - M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ - M(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ - M(Bool, replace_long_file_name_to_hash, true, "If the file name for column is too long (more than 'max_file_name_length' bytes) replace it to SipHash128", 0) \ - M(UInt64, max_file_name_length, 127, "The maximal length of the file name to keep it as is without hashing", 0) \ - M(UInt64, min_bytes_for_full_part_storage, 0, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, min_rows_for_full_part_storage, 0, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, compact_parts_max_bytes_to_buffer, 128 * 1024 * 1024, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, compact_parts_max_granules_to_buffer, 128, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, compact_parts_merge_max_bytes_to_prefetch_part, 16 * 1024 * 1024, "Only available in ClickHouse Cloud", 0) \ - M(Bool, load_existing_rows_count_for_old_parts, false, "Whether to load existing_rows_count for existing parts. If false, existing_rows_count will be equal to rows_count for existing parts.", 0) \ - M(Bool, use_compact_variant_discriminators_serialization, true, "Use compact version of Variant discriminators serialization.", 0) \ - \ - /** Merge and insert settings */ \ - M(UInt64, max_compression_threads, 1, "Maximum number of threads for writing compressed data. This is an expert-level setting, do not change it.", 0) \ + DECLARE(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ + DECLARE(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ + DECLARE(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ + DECLARE(Bool, replace_long_file_name_to_hash, true, "If the file name for column is too long (more than 'max_file_name_length' bytes) replace it to SipHash128", 0) \ + DECLARE(UInt64, max_file_name_length, 127, "The maximal length of the file name to keep it as is without hashing", 0) \ + DECLARE(UInt64, min_bytes_for_full_part_storage, 0, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, min_rows_for_full_part_storage, 0, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, compact_parts_max_bytes_to_buffer, 128 * 1024 * 1024, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, compact_parts_max_granules_to_buffer, 128, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, compact_parts_merge_max_bytes_to_prefetch_part, 16 * 1024 * 1024, "Only available in ClickHouse Cloud", 0) \ + DECLARE(Bool, load_existing_rows_count_for_old_parts, false, "Whether to load existing_rows_count for existing parts. If false, existing_rows_count will be equal to rows_count for existing parts.", 0) \ + DECLARE(Bool, use_compact_variant_discriminators_serialization, true, "Use compact version of Variant discriminators serialization.", 0) \ \ /** Merge selector settings. */ \ - M(UInt64, merge_selector_blurry_base_scale_factor, 0, "Controls when the logic kicks in relatively to the number of parts in partition. The bigger the factor the more belated reaction will be.", 0) \ - M(UInt64, merge_selector_window_size, 1000, "How many parts to look at once.", 0) \ + DECLARE(UInt64, merge_selector_blurry_base_scale_factor, 0, "Controls when the logic kicks in relatively to the number of parts in partition. The bigger the factor the more belated reaction will be.", 0) \ + DECLARE(UInt64, merge_selector_window_size, 1000, "How many parts to look at once.", 0) \ \ /** Merge settings. */ \ - M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ - M(UInt64, merge_max_block_size_bytes, 10 * 1024 * 1024, "How many bytes in blocks should be formed for merge operations. By default has the same value as `index_granularity_bytes`.", 0) \ - M(UInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \ - M(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ - M(UInt64, max_replicated_merges_in_queue, 1000, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ - M(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ - M(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ - M(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ - M(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 20, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(UInt64, max_number_of_mutations_for_replica, 0, "Limit the number of part mutations per replica to the specified amount. Zero means no limit on the number of mutations per replica (the execution can still be constrained by other settings).", 0) \ - M(UInt64, max_number_of_merges_with_ttl_in_pool, 2, "When there is more than specified number of merges with TTL entries in pool, do not assign new merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ - M(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories. You should not lower this value because merges and mutations may not be able to work with low value of this setting.", 0) \ - M(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ - M(UInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ - M(UInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ - M(UInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ - M(Bool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ - M(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ - M(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \ - M(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \ - M(UInt64, merge_selecting_sleep_ms, 5000, "Minimum time to wait before trying to select parts to merge again after no parts were selected. A lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ - M(UInt64, max_merge_selecting_sleep_ms, 60000, "Maximum time to wait before trying to select parts to merge again after no parts were selected. A lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ - M(Float, merge_selecting_sleep_slowdown_factor, 1.2f, "The sleep time for merge selecting task is multiplied by this factor when there's nothing to merge and divided when a merge was assigned", 0) \ - M(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ - M(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ - M(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ - M(Bool, min_age_to_force_merge_on_partition_only, false, "Whether min_age_to_force_merge_seconds should be applied only on the entire partition and not on subset.", false) \ - M(UInt64, number_of_free_entries_in_pool_to_execute_optimize_entire_partition, 25, "When there is less than specified number of free entries in pool, do not try to execute optimize entire partition with a merge (this merge is created when set min_age_to_force_merge_seconds > 0 and min_age_to_force_merge_on_partition_only = true). This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ - M(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", 0) \ - M(UInt64, replicated_max_mutations_in_one_entry, 10000, "Max number of mutation commands that can be merged together and executed in one MUTATE_PART entry (0 means unlimited)", 0) \ - M(UInt64, number_of_mutations_to_delay, 500, "If table has at least that many unfinished mutations, artificially slow down mutations of table. Disabled if set to 0", 0) \ - M(UInt64, number_of_mutations_to_throw, 1000, "If table has at least that many unfinished mutations, throw 'Too many mutations' exception. Disabled if set to 0", 0) \ - M(UInt64, min_delay_to_mutate_ms, 10, "Min delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ - M(UInt64, max_delay_to_mutate_ms, 1000, "Max delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ - M(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ - M(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ - M(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ - M(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ - M(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", 0) \ + DECLARE(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ + DECLARE(UInt64, merge_max_block_size_bytes, 10 * 1024 * 1024, "How many bytes in blocks should be formed for merge operations. By default has the same value as `index_granularity_bytes`.", 0) \ + DECLARE(UInt64, max_bytes_to_merge_at_max_space_in_pool, 150ULL * 1024 * 1024 * 1024, "Maximum in total size of parts to merge, when there are maximum free threads in background pool (or entries in replication queue).", 0) \ + DECLARE(UInt64, max_bytes_to_merge_at_min_space_in_pool, 1024 * 1024, "Maximum in total size of parts to merge, when there are minimum free threads in background pool (or entries in replication queue).", 0) \ + DECLARE(UInt64, max_replicated_merges_in_queue, 1000, "How many tasks of merging and mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + DECLARE(UInt64, max_replicated_mutations_in_queue, 8, "How many tasks of mutating parts are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + DECLARE(UInt64, max_replicated_merges_with_ttl_in_queue, 1, "How many tasks of merging parts with TTL are allowed simultaneously in ReplicatedMergeTree queue.", 0) \ + DECLARE(UInt64, number_of_free_entries_in_pool_to_lower_max_size_of_merge, 8, "When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). This is to allow small merges to process - not filling the pool with long running merges.", 0) \ + DECLARE(UInt64, number_of_free_entries_in_pool_to_execute_mutation, 20, "When there is less than specified number of free entries in pool, do not execute part mutations. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + DECLARE(UInt64, max_number_of_mutations_for_replica, 0, "Limit the number of part mutations per replica to the specified amount. Zero means no limit on the number of mutations per replica (the execution can still be constrained by other settings).", 0) \ + DECLARE(UInt64, max_number_of_merges_with_ttl_in_pool, 2, "When there is more than specified number of merges with TTL entries in pool, do not assign new merge with TTL. This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + DECLARE(Seconds, old_parts_lifetime, 8 * 60, "How many seconds to keep obsolete parts.", 0) \ + DECLARE(Seconds, temporary_directories_lifetime, 86400, "How many seconds to keep tmp_-directories. You should not lower this value because merges and mutations may not be able to work with low value of this setting.", 0) \ + DECLARE(Seconds, lock_acquire_timeout_for_background_operations, DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC, "For background operations like merges, mutations etc. How many seconds before failing to acquire table locks.", 0) \ + DECLARE(UInt64, min_rows_to_fsync_after_merge, 0, "Minimal number of rows to do fsync for part after merge (0 - disabled)", 0) \ + DECLARE(UInt64, min_compressed_bytes_to_fsync_after_merge, 0, "Minimal number of compressed bytes to do fsync for part after merge (0 - disabled)", 0) \ + DECLARE(UInt64, min_compressed_bytes_to_fsync_after_fetch, 0, "Minimal number of compressed bytes to do fsync for part after fetch (0 - disabled)", 0) \ + DECLARE(Bool, fsync_after_insert, false, "Do fsync for every inserted part. Significantly decreases performance of inserts, not recommended to use with wide parts.", 0) \ + DECLARE(Bool, fsync_part_directory, false, "Do fsync for part directory after all part operations (writes, renames, etc.).", 0) \ + DECLARE(UInt64, non_replicated_deduplication_window, 0, "How many last blocks of hashes should be kept on disk (0 - disabled).", 0) \ + DECLARE(UInt64, max_parts_to_merge_at_once, 100, "Max amount of parts which can be merged at once (0 - disabled). Doesn't affect OPTIMIZE FINAL query.", 0) \ + DECLARE(UInt64, merge_selecting_sleep_ms, 5000, "Minimum time to wait before trying to select parts to merge again after no parts were selected. A lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + DECLARE(UInt64, max_merge_selecting_sleep_ms, 60000, "Maximum time to wait before trying to select parts to merge again after no parts were selected. A lower setting will trigger selecting tasks in background_schedule_pool frequently which result in large amount of requests to zookeeper in large-scale clusters", 0) \ + DECLARE(Float, merge_selecting_sleep_slowdown_factor, 1.2f, "The sleep time for merge selecting task is multiplied by this factor when there's nothing to merge and divided when a merge was assigned", 0) \ + DECLARE(UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60, "The period of executing the clear old temporary directories operation in background.", 0) \ + DECLARE(UInt64, merge_tree_clear_old_parts_interval_seconds, 1, "The period of executing the clear old parts operation in background.", 0) \ + DECLARE(UInt64, min_age_to_force_merge_seconds, 0, "If all parts in a certain range are older than this value, range will be always eligible for merging. Set to 0 to disable.", 0) \ + DECLARE(Bool, min_age_to_force_merge_on_partition_only, false, "Whether min_age_to_force_merge_seconds should be applied only on the entire partition and not on subset.", false) \ + DECLARE(UInt64, number_of_free_entries_in_pool_to_execute_optimize_entire_partition, 25, "When there is less than specified number of free entries in pool, do not try to execute optimize entire partition with a merge (this merge is created when set min_age_to_force_merge_seconds > 0 and min_age_to_force_merge_on_partition_only = true). This is to leave free threads for regular merges and avoid \"Too many parts\"", 0) \ + DECLARE(Bool, remove_rolled_back_parts_immediately, 1, "Setting for an incomplete experimental feature.", EXPERIMENTAL) \ + DECLARE(UInt64, replicated_max_mutations_in_one_entry, 10000, "Max number of mutation commands that can be merged together and executed in one MUTATE_PART entry (0 means unlimited)", 0) \ + DECLARE(UInt64, number_of_mutations_to_delay, 500, "If table has at least that many unfinished mutations, artificially slow down mutations of table. Disabled if set to 0", 0) \ + DECLARE(UInt64, number_of_mutations_to_throw, 1000, "If table has at least that many unfinished mutations, throw 'Too many mutations' exception. Disabled if set to 0", 0) \ + DECLARE(UInt64, min_delay_to_mutate_ms, 10, "Min delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ + DECLARE(UInt64, max_delay_to_mutate_ms, 1000, "Max delay of mutating MergeTree table in milliseconds, if there are a lot of unfinished mutations", 0) \ + DECLARE(Bool, exclude_deleted_rows_for_part_size_in_merge, false, "Use an estimated source part size (excluding lightweight deleted rows) when selecting parts to merge", 0) \ + DECLARE(String, merge_workload, "", "Name of workload to be used to access resources for merges", 0) \ + DECLARE(String, mutation_workload, "", "Name of workload to be used to access resources for mutations", 0) \ + DECLARE(Milliseconds, background_task_preferred_step_execution_time_ms, 50, "Target time to execution of one step of merge or mutation. Can be exceeded if one step takes longer time", 0) \ + DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", EXPERIMENTAL) \ + DECLARE(Bool, merge_selector_enable_heuristic_to_remove_small_parts_at_right, true, "Enable heuristic for selecting parts for merge which removes parts from right side of range, if their size is less than specified ratio (0.01) of sum_size. Works for Simple and StochasticSimple merge selectors", 0) \ + DECLARE(Float, merge_selector_base, 5.0, "Affects write amplification of assigned merges (expert level setting, don't change if you don't understand what it is doing). Works for Simple and StochasticSimple merge selectors", 0) \ \ /** Inserts settings. */ \ - M(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ - M(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \ - M(UInt64, parts_to_throw_insert, 3000, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ - M(UInt64, inactive_parts_to_throw_insert, 0, "If more than this number inactive parts in single partition, throw 'Too many inactive parts ...' exception.", 0) \ - M(UInt64, max_avg_part_size_for_too_many_parts, 1ULL * 1024 * 1024 * 1024, "The 'too many parts' check according to 'parts_to_delay_insert' and 'parts_to_throw_insert' will be active only if the average part size (in the relevant partition) is not larger than the specified threshold. If it is larger than the specified threshold, the INSERTs will be neither delayed or rejected. This allows to have hundreds of terabytes in a single table on a single server if the parts are successfully merged to larger parts. This does not affect the thresholds on inactive parts or total parts.", 0) \ - M(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ - M(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \ - M(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ - M(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background.", 0) \ - M(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ - M(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ - M(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ - M(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ - M(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ - M(UInt64, min_free_disk_bytes_to_perform_insert, 0, "Minimum free disk space bytes to perform an insert.", 0) \ - M(Float, min_free_disk_ratio_to_perform_insert, 0.0, "Minimum free disk space ratio to perform an insert.", 0) \ + DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ + DECLARE(UInt64, inactive_parts_to_delay_insert, 0, "If table contains at least that many inactive parts in single partition, artificially slow down insert into table.", 0) \ + DECLARE(UInt64, parts_to_throw_insert, 3000, "If more than this number active parts in single partition, throw 'Too many parts ...' exception.", 0) \ + DECLARE(UInt64, inactive_parts_to_throw_insert, 0, "If more than this number inactive parts in single partition, throw 'Too many inactive parts ...' exception.", 0) \ + DECLARE(UInt64, max_avg_part_size_for_too_many_parts, 1ULL * 1024 * 1024 * 1024, "The 'too many parts' check according to 'parts_to_delay_insert' and 'parts_to_throw_insert' will be active only if the average part size (in the relevant partition) is not larger than the specified threshold. If it is larger than the specified threshold, the INSERTs will be neither delayed or rejected. This allows to have hundreds of terabytes in a single table on a single server if the parts are successfully merged to larger parts. This does not affect the thresholds on inactive parts or total parts.", 0) \ + DECLARE(UInt64, max_delay_to_insert, 1, "Max delay of inserting data into MergeTree table in seconds, if there are a lot of unmerged parts in single partition.", 0) \ + DECLARE(UInt64, min_delay_to_insert_ms, 10, "Min delay of inserting data into MergeTree table in milliseconds, if there are a lot of unmerged parts in single partition.", 0) \ + DECLARE(UInt64, max_parts_in_total, 100000, "If more than this number active parts in all partitions in total, throw 'Too many parts ...' exception.", 0) \ + DECLARE(Bool, async_insert, false, "If true, data from INSERT query is stored in queue and later flushed to table in background.", 0) \ + DECLARE(Bool, add_implicit_sign_column_constraint_for_collapsing_engine, false, "If true, add implicit constraint for sign column for CollapsingMergeTree engine.", 0) \ + DECLARE(Milliseconds, sleep_before_commit_local_part_in_replicated_table_ms, 0, "For testing. Do not change it.", 0) \ + DECLARE(Bool, optimize_row_order, false, "Allow reshuffling of rows during part inserts and merges to improve the compressibility of the new part", 0) \ + DECLARE(Bool, use_adaptive_write_buffer_for_dynamic_subcolumns, true, "Allow to use adaptive writer buffers during writing dynamic subcolumns to reduce memory usage", 0) \ + DECLARE(UInt64, adaptive_write_buffer_initial_size, 16 * 1024, "Initial size of an adaptive write buffer", 0) \ + DECLARE(UInt64, min_free_disk_bytes_to_perform_insert, 0, "Minimum free disk space bytes to perform an insert.", 0) \ + DECLARE(Float, min_free_disk_ratio_to_perform_insert, 0.0, "Minimum free disk space ratio to perform an insert.", 0) \ \ /* Part removal settings. */ \ - M(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ + DECLARE(UInt64, simultaneous_parts_removal_limit, 0, "Maximum number of parts to remove during one CleanupThread iteration (0 means unlimited).", 0) \ \ /** Replication settings. */ \ - M(UInt64, replicated_deduplication_window, 1000, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ - M(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ - M(UInt64, replicated_deduplication_window_for_async_inserts, 10000, "How many last hash values of async_insert blocks should be kept in ZooKeeper (old blocks will be deleted).", 0) \ - M(UInt64, replicated_deduplication_window_seconds_for_async_inserts, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window_for_async_inserts\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ - M(Milliseconds, async_block_ids_cache_update_wait_ms, 100, "How long each insert iteration will wait for async_block_ids_cache update", 0) \ - M(Bool, use_async_block_ids_cache, true, "Use in-memory cache to filter duplicated async inserts based on block ids", 0) \ - M(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \ - M(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ - M(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ - M(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ - M(Seconds, execute_merges_on_single_replica_time_threshold, 0, "When greater than zero only a single replica starts the merge immediately, others wait up to that amount of time to download the result instead of doing merges locally. If the chosen replica doesn't finish the merge during that amount of time, fallback to standard behavior happens.", 0) \ - M(Seconds, remote_fs_execute_merges_on_single_replica_time_threshold, 3 * 60 * 60, "When greater than zero only a single replica starts the merge immediately if merged part on shared storage and 'allow_remote_fs_zero_copy_replication' is enabled.", 0) \ - M(Seconds, try_fetch_recompressed_part_timeout, 7200, "Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.", 0) \ - M(Bool, always_fetch_merged_part, false, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ - M(UInt64, max_suspicious_broken_parts, 100, "Max broken parts, if more - deny automatic deletion.", 0) \ - M(UInt64, max_suspicious_broken_parts_bytes, 1ULL * 1024 * 1024 * 1024, "Max size of all broken parts, if more - deny automatic deletion.", 0) \ - M(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ - M(UInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ - M(Float, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \ - M(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ - M(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ - M(Seconds, initialization_retry_period, 60, "Retry period for table initialization, in seconds.", 0) \ - M(Bool, detach_old_local_parts_when_cloning_replica, true, "Do not remove old local parts when repairing lost replica.", 0) \ - M(Bool, detach_not_byte_identical_parts, false, "Do not remove non byte-idential parts for ReplicatedMergeTree, instead detach them (maybe useful for further analysis).", 0) \ - M(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ - M(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ - M(Milliseconds, wait_for_unique_parts_send_before_shutdown_ms, 0, "Before shutdown table will wait for required amount time for unique parts (exist only on current replica) to be fetched by other replicas (0 means disabled).", 0) \ - M(Float, fault_probability_before_part_commit, 0, "For testing. Do not change it.", 0) \ - M(Float, fault_probability_after_part_commit, 0, "For testing. Do not change it.", 0) \ - M(Bool, shared_merge_tree_disable_merges_and_mutations_assignment, false, "Only available in ClickHouse Cloud", 0) \ - M(Float, shared_merge_tree_partitions_hint_ratio_to_reload_merge_pred_for_mutations, 0.5, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, shared_merge_tree_parts_load_batch_size, 32, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, replicated_deduplication_window, 1000, "How many last blocks of hashes should be kept in ZooKeeper (old blocks will be deleted).", 0) \ + DECLARE(UInt64, replicated_deduplication_window_seconds, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ + DECLARE(UInt64, replicated_deduplication_window_for_async_inserts, 10000, "How many last hash values of async_insert blocks should be kept in ZooKeeper (old blocks will be deleted).", 0) \ + DECLARE(UInt64, replicated_deduplication_window_seconds_for_async_inserts, 7 * 24 * 60 * 60 /* one week */, "Similar to \"replicated_deduplication_window_for_async_inserts\", but determines old blocks by their lifetime. Hash of an inserted block will be deleted (and the block will not be deduplicated after) if it outside of one \"window\". You can set very big replicated_deduplication_window to avoid duplicating INSERTs during that period of time.", 0) \ + DECLARE(Milliseconds, async_block_ids_cache_update_wait_ms, 100, "How long each insert iteration will wait for async_block_ids_cache update", 0) \ + DECLARE(Bool, use_async_block_ids_cache, true, "Use in-memory cache to filter duplicated async inserts based on block ids", 0) \ + DECLARE(UInt64, max_replicated_logs_to_keep, 1000, "How many records may be in log, if there is inactive replica. Inactive replica becomes lost when when this number exceed.", 0) \ + DECLARE(UInt64, min_replicated_logs_to_keep, 10, "Keep about this number of last records in ZooKeeper log, even if they are obsolete. It doesn't affect work of tables: used only to diagnose ZooKeeper log before cleaning.", 0) \ + DECLARE(Seconds, prefer_fetch_merged_part_time_threshold, 3600, "If time passed after replication log entry creation exceeds this threshold and sum size of parts is greater than \"prefer_fetch_merged_part_size_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + DECLARE(UInt64, prefer_fetch_merged_part_size_threshold, 10ULL * 1024 * 1024 * 1024, "If sum size of parts exceeds this threshold and time passed after replication log entry creation is greater than \"prefer_fetch_merged_part_time_threshold\", prefer fetching merged part from replica instead of doing merge locally. To speed up very long merges.", 0) \ + DECLARE(Seconds, execute_merges_on_single_replica_time_threshold, 0, "When greater than zero only a single replica starts the merge immediately, others wait up to that amount of time to download the result instead of doing merges locally. If the chosen replica doesn't finish the merge during that amount of time, fallback to standard behavior happens.", 0) \ + DECLARE(Seconds, remote_fs_execute_merges_on_single_replica_time_threshold, 3 * 60 * 60, "When greater than zero only a single replica starts the merge immediately if merged part on shared storage and 'allow_remote_fs_zero_copy_replication' is enabled.", 0) \ + DECLARE(Seconds, try_fetch_recompressed_part_timeout, 7200, "Recompression works slow in most cases, so we don't start merge with recompression until this timeout and trying to fetch recompressed part from replica which assigned this merge with recompression.", 0) \ + DECLARE(Bool, always_fetch_merged_part, false, "If true, replica never merge parts and always download merged parts from other replicas.", 0) \ + DECLARE(UInt64, max_suspicious_broken_parts, 100, "Max broken parts, if more - deny automatic deletion.", 0) \ + DECLARE(UInt64, max_suspicious_broken_parts_bytes, 1ULL * 1024 * 1024 * 1024, "Max size of all broken parts, if more - deny automatic deletion.", 0) \ + DECLARE(UInt64, max_files_to_modify_in_alter_columns, 75, "Not apply ALTER if number of files for modification(deletion, addition) more than this.", 0) \ + DECLARE(UInt64, max_files_to_remove_in_alter_columns, 50, "Not apply ALTER, if number of files for deletion more than this.", 0) \ + DECLARE(Float, replicated_max_ratio_of_wrong_parts, 0.5, "If ratio of wrong parts to total number of parts is less than this - allow to start.", 0) \ + DECLARE(Bool, replicated_can_become_leader, true, "If true, Replicated tables replicas on this node will try to acquire leadership.", 0) \ + DECLARE(Seconds, zookeeper_session_expiration_check_period, 60, "ZooKeeper session expiration check period, in seconds.", 0) \ + DECLARE(Seconds, initialization_retry_period, 60, "Retry period for table initialization, in seconds.", 0) \ + DECLARE(Bool, detach_old_local_parts_when_cloning_replica, true, "Do not remove old local parts when repairing lost replica.", 0) \ + DECLARE(Bool, detach_not_byte_identical_parts, false, "Do not remove non byte-idential parts for ReplicatedMergeTree, instead detach them (maybe useful for further analysis).", 0) \ + DECLARE(UInt64, max_replicated_fetches_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated fetches. Zero means unlimited.", 0) \ + DECLARE(UInt64, max_replicated_sends_network_bandwidth, 0, "The maximum speed of data exchange over the network in bytes per second for replicated sends. Zero means unlimited.", 0) \ + DECLARE(Milliseconds, wait_for_unique_parts_send_before_shutdown_ms, 0, "Before shutdown table will wait for required amount time for unique parts (exist only on current replica) to be fetched by other replicas (0 means disabled).", 0) \ + DECLARE(Float, fault_probability_before_part_commit, 0, "For testing. Do not change it.", 0) \ + DECLARE(Float, fault_probability_after_part_commit, 0, "For testing. Do not change it.", 0) \ + DECLARE(Bool, shared_merge_tree_disable_merges_and_mutations_assignment, false, "Only available in ClickHouse Cloud", 0) \ + DECLARE(Float, shared_merge_tree_partitions_hint_ratio_to_reload_merge_pred_for_mutations, 0.5, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, shared_merge_tree_parts_load_batch_size, 32, "Only available in ClickHouse Cloud", 0) \ \ /** Check delay of replicas settings. */ \ - M(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ - M(UInt64, cleanup_delay_period, 30, "Minimum period to clean old queue logs, blocks hashes and parts.", 0) \ - M(UInt64, max_cleanup_delay_period, 300, "Maximum period to clean old queue logs, blocks hashes and parts.", 0) \ - M(UInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ - M(UInt64, cleanup_thread_preferred_points_per_iteration, 150, "Preferred batch size for background cleanup (points are abstract but 1 point is approximately equivalent to 1 inserted block).", 0) \ - M(UInt64, cleanup_threads, 128, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, kill_delay_period, 30, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, kill_delay_period_random_add, 10, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, kill_threads, 128, "Only available in ClickHouse Cloud", 0) \ - M(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ - M(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ - M(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ - M(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * 8192, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \ - M(UInt64, vertical_merge_algorithm_min_bytes_to_activate, 0, "Minimal (approximate) uncompressed size in bytes in merging parts to activate Vertical merge algorithm.", 0) \ - M(UInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \ - M(Bool, vertical_merge_remote_filesystem_prefetch, true, "If true prefetching of data from remote filesystem is used for the next column during merge", 0) \ - M(UInt64, max_postpone_time_for_failed_mutations_ms, 5ULL * 60 * 1000, "The maximum postpone time for failed mutations.", 0) \ + DECLARE(UInt64, min_relative_delay_to_measure, 120, "Calculate relative replica delay only if absolute delay is not less that this value.", 0) \ + DECLARE(UInt64, cleanup_delay_period, 30, "Minimum period to clean old queue logs, blocks hashes and parts.", 0) \ + DECLARE(UInt64, max_cleanup_delay_period, 300, "Maximum period to clean old queue logs, blocks hashes and parts.", 0) \ + DECLARE(UInt64, cleanup_delay_period_random_add, 10, "Add uniformly distributed value from 0 to x seconds to cleanup_delay_period to avoid thundering herd effect and subsequent DoS of ZooKeeper in case of very large number of tables.", 0) \ + DECLARE(UInt64, cleanup_thread_preferred_points_per_iteration, 150, "Preferred batch size for background cleanup (points are abstract but 1 point is approximately equivalent to 1 inserted block).", 0) \ + DECLARE(UInt64, cleanup_threads, 128, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, kill_delay_period, 30, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, kill_delay_period_random_add, 10, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, kill_threads, 128, "Only available in ClickHouse Cloud", 0) \ + DECLARE(UInt64, min_relative_delay_to_close, 300, "Minimal delay from other replicas to close, stop serving requests and not return Ok during status check.", 0) \ + DECLARE(UInt64, min_absolute_delay_to_close, 0, "Minimal absolute delay to close, stop serving requests and not return Ok during status check.", 0) \ + DECLARE(UInt64, enable_vertical_merge_algorithm, 1, "Enable usage of Vertical merge algorithm.", 0) \ + DECLARE(UInt64, vertical_merge_algorithm_min_rows_to_activate, 16 * 8192, "Minimal (approximate) sum of rows in merging parts to activate Vertical merge algorithm.", 0) \ + DECLARE(UInt64, vertical_merge_algorithm_min_bytes_to_activate, 0, "Minimal (approximate) uncompressed size in bytes in merging parts to activate Vertical merge algorithm.", 0) \ + DECLARE(UInt64, vertical_merge_algorithm_min_columns_to_activate, 11, "Minimal amount of non-PK columns to activate Vertical merge algorithm.", 0) \ + DECLARE(Bool, vertical_merge_remote_filesystem_prefetch, true, "If true prefetching of data from remote filesystem is used for the next column during merge", 0) \ + DECLARE(UInt64, max_postpone_time_for_failed_mutations_ms, 5ULL * 60 * 1000, "The maximum postpone time for failed mutations.", 0) \ \ /** Compatibility settings */ \ - M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ - M(Bool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \ - M(Bool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \ - M(Bool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \ - M(UInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \ - M(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ - M(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ - M(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ - M(Int64, merge_with_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \ - M(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ - M(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ - M(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ - M(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \ - M(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ - M(UInt64, zero_copy_concurrent_part_removal_max_split_times, 5, "Max recursion depth for splitting independent Outdated parts ranges into smaller subranges (highly not recommended to change)", 0) \ - M(Float, zero_copy_concurrent_part_removal_max_postpone_ratio, static_cast(0.05), "Max percentage of top level parts to postpone removal in order to get smaller independent ranges (highly not recommended to change)", 0) \ - M(String, storage_policy, "default", "Name of storage disk policy", 0) \ - M(String, disk, "", "Name of storage disk. Can be specified instead of storage policy.", 0) \ - M(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ - M(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \ - M(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ - M(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ - M(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ - M(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ - M(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ - M(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ - M(Bool, allow_vertical_merges_from_compact_to_wide_parts, true, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ - M(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ - M(UInt64, zero_copy_merge_mutation_min_parts_size_sleep_before_lock, 1ULL * 1024 * 1024 * 1024, "If zero copy replication is enabled sleep random amount of time before trying to lock depending on parts size for merge or mutation", 0) \ - M(Bool, allow_floating_point_partition_key, false, "Allow floating point as partition key", 0) \ - M(UInt64, sleep_before_loading_outdated_parts_ms, 0, "For testing. Do not change it.", 0) \ - M(Bool, always_use_copy_instead_of_hardlinks, false, "Always copy data instead of hardlinking during mutations/replaces/detaches and so on.", 0) \ - M(Bool, disable_freeze_partition_for_zero_copy_replication, true, "Disable FREEZE PARTITION query for zero copy replication.", 0) \ - M(Bool, disable_detach_partition_for_zero_copy_replication, true, "Disable DETACH PARTITION query for zero copy replication.", 0) \ - M(Bool, disable_fetch_partition_for_zero_copy_replication, true, "Disable FETCH PARTITION query for zero copy replication.", 0) \ - M(Bool, enable_block_number_column, false, "Enable persisting column _block_number for each row.", 0) ALIAS(allow_experimental_block_number_column) \ - M(Bool, enable_block_offset_column, false, "Enable persisting column _block_offset for each row.", 0) \ + DECLARE(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \ + DECLARE(Bool, compatibility_allow_sampling_expression_not_in_primary_key, false, "Allow to create a table with sampling expression not in primary key. This is needed only to temporarily allow to run the server with wrong tables for backward compatibility.", 0) \ + DECLARE(Bool, use_minimalistic_checksums_in_zookeeper, true, "Use small format (dozens bytes) for part checksums in ZooKeeper instead of ordinary ones (dozens KB). Before enabling check that all replicas support new format.", 0) \ + DECLARE(Bool, use_minimalistic_part_header_in_zookeeper, true, "Store part header (checksums and columns) in a compact format and a single part znode instead of separate znodes (/columns and /checksums). This can dramatically reduce snapshot size in ZooKeeper. Before enabling check that all replicas support new format.", 0) \ + DECLARE(UInt64, finished_mutations_to_keep, 100, "How many records about mutations that are done to keep. If zero, then keep all of them.", 0) \ + DECLARE(UInt64, min_merge_bytes_to_use_direct_io, 10ULL * 1024 * 1024 * 1024, "Minimal amount of bytes to enable O_DIRECT in merge (0 - disabled).", 0) \ + DECLARE(UInt64, index_granularity_bytes, 10 * 1024 * 1024, "Approximate amount of bytes in single granule (0 - disabled).", 0) \ + DECLARE(UInt64, min_index_granularity_bytes, 1024, "Minimum amount of bytes in single granule.", 1024) \ + DECLARE(Int64, merge_with_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with delete TTL can be repeated.", 0) \ + DECLARE(Int64, merge_with_recompression_ttl_timeout, 3600 * 4, "Minimal time in seconds, when merge with recompression TTL can be repeated.", 0) \ + DECLARE(Bool, ttl_only_drop_parts, false, "Only drop altogether the expired parts and not partially prune them.", 0) \ + DECLARE(Bool, materialize_ttl_recalculate_only, false, "Only recalculate ttl info when MATERIALIZE TTL", 0) \ + DECLARE(Bool, enable_mixed_granularity_parts, true, "Enable parts with adaptive and non adaptive granularity", 0) \ + DECLARE(UInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.", 0) \ + DECLARE(UInt64, zero_copy_concurrent_part_removal_max_split_times, 5, "Max recursion depth for splitting independent Outdated parts ranges into smaller subranges (highly not recommended to change)", 0) \ + DECLARE(Float, zero_copy_concurrent_part_removal_max_postpone_ratio, static_cast(0.05), "Max percentage of top level parts to postpone removal in order to get smaller independent ranges (highly not recommended to change)", 0) \ + DECLARE(String, storage_policy, "default", "Name of storage disk policy", 0) \ + DECLARE(String, disk, "", "Name of storage disk. Can be specified instead of storage policy.", 0) \ + DECLARE(Bool, allow_nullable_key, false, "Allow Nullable types as primary keys.", 0) \ + DECLARE(Bool, remove_empty_parts, true, "Remove empty parts after they were pruned by TTL, mutation, or collapsing merge algorithm.", 0) \ + DECLARE(Bool, assign_part_uuids, false, "Generate UUIDs for parts. Before enabling check that all replicas support new format.", 0) \ + DECLARE(Int64, max_partitions_to_read, -1, "Limit the max number of partitions that can be accessed in one query. <= 0 means unlimited. This setting is the default that can be overridden by the query-level setting with the same name.", 0) \ + DECLARE(UInt64, max_concurrent_queries, 0, "Max number of concurrently executed queries related to the MergeTree table (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ + DECLARE(UInt64, min_marks_to_honor_max_concurrent_queries, 0, "Minimal number of marks to honor the MergeTree-level's max_concurrent_queries (0 - disabled). Queries will still be limited by other max_concurrent_queries settings.", 0) \ + DECLARE(UInt64, min_bytes_to_rebalance_partition_over_jbod, 0, "Minimal amount of bytes to enable part rebalance over JBOD array (0 - disabled).", 0) \ + DECLARE(Bool, check_sample_column_is_correct, true, "Check columns or columns by hash for sampling are unsigned integer.", 0) \ + DECLARE(Bool, allow_vertical_merges_from_compact_to_wide_parts, true, "Allows vertical merges from compact to wide parts. This settings must have the same value on all replicas", 0) \ + DECLARE(Bool, enable_the_endpoint_id_with_zookeeper_name_prefix, false, "Enable the endpoint id with zookeeper name prefix for the replicated merge tree table", 0) \ + DECLARE(UInt64, zero_copy_merge_mutation_min_parts_size_sleep_before_lock, 1ULL * 1024 * 1024 * 1024, "If zero copy replication is enabled sleep random amount of time before trying to lock depending on parts size for merge or mutation", 0) \ + DECLARE(Bool, allow_floating_point_partition_key, false, "Allow floating point as partition key", 0) \ + DECLARE(UInt64, sleep_before_loading_outdated_parts_ms, 0, "For testing. Do not change it.", 0) \ + DECLARE(Bool, always_use_copy_instead_of_hardlinks, false, "Always copy data instead of hardlinking during mutations/replaces/detaches and so on.", 0) \ + DECLARE(Bool, disable_freeze_partition_for_zero_copy_replication, true, "Disable FREEZE PARTITION query for zero copy replication.", 0) \ + DECLARE(Bool, disable_detach_partition_for_zero_copy_replication, true, "Disable DETACH PARTITION query for zero copy replication.", 0) \ + DECLARE(Bool, disable_fetch_partition_for_zero_copy_replication, true, "Disable FETCH PARTITION query for zero copy replication.", 0) \ + DECLARE(Bool, enable_block_number_column, false, "Enable persisting column _block_number for each row.", 0) ALIAS(allow_experimental_block_number_column) \ + DECLARE(Bool, enable_block_offset_column, false, "Enable persisting column _block_offset for each row.", 0) \ \ /** Experimental/work in progress feature. Unsafe for production. */ \ - M(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", 0) \ - M(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", 0) \ - M(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", 0) \ - M(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", 0) \ - M(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", 0) \ - M(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", 0) \ - M(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", 0) \ - M(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", 0) \ + DECLARE(UInt64, part_moves_between_shards_enable, 0, "Experimental/Incomplete feature to move parts between shards. Does not take into account sharding expressions.", EXPERIMENTAL) \ + DECLARE(UInt64, part_moves_between_shards_delay_seconds, 30, "Time to wait before/after moving parts between shards.", EXPERIMENTAL) \ + DECLARE(Bool, allow_remote_fs_zero_copy_replication, false, "Don't use this setting in production, because it is not ready.", BETA) \ + DECLARE(String, remote_fs_zero_copy_zookeeper_path, "/clickhouse/zero_copy", "ZooKeeper path for zero-copy table-independent info.", EXPERIMENTAL) \ + DECLARE(Bool, remote_fs_zero_copy_path_compatible_mode, false, "Run zero-copy in compatible mode during conversion process.", EXPERIMENTAL) \ + DECLARE(Bool, cache_populated_by_fetch, false, "Only available in ClickHouse Cloud", EXPERIMENTAL) \ + DECLARE(Bool, force_read_through_cache_for_merges, false, "Force read-through filesystem cache for merges", EXPERIMENTAL) \ + DECLARE(Bool, allow_experimental_replacing_merge_with_cleanup, false, "Allow experimental CLEANUP merges for ReplacingMergeTree with is_deleted column.", EXPERIMENTAL) \ \ /** Compress marks and primary key. */ \ - M(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ - M(Bool, compress_primary_key, true, "Primary key support compression, reduce primary key file size and speed up network transmission.", 0) \ - M(String, marks_compression_codec, "ZSTD(3)", "Compression encoding used by marks, marks are small enough and cached, so the default compression is ZSTD(3).", 0) \ - M(String, primary_key_compression_codec, "ZSTD(3)", "Compression encoding used by primary, primary key is small enough and cached, so the default compression is ZSTD(3).", 0) \ - M(UInt64, marks_compress_block_size, 65536, "Mark compress block size, the actual size of the block to compress.", 0) \ - M(UInt64, primary_key_compress_block_size, 65536, "Primary compress block size, the actual size of the block to compress.", 0) \ - M(Bool, primary_key_lazy_load, true, "Load primary key in memory on first use instead of on table initialization. This can save memory in the presence of a large number of tables.", 0) \ - M(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ + DECLARE(Bool, compress_marks, true, "Marks support compression, reduce mark file size and speed up network transmission.", 0) \ + DECLARE(Bool, compress_primary_key, true, "Primary key support compression, reduce primary key file size and speed up network transmission.", 0) \ + DECLARE(String, marks_compression_codec, "ZSTD(3)", "Compression encoding used by marks, marks are small enough and cached, so the default compression is ZSTD(3).", 0) \ + DECLARE(String, primary_key_compression_codec, "ZSTD(3)", "Compression encoding used by primary, primary key is small enough and cached, so the default compression is ZSTD(3).", 0) \ + DECLARE(UInt64, marks_compress_block_size, 65536, "Mark compress block size, the actual size of the block to compress.", 0) \ + DECLARE(UInt64, primary_key_compress_block_size, 65536, "Primary compress block size, the actual size of the block to compress.", 0) \ + DECLARE(Bool, primary_key_lazy_load, true, "Load primary key in memory on first use instead of on table initialization. This can save memory in the presence of a large number of tables.", 0) \ + DECLARE(Float, primary_key_ratio_of_unique_prefix_values_to_skip_suffix_columns, 0.9f, "If the value of a column of the primary key in data part changes at least in this ratio of times, skip loading next columns in memory. This allows to save memory usage by not loading useless columns of the primary key.", 0) \ + DECLARE(Bool, prewarm_mark_cache, false, "If true mark cache will be prewarmed by saving marks to mark cache on inserts, merges, fetches and on startup of server", 0) \ + DECLARE(String, columns_to_prewarm_mark_cache, "", "List of columns to prewarm mark cache for (if enabled). Empty means all columns", 0) \ /** Projection settings. */ \ - M(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ - M(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts, or rebuild the projections.", 0) \ - M(DeduplicateMergeProjectionMode, deduplicate_merge_projection_mode, DeduplicateMergeProjectionMode::THROW, "Whether to allow create projection for the table with non-classic MergeTree. Ignore option is purely for compatibility which might result in incorrect answer. Otherwise, if allowed, what is the action when merge, drop or rebuild.", 0) \ + DECLARE(UInt64, max_projections, 25, "The maximum number of merge tree projections.", 0) \ + DECLARE(LightweightMutationProjectionMode, lightweight_mutation_projection_mode, LightweightMutationProjectionMode::THROW, "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop projections of this table's relevant parts, or rebuild the projections.", 0) \ + DECLARE(DeduplicateMergeProjectionMode, deduplicate_merge_projection_mode, DeduplicateMergeProjectionMode::THROW, "Whether to allow create projection for the table with non-classic MergeTree. Ignore option is purely for compatibility which might result in incorrect answer. Otherwise, if allowed, what is the action when merge, drop or rebuild.", 0) \ #define MAKE_OBSOLETE_MERGE_TREE_SETTING(M, TYPE, NAME, DEFAULT) \ - M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", BaseSettingsHelpers::Flags::OBSOLETE) + M(TYPE, NAME, DEFAULT, "Obsolete setting, does nothing.", SettingsTierType::OBSOLETE) #define OBSOLETE_MERGE_TREE_SETTINGS(M, ALIAS) \ /** Obsolete settings that do nothing but left for compatibility reasons. */ \ @@ -278,8 +281,9 @@ namespace ErrorCodes MERGE_TREE_SETTINGS(M, ALIAS) \ OBSOLETE_MERGE_TREE_SETTINGS(M, ALIAS) -DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) +// clang-format on +DECLARE_SETTINGS_TRAITS(MergeTreeSettingsTraits, LIST_OF_MERGE_TREE_SETTINGS) /** Settings for the MergeTree family of engines. * Could be loaded from config or from a CREATE TABLE query (SETTINGS clause). @@ -333,7 +337,7 @@ void MergeTreeSettingsImpl::loadFromQuery(ASTStorage & storage_def, ContextPtr c else if (name == "storage_policy") found_storage_policy_setting = true; - if (found_disk_setting && found_storage_policy_setting) + if (!is_attach && found_disk_setting && found_storage_policy_setting) { throw Exception( ErrorCodes::BAD_ARGUMENTS, @@ -489,8 +493,7 @@ void MergeTreeColumnSettings::validate(const SettingsChanges & changes) } } -#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ - MergeTreeSettings ## TYPE NAME = & MergeTreeSettings ## Impl :: NAME; +#define INITIALIZE_SETTING_EXTERN(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) MergeTreeSettings##TYPE NAME = &MergeTreeSettingsImpl ::NAME; namespace MergeTreeSetting { @@ -514,18 +517,7 @@ MergeTreeSettings::MergeTreeSettings(MergeTreeSettings && settings) noexcept MergeTreeSettings::~MergeTreeSettings() = default; -#define IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR(CLASS_NAME, TYPE) \ - const SettingField##TYPE & MergeTreeSettings::operator[](CLASS_NAME##TYPE t) const \ - { \ - return impl.get()->*t; \ - } \ -SettingField##TYPE & MergeTreeSettings::operator[](CLASS_NAME##TYPE t) \ - { \ - return impl.get()->*t; \ - } - MERGETREE_SETTINGS_SUPPORTED_TYPES(MergeTreeSettings, IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR) -#undef IMPLEMENT_SETTING_SUBSCRIPT_OPERATOR bool MergeTreeSettings::has(std::string_view name) const { @@ -662,7 +654,8 @@ void MergeTreeSettings::dumpToSystemMergeTreeSettingsColumns(MutableColumnsAndCo res_columns[5]->insert(max); res_columns[6]->insert(writability == SettingConstraintWritability::CONST); res_columns[7]->insert(setting.getTypeName()); - res_columns[8]->insert(setting.isObsolete()); + res_columns[8]->insert(setting.getTier() == SettingsTierType::OBSOLETE); + res_columns[9]->insert(setting.getTier()); } } diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 1e42f16736d..99852309c77 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -94,7 +94,7 @@ void MergeTreeSink::consume(Chunk & chunk) DelayedPartitions partitions; const Settings & settings = context->getSettingsRef(); - size_t streams = 0; + size_t total_streams = 0; bool support_parallel_write = false; auto token_info = chunk.getChunkInfos().get(); @@ -153,16 +153,18 @@ void MergeTreeSink::consume(Chunk & chunk) max_insert_delayed_streams_for_parallel_write = 0; /// In case of too much columns/parts in block, flush explicitly. - streams += temp_part.streams.size(); + size_t current_streams = 0; + for (const auto & stream : temp_part.streams) + current_streams += stream.stream->getNumberOfOpenStreams(); - if (streams > max_insert_delayed_streams_for_parallel_write) + if (total_streams + current_streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(); delayed_chunk = std::make_unique(); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(); - streams = 0; + total_streams = 0; support_parallel_write = false; partitions = DelayedPartitions{}; } @@ -174,6 +176,8 @@ void MergeTreeSink::consume(Chunk & chunk) .block_dedup_token = block_dedup_token, .part_counters = std::move(part_counters), }); + + total_streams += current_streams; } if (need_to_define_dedup_token) @@ -243,6 +247,15 @@ void MergeTreeSink::finishDelayedChunk() /// Part can be deduplicated, so increment counters and add to part log only if it's really added if (added) { + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + { + for (const auto & stream : partition.temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*part, marks, mark_cache); + } + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot)); StorageMergeTree::incrementInsertedPartsProfileEvent(part->getType()); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 4ee68580d3f..77c34aae30a 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -25,6 +25,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( CompressionCodecPtr default_codec_, TransactionID tid, bool reset_columns_, + bool save_marks_in_cache, bool blocks_are_granules_size, const WriteSettings & write_settings_, const MergeTreeIndexGranularity & computed_index_granularity) @@ -39,6 +40,7 @@ MergedBlockOutputStream::MergedBlockOutputStream( storage_settings, data_part->index_granularity_info.mark_type.adaptive, /* rewrite_primary_key = */ true, + save_marks_in_cache, blocks_are_granules_size); /// TODO: looks like isStoredOnDisk() is always true for MergeTreeDataPart diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.h b/src/Storages/MergeTree/MergedBlockOutputStream.h index e212fe5bb5a..060778866e0 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -24,6 +24,7 @@ public: CompressionCodecPtr default_codec_, TransactionID tid, bool reset_columns_ = false, + bool save_marks_in_cache = false, bool blocks_are_granules_size = false, const WriteSettings & write_settings = {}, const MergeTreeIndexGranularity & computed_index_granularity = {}); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index 05cd77dcd40..bed539dfe02 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -19,6 +19,7 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( const MergeTreeIndices & indices_to_recalc, const ColumnsStatistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_, + bool save_marks_in_cache, const MergeTreeIndexGranularity & index_granularity, const MergeTreeIndexGranularityInfo * index_granularity_info) : IMergedBlockOutputStream(data_part->storage.getSettings(), data_part->getDataPartStoragePtr(), metadata_snapshot_, columns_list_, /*reset_columns=*/ true) @@ -30,7 +31,9 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( data_part->storage.getContext()->getWriteSettings(), storage_settings, index_granularity_info ? index_granularity_info->mark_type.adaptive : data_part->storage.canUseAdaptiveGranularity(), - /* rewrite_primary_key = */ false); + /* rewrite_primary_key = */ false, + save_marks_in_cache, + /* blocks_are_granules_size = */ false); writer = createMergeTreeDataPartWriter( data_part->getType(), diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index e837a62743e..f6bf9e37a58 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -22,6 +22,7 @@ public: const MergeTreeIndices & indices_to_recalc_, const ColumnsStatistics & stats_to_recalc_, WrittenOffsetColumns * offset_columns_ = nullptr, + bool save_marks_in_cache = false, const MergeTreeIndexGranularity & index_granularity = {}, const MergeTreeIndexGranularityInfo * index_granularity_info_ = nullptr); diff --git a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp index 54215cd2dba..6716144ce81 100644 --- a/src/Storages/MergeTree/MutateFromLogEntryTask.cpp +++ b/src/Storages/MergeTree/MutateFromLogEntryTask.cpp @@ -226,6 +226,10 @@ ReplicatedMergeMutateTaskBase::PrepareResult MutateFromLogEntryTask::prepare() future_mutated_part, task_context); + storage.writePartLog( + PartLogElement::MUTATE_PART_START, {}, 0, + entry.new_part_name, new_part, future_mutated_part->parts, merge_mutate_entry.get(), {}); + mutate_task = storage.merger_mutator.mutatePartToTemporaryPart( future_mutated_part, metadata_snapshot, commands, merge_mutate_entry.get(), entry.create_time, task_context, NO_TRANSACTION_PTR, reserved_space, table_lock_holder); diff --git a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp index 53aef36404e..fbc20b282ca 100644 --- a/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp +++ b/src/Storages/MergeTree/MutatePlainMergeTreeTask.cpp @@ -39,6 +39,10 @@ void MutatePlainMergeTreeTask::prepare() future_part, task_context); + storage.writePartLog( + PartLogElement::MUTATE_PART_START, {}, 0, + future_part->name, new_part, future_part->parts, merge_list_entry.get(), {}); + stopwatch = std::make_unique(); write_part_log = [this] (const ExecutionStatus & execution_status) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 2e7847fc99f..936df7b0275 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -1623,6 +1623,7 @@ private: ctx->compression_codec, ctx->txn ? ctx->txn->tid : Tx::PrehistoricTID, /*reset_columns=*/ true, + /*save_marks_in_cache=*/ false, /*blocks_are_granules_size=*/ false, ctx->context->getWriteSettings(), computed_granularity); @@ -1851,6 +1852,7 @@ private: std::vector(ctx->indices_to_recalc.begin(), ctx->indices_to_recalc.end()), ColumnsStatistics(ctx->stats_to_recalc.begin(), ctx->stats_to_recalc.end()), nullptr, + /*save_marks_in_cache=*/ false, ctx->source_part->index_granularity, &ctx->source_part->index_granularity_info ); @@ -2164,6 +2166,7 @@ bool MutateTask::prepare() context_for_reading->setSetting("apply_mutations_on_fly", false); /// Skip using large sets in KeyCondition context_for_reading->setSetting("use_index_for_in_with_subqueries_max_values", 100000); + context_for_reading->setSetting("use_concurrency_control", false); for (const auto & command : *ctx->commands) if (!canSkipMutationCommandForPart(ctx->source_part, command, context_for_reading)) @@ -2286,7 +2289,7 @@ bool MutateTask::prepare() String tmp_part_dir_name = prefix + ctx->future_part->name; ctx->temporary_directory_lock = ctx->data->getTemporaryPartDirectoryHolder(tmp_part_dir_name); - auto builder = ctx->data->getDataPartBuilder(ctx->future_part->name, single_disk_volume, tmp_part_dir_name); + auto builder = ctx->data->getDataPartBuilder(ctx->future_part->name, single_disk_volume, tmp_part_dir_name, getReadSettings()); builder.withPartFormat(ctx->future_part->part_format); builder.withPartInfo(ctx->future_part->part_info); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp index 22b8ccca151..c258048354e 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -20,7 +21,6 @@ namespace ErrorCodes { extern const int SUPPORT_IS_DISABLED; extern const int REPLICA_STATUS_CHANGED; - extern const int LOGICAL_ERROR; } ReplicatedMergeTreeAttachThread::ReplicatedMergeTreeAttachThread(StorageReplicatedMergeTree & storage_) @@ -123,67 +123,6 @@ void ReplicatedMergeTreeAttachThread::checkHasReplicaMetadataInZooKeeper(const z } } -Int32 ReplicatedMergeTreeAttachThread::fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper) -{ - const String & zookeeper_path = storage.zookeeper_path; - const String & replica_path = storage.replica_path; - const bool replica_readonly = storage.is_readonly; - - for (size_t i = 0; i != 2; ++i) - { - String replica_metadata_version_str; - const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version_str); - if (!replica_metadata_version_exists) - return -1; - - const Int32 metadata_version = parse(replica_metadata_version_str); - - if (metadata_version != 0 || replica_readonly) - { - /// No need to fix anything - return metadata_version; - } - - Coordination::Stat stat; - zookeeper->get(fs::path(zookeeper_path) / "metadata", &stat); - if (stat.version == 0) - { - /// No need to fix anything - return metadata_version; - } - - ReplicatedMergeTreeQueue & queue = storage.queue; - queue.pullLogsToQueue(zookeeper); - if (queue.getStatus().metadata_alters_in_queue != 0) - { - LOG_DEBUG(log, "No need to update metadata_version as there are ALTER_METADATA entries in the queue"); - return metadata_version; - } - - const Coordination::Requests ops = { - zkutil::makeSetRequest(fs::path(replica_path) / "metadata_version", std::to_string(stat.version), 0), - zkutil::makeCheckRequest(fs::path(zookeeper_path) / "metadata", stat.version), - }; - Coordination::Responses ops_responses; - const auto code = zookeeper->tryMulti(ops, ops_responses); - if (code == Coordination::Error::ZOK) - { - LOG_DEBUG(log, "Successfully set metadata_version to {}", stat.version); - return stat.version; - } - if (code != Coordination::Error::ZBADVERSION) - { - throw zkutil::KeeperException(code); - } - } - - /// Second attempt is only possible if metadata_version != 0 or metadata.version changed during the first attempt. - /// If metadata_version != 0, on second attempt we will return the new metadata_version. - /// If metadata.version changed, on second attempt we will either get metadata_version != 0 and return the new metadata_version or we will get metadata_alters_in_queue != 0 and return 0. - /// Either way, on second attempt this method should return. - throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fix replica metadata_version in ZooKeeper after two attempts"); -} - void ReplicatedMergeTreeAttachThread::runImpl() { storage.setZooKeeper(); @@ -227,33 +166,6 @@ void ReplicatedMergeTreeAttachThread::runImpl() /// Just in case it was not removed earlier due to connection loss zookeeper->tryRemove(replica_path + "/flags/force_restore_data"); - const Int32 replica_metadata_version = fixReplicaMetadataVersionIfNeeded(zookeeper); - const bool replica_metadata_version_exists = replica_metadata_version != -1; - if (replica_metadata_version_exists) - { - storage.setInMemoryMetadata(metadata_snapshot->withMetadataVersion(replica_metadata_version)); - } - else - { - /// Table was created before 20.4 and was never altered, - /// let's initialize replica metadata version from global metadata version. - Coordination::Stat table_metadata_version_stat; - zookeeper->get(zookeeper_path + "/metadata", &table_metadata_version_stat); - - Coordination::Requests ops; - ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/metadata", table_metadata_version_stat.version)); - ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(table_metadata_version_stat.version), zkutil::CreateMode::Persistent)); - - Coordination::Responses res; - auto code = zookeeper->tryMulti(ops, res); - - if (code == Coordination::Error::ZBADVERSION) - throw Exception(ErrorCodes::REPLICA_STATUS_CHANGED, "Failed to initialize metadata_version " - "because table was concurrently altered, will retry"); - - zkutil::KeeperMultiException::check(code, ops, res); - } - storage.checkTableStructure(replica_path, metadata_snapshot); storage.checkParts(skip_sanity_checks); diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h index bfc97442598..250a5ed34d1 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeAttachThread.h @@ -48,8 +48,6 @@ private: void runImpl(); void finalizeInitialization(); - - Int32 fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper); }; } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 6b1581645f8..b1564b58a6c 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -615,7 +615,7 @@ std::pair ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::Zo { std::lock_guard lock(pull_logs_to_queue_mutex); - if (reason != LOAD) + if (reason != LOAD && reason != FIX_METADATA_VERSION) { /// It's totally ok to load queue on readonly replica (that's what RestartingThread does on initialization). /// It's ok if replica became readonly due to connection loss after we got current zookeeper (in this case zookeeper must be expired). diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 9d3349663e2..6ec8818b0c6 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -334,6 +334,7 @@ public: UPDATE, MERGE_PREDICATE, SYNC, + FIX_METADATA_VERSION, OTHER, }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 9d3e26cdc8d..93124e634bd 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -29,6 +29,8 @@ namespace MergeTreeSetting namespace ErrorCodes { extern const int REPLICA_IS_ALREADY_ACTIVE; + extern const int REPLICA_STATUS_CHANGED; + extern const int LOGICAL_ERROR; } namespace FailPoints @@ -207,6 +209,36 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() throw; } + const Int32 replica_metadata_version = fixReplicaMetadataVersionIfNeeded(zookeeper); + const bool replica_metadata_version_exists = replica_metadata_version != -1; + if (replica_metadata_version_exists) + { + storage.setInMemoryMetadata(storage.getInMemoryMetadataPtr()->withMetadataVersion(replica_metadata_version)); + } + else + { + /// Table was created before 20.4 and was never altered, + /// let's initialize replica metadata version from global metadata version. + + const String & zookeeper_path = storage.zookeeper_path, & replica_path = storage.replica_path; + + Coordination::Stat table_metadata_version_stat; + zookeeper->get(zookeeper_path + "/metadata", &table_metadata_version_stat); + + Coordination::Requests ops; + ops.emplace_back(zkutil::makeCheckRequest(zookeeper_path + "/metadata", table_metadata_version_stat.version)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/metadata_version", toString(table_metadata_version_stat.version), zkutil::CreateMode::Persistent)); + + Coordination::Responses res; + auto code = zookeeper->tryMulti(ops, res); + + if (code == Coordination::Error::ZBADVERSION) + throw Exception(ErrorCodes::REPLICA_STATUS_CHANGED, "Failed to initialize metadata_version " + "because table was concurrently altered, will retry"); + + zkutil::KeeperMultiException::check(code, ops, res); + } + storage.queue.removeCurrentPartsFromMutations(); storage.last_queue_update_finish_time.store(time(nullptr)); @@ -424,4 +456,64 @@ void ReplicatedMergeTreeRestartingThread::setNotReadonly() storage.readonly_start_time.store(0, std::memory_order_relaxed); } + +Int32 ReplicatedMergeTreeRestartingThread::fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper) +{ + const String & zookeeper_path = storage.zookeeper_path; + const String & replica_path = storage.replica_path; + + const size_t num_attempts = 2; + for (size_t attempt = 0; attempt != num_attempts; ++attempt) + { + String replica_metadata_version_str; + Coordination::Stat replica_stat; + const bool replica_metadata_version_exists = zookeeper->tryGet(replica_path + "/metadata_version", replica_metadata_version_str, &replica_stat); + if (!replica_metadata_version_exists) + return -1; + + const Int32 metadata_version = parse(replica_metadata_version_str); + if (metadata_version != 0) + return metadata_version; + + Coordination::Stat table_stat; + zookeeper->get(fs::path(zookeeper_path) / "metadata", &table_stat); + if (table_stat.version == 0) + return metadata_version; + + ReplicatedMergeTreeQueue & queue = storage.queue; + queue.pullLogsToQueue(zookeeper, {}, ReplicatedMergeTreeQueue::FIX_METADATA_VERSION); + if (queue.getStatus().metadata_alters_in_queue != 0) + { + LOG_INFO(log, "Skipping updating metadata_version as there are ALTER_METADATA entries in the queue"); + return metadata_version; + } + + const Coordination::Requests ops = { + zkutil::makeSetRequest(fs::path(replica_path) / "metadata_version", std::to_string(table_stat.version), replica_stat.version), + zkutil::makeCheckRequest(fs::path(zookeeper_path) / "metadata", table_stat.version), + }; + Coordination::Responses ops_responses; + const Coordination::Error code = zookeeper->tryMulti(ops, ops_responses); + if (code == Coordination::Error::ZOK) + { + LOG_DEBUG(log, "Successfully set metadata_version to {}", table_stat.version); + return table_stat.version; + } + + if (code == Coordination::Error::ZBADVERSION) + { + LOG_WARNING(log, "Cannot fix metadata_version because either metadata.version or metadata_version.version changed, attempts left = {}", num_attempts - attempt - 1); + continue; + } + + throw zkutil::KeeperException(code); + } + + /// Second attempt is only possible if either metadata_version.version or metadata.version changed during the first attempt. + /// If metadata_version changed to non-zero value during the first attempt, on second attempt we will return the new metadata_version. + /// If metadata.version changed during first attempt, on second attempt we will either get metadata_version != 0 and return the new metadata_version or we will get metadata_alters_in_queue != 0 and return 0. + /// So either first or second attempt should return unless metadata_version was rewritten from 0 to 0 during the first attempt which is highly unlikely. + throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to fix replica metadata_version in ZooKeeper after two attempts"); +} + } diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index d719505ae5e..6f450dc1d40 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -68,6 +69,9 @@ private: /// Disable readonly mode for table void setNotReadonly(); + + /// Fix replica metadata_version if needed + Int32 fixReplicaMetadataVersionIfNeeded(zkutil::ZooKeeperPtr zookeeper); }; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 95469337f8a..f3ae6e77ac3 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -341,7 +341,7 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) using DelayedPartitions = std::vector; DelayedPartitions partitions; - size_t streams = 0; + size_t total_streams = 0; bool support_parallel_write = false; for (auto & current_block : part_blocks) @@ -418,15 +418,18 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) max_insert_delayed_streams_for_parallel_write = 0; /// In case of too much columns/parts in block, flush explicitly. - streams += temp_part.streams.size(); - if (streams > max_insert_delayed_streams_for_parallel_write) + size_t current_streams = 0; + for (const auto & stream : temp_part.streams) + current_streams += stream.stream->getNumberOfOpenStreams(); + + if (total_streams + current_streams > max_insert_delayed_streams_for_parallel_write) { finishDelayedChunk(zookeeper); delayed_chunk = std::make_unique::DelayedChunk>(replicas_num); delayed_chunk->partitions = std::move(partitions); finishDelayedChunk(zookeeper); - streams = 0; + total_streams = 0; support_parallel_write = false; partitions = DelayedPartitions{}; } @@ -447,6 +450,8 @@ void ReplicatedMergeTreeSinkImpl::consume(Chunk & chunk) std::move(unmerged_block), std::move(part_counters) /// profile_events_scope must be reset here. )); + + total_streams += current_streams; } if (need_to_define_dedup_token) @@ -481,6 +486,17 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithF /// Set a special error code if the block is duplicate int error = (deduplicate && deduplicated) ? ErrorCodes::INSERT_WAS_DEDUPLICATED : 0; + auto * mark_cache = storage.getContext()->getMarkCache().get(); + + if (!error && mark_cache) + { + for (const auto & stream : partition.temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*part, marks, mark_cache); + } + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart(storage.getContext(), PartLog::PartLogEntry(part, partition.elapsed_ns, counters_snapshot), ExecutionStatus(error)); StorageReplicatedMergeTree::incrementInsertedPartsProfileEvent(part->getType()); @@ -521,8 +537,18 @@ void ReplicatedMergeTreeSinkImpl::finishDelayedChunk(const ZooKeeperWithFa { partition.temp_part.finalize(); auto conflict_block_ids = commitPart(zookeeper, partition.temp_part.part, partition.block_id, delayed_chunk->replicas_num).first; + if (conflict_block_ids.empty()) { + if (auto * mark_cache = storage.getContext()->getMarkCache().get()) + { + for (const auto & stream : partition.temp_part.streams) + { + auto marks = stream.stream->releaseCachedMarks(); + addMarksToCache(*partition.temp_part.part, marks, mark_cache); + } + } + auto counters_snapshot = std::make_shared(partition.part_counters.getPartiallyAtomicSnapshot()); PartLog::addNewPart( storage.getContext(), diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 2a1ddf32431..34e699bcef7 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -135,7 +135,6 @@ bool isRetryableException(std::exception_ptr exception_ptr) } } - static IMergeTreeDataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, const IDataPartStorage & data_part_storage, @@ -422,6 +421,7 @@ IMergeTreeDataPart::Checksums checkDataPart( } ReadSettings read_settings; + read_settings.read_through_distributed_cache = false; read_settings.enable_filesystem_cache = false; read_settings.enable_filesystem_cache_log = false; read_settings.enable_filesystem_read_prefetches_log = false; From 863887cca5674088ebe15a95d07b2ee0aebf4597 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:50:47 +0100 Subject: [PATCH 43/91] Reset WriteSettings to master --- src/IO/WriteSettings.h | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/IO/WriteSettings.h b/src/IO/WriteSettings.h index 4eeb01b5acc..94410f787f0 100644 --- a/src/IO/WriteSettings.h +++ b/src/IO/WriteSettings.h @@ -4,7 +4,6 @@ #include #include - namespace DB { @@ -29,8 +28,6 @@ struct WriteSettings bool use_adaptive_write_buffer = false; size_t adaptive_write_buffer_initial_size = 16 * 1024; - size_t max_compression_threads = 1; - bool write_through_distributed_cache = false; DistributedCacheSettings distributed_cache_settings; From c78272871f26d541800bb783d886325efa3c3ee7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:51:32 +0100 Subject: [PATCH 44/91] Rollback some changes --- .../0_stateless/03254_parallel_compression.sql | 11 ----------- 1 file changed, 11 deletions(-) delete mode 100644 tests/queries/0_stateless/03254_parallel_compression.sql diff --git a/tests/queries/0_stateless/03254_parallel_compression.sql b/tests/queries/0_stateless/03254_parallel_compression.sql deleted file mode 100644 index a17deed7d8c..00000000000 --- a/tests/queries/0_stateless/03254_parallel_compression.sql +++ /dev/null @@ -1,11 +0,0 @@ -DROP TABLE IF EXISTS test2; - -CREATE TABLE test2 -( - k UInt64 -) ENGINE = MergeTree ORDER BY k SETTINGS min_compress_block_size = 10240, min_bytes_for_wide_part = 1, max_compression_threads = 64; - -INSERT INTO test2 SELECT number FROM numbers(20000); -SELECT sum(k) = (9999 * 10000 / 2 + 10000 * 9999) FROM test2 WHERE k > 10000; - -DROP TABLE test2; From 3fa72482a75551c3132fa2f7ee8770d4f6862f98 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 8 Nov 2024 00:56:38 +0100 Subject: [PATCH 45/91] Revert some changes --- src/Compression/ParallelCompressedWriteBuffer.cpp | 4 ---- src/Compression/ParallelCompressedWriteBuffer.h | 10 ---------- 2 files changed, 14 deletions(-) diff --git a/src/Compression/ParallelCompressedWriteBuffer.cpp b/src/Compression/ParallelCompressedWriteBuffer.cpp index 303e1ece68a..3831d07e91a 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.cpp +++ b/src/Compression/ParallelCompressedWriteBuffer.cpp @@ -44,8 +44,6 @@ void ParallelCompressedWriteBuffer::nextImpl() /// The buffer will be compressed and processed in the thread. current_buffer->busy = true; current_buffer->sequence_num = current_sequence_num; - current_buffer->out_callback = callback; - callback = {}; ++current_sequence_num; current_buffer->uncompressed_size = offset(); pool.scheduleOrThrowOnError([this, my_current_buffer = current_buffer, thread_group = CurrentThread::getGroup()] @@ -155,8 +153,6 @@ void ParallelCompressedWriteBuffer::compress(Iterator buffer) } std::unique_lock lock(mutex); - if (buffer->out_callback) - buffer->out_callback(); buffer->busy = false; cond.notify_all(); } diff --git a/src/Compression/ParallelCompressedWriteBuffer.h b/src/Compression/ParallelCompressedWriteBuffer.h index 8c5f249b06c..38a3a083e19 100644 --- a/src/Compression/ParallelCompressedWriteBuffer.h +++ b/src/Compression/ParallelCompressedWriteBuffer.h @@ -31,13 +31,6 @@ public: ~ParallelCompressedWriteBuffer() override; - /// This function will be called once after compressing the next data and sending it to the out. - /// It can be used to fill information about marks. - void setCompletionCallback(std::function callback_) - { - callback = callback_; - } - private: void nextImpl() override; void finalizeImpl() override; @@ -61,15 +54,12 @@ private: BufferPair * previous = nullptr; size_t sequence_num = 0; bool busy = false; - std::function out_callback; }; std::mutex mutex; std::condition_variable cond; std::list buffers; - std::function callback; - using Iterator = std::list::iterator; Iterator current_buffer; size_t current_sequence_num = 0; From f24dca21a56f97ce2d422bc3e411868eca5c751c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Nov 2024 09:15:15 +0100 Subject: [PATCH 46/91] Implement CLICKHOUSE_RUN_AS_ROOT instead of preser UID/GID --- docker/keeper/entrypoint.sh | 25 +++++++++++++++++-------- docker/server/entrypoint.sh | 25 +++++++++++++++++-------- 2 files changed, 34 insertions(+), 16 deletions(-) diff --git a/docker/keeper/entrypoint.sh b/docker/keeper/entrypoint.sh index 92b91a0f8c3..31e4c8b63da 100644 --- a/docker/keeper/entrypoint.sh +++ b/docker/keeper/entrypoint.sh @@ -5,19 +5,28 @@ set -eo pipefail shopt -s nullglob DO_CHOWN=1 -if [ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" ]; then +if [[ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" || "${CLICKHOUSE_RUN_AS_ROOT:=0}" = "1" ]]; then DO_CHOWN=0 fi +# CLICKHOUSE_UID and CLICKHOUSE_GID are kept for backward compatibility, but deprecated +# One must use either "docker run --user" or CLICKHOUSE_RUN_AS_ROOT=1 to run the process as +# FIXME: Remove ALL CLICKHOUSE_UID CLICKHOUSE_GID before 25.3 +if [[ "${CLICKHOUSE_UID:-}" || "${CLICKHOUSE_GID:-}" ]]; then + echo 'WARNING: Support for CLICKHOUSE_UID/CLICKHOUSE_GID will be removed in a couple of releases.' >&2 + echo 'WARNING: Either use a proper "docker run --user=xxx:xxxx" argument instead of CLICKHOUSE_UID/CLICKHOUSE_GID' >&2 + echo 'WARNING: or set "CLICKHOUSE_RUN_AS_ROOT=1" ENV to run the clickhouse-server as root:root' >&2 +fi + # support `docker run --user=xxx:xxxx` -if [ "$(id -u)" = "0" ]; then - # CLICKHOUSE_UID and CLICKHOUSE_GID are kept for backward compatibility - if [[ "${CLICKHOUSE_UID:-}" || "${CLICKHOUSE_GID:-}" ]]; then - echo 'WARNING: consider using a proper "--user=xxx:xxxx" running argument instead of CLICKHOUSE_UID/CLICKHOUSE_GID' >&2 - echo 'Support for CLICKHOUSE_UID/CLICKHOUSE_GID will be removed in a couple of releases' >&2 +if [[ "$(id -u)" = "0" ]]; then + if [[ "$CLICKHOUSE_RUN_AS_ROOT" = 1 ]]; then + USER=0 + GROUP=0 + else + USER="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" + GROUP="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" fi - USER="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" - GROUP="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" if command -v gosu &> /dev/null; then gosu="gosu $USER:$GROUP" elif command -v su-exec &> /dev/null; then diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 5a91d54d32b..443bcd7a176 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -4,19 +4,28 @@ set -eo pipefail shopt -s nullglob DO_CHOWN=1 -if [ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" ]; then +if [[ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" || "${CLICKHOUSE_RUN_AS_ROOT:=0}" = "1" ]]; then DO_CHOWN=0 fi +# CLICKHOUSE_UID and CLICKHOUSE_GID are kept for backward compatibility, but deprecated +# One must use either "docker run --user" or CLICKHOUSE_RUN_AS_ROOT=1 to run the process as +# FIXME: Remove ALL CLICKHOUSE_UID CLICKHOUSE_GID before 25.3 +if [[ "${CLICKHOUSE_UID:-}" || "${CLICKHOUSE_GID:-}" ]]; then + echo 'WARNING: Support for CLICKHOUSE_UID/CLICKHOUSE_GID will be removed in a couple of releases.' >&2 + echo 'WARNING: Either use a proper "docker run --user=xxx:xxxx" argument instead of CLICKHOUSE_UID/CLICKHOUSE_GID' >&2 + echo 'WARNING: or set "CLICKHOUSE_RUN_AS_ROOT=1" ENV to run the clickhouse-server as root:root' >&2 +fi + # support `docker run --user=xxx:xxxx` -if [ "$(id -u)" = "0" ]; then - # CLICKHOUSE_UID and CLICKHOUSE_GID are kept for backward compatibility - if [[ "${CLICKHOUSE_UID:-}" || "${CLICKHOUSE_GID:-}" ]]; then - echo 'WARNING: consider using a proper "--user=xxx:xxxx" running argument instead of CLICKHOUSE_UID/CLICKHOUSE_GID' >&2 - echo 'Support for CLICKHOUSE_UID/CLICKHOUSE_GID will be removed in a couple of releases' >&2 +if [[ "$(id -u)" = "0" ]]; then + if [[ "$CLICKHOUSE_RUN_AS_ROOT" = 1 ]]; then + USER=0 + GROUP=0 + else + USER="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" + GROUP="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" fi - USER="${CLICKHOUSE_UID:-"$(id -u clickhouse)"}" - GROUP="${CLICKHOUSE_GID:-"$(id -g clickhouse)"}" else USER="$(id -u)" GROUP="$(id -g)" From 0f945cadc74aed12e6a1f05d7cde98aa02e369b7 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Fri, 8 Nov 2024 17:34:53 +0800 Subject: [PATCH 47/91] Update settings.md Remove duplicated `background_pool_size` description --- .../server-configuration-parameters/settings.md | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 02fa5a8ca58..c5f92ccdf68 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -131,16 +131,6 @@ Type: UInt64 Default: 8 -## background_pool_size - -Sets the number of threads performing background merges and mutations for tables with MergeTree engines. You can only increase the number of threads at runtime. To lower the number of threads you have to restart the server. By adjusting this setting, you manage CPU and disk load. Smaller pool size utilizes less CPU and disk resources, but background processes advance slower which might eventually impact query performance. - -Before changing it, please also take a look at related MergeTree settings, such as `number_of_free_entries_in_pool_to_lower_max_size_of_merge` and `number_of_free_entries_in_pool_to_execute_mutation`. - -Type: UInt64 - -Default: 16 - ## background_schedule_pool_size The maximum number of threads that will be used for constantly executing some lightweight periodic operations for replicated tables, Kafka streaming, and DNS cache updates. From d75a41f04ccb536b4083034b076e0f6a012e6d06 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Fri, 8 Nov 2024 11:24:28 +0100 Subject: [PATCH 48/91] init --- tests/queries/0_stateless/01287_max_execution_speed.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/01287_max_execution_speed.sql b/tests/queries/0_stateless/01287_max_execution_speed.sql index 0d132999481..89c3050a256 100644 --- a/tests/queries/0_stateless/01287_max_execution_speed.sql +++ b/tests/queries/0_stateless/01287_max_execution_speed.sql @@ -1,5 +1,8 @@ -- Tags: no-fasttest, no-debug, no-tsan, no-msan, no-asan +SET max_rows_to_read=0; +SET max_bytes_to_read=0; + SET min_execution_speed = 100000000000, timeout_before_checking_execution_speed = 0; SELECT count() FROM system.numbers; -- { serverError TOO_SLOW } SET min_execution_speed = 0; From cf1da69f93c4c8e982b89a73565c16642ab0f18f Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Nov 2024 11:44:02 +0100 Subject: [PATCH 49/91] Make keeper entrypoint less verbose, like the in the server --- docker/keeper/entrypoint.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/keeper/entrypoint.sh b/docker/keeper/entrypoint.sh index 31e4c8b63da..2b96e4dd655 100644 --- a/docker/keeper/entrypoint.sh +++ b/docker/keeper/entrypoint.sh @@ -1,6 +1,5 @@ #!/bin/bash -set +x set -eo pipefail shopt -s nullglob From 11f3568f5b661330e3fa94fb1515807dd73d7e22 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Nov 2024 11:45:12 +0100 Subject: [PATCH 50/91] First check the ROOT to assign the env --- docker/keeper/entrypoint.sh | 2 +- docker/server/entrypoint.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docker/keeper/entrypoint.sh b/docker/keeper/entrypoint.sh index 2b96e4dd655..934605b0b6f 100644 --- a/docker/keeper/entrypoint.sh +++ b/docker/keeper/entrypoint.sh @@ -4,7 +4,7 @@ set -eo pipefail shopt -s nullglob DO_CHOWN=1 -if [[ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" || "${CLICKHOUSE_RUN_AS_ROOT:=0}" = "1" ]]; then +if [[ "${CLICKHOUSE_RUN_AS_ROOT:=0}" = "1" || "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" ]]; then DO_CHOWN=0 fi diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index 443bcd7a176..2f87008f2e5 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -4,7 +4,7 @@ set -eo pipefail shopt -s nullglob DO_CHOWN=1 -if [[ "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" || "${CLICKHOUSE_RUN_AS_ROOT:=0}" = "1" ]]; then +if [[ "${CLICKHOUSE_RUN_AS_ROOT:=0}" = "1" || "${CLICKHOUSE_DO_NOT_CHOWN:-0}" = "1" ]]; then DO_CHOWN=0 fi From 87b9f5cb4ef65bd8c7313bd4f2563e41b974e951 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2024 12:24:29 +0100 Subject: [PATCH 51/91] Add min_parts_to_merge_at_once setting --- .../MergeTree/MergeSelectors/SimpleMergeSelector.cpp | 5 ++++- src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.h | 2 ++ src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp | 2 ++ src/Storages/MergeTree/MergeTreeSettings.cpp | 1 + 4 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.cpp b/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.cpp index c393349ef32..4f786215cbe 100644 --- a/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.cpp +++ b/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.cpp @@ -116,7 +116,7 @@ bool allow( double sum_size, double max_size, double min_age, - double range_size, + size_t range_size, double partition_size, double min_size_to_lower_base_log, double max_size_to_lower_base_log, @@ -125,6 +125,9 @@ bool allow( if (settings.min_age_to_force_merge && min_age >= settings.min_age_to_force_merge) return true; + if (settings.min_parts_to_merge_at_once && range_size < settings.min_parts_to_merge_at_once) + return false; + /// Map size to 0..1 using logarithmic scale /// Use log(1 + x) instead of log1p(x) because our sum_size is always integer. /// Also log1p seems to be slow and significantly affect performance of merges assignment. diff --git a/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.h b/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.h index 2d4129b8bf8..1e7676c6aed 100644 --- a/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.h +++ b/src/Storages/MergeTree/MergeSelectors/SimpleMergeSelector.h @@ -90,6 +90,8 @@ public: { /// Zero means unlimited. Can be overridden by the same merge tree setting. size_t max_parts_to_merge_at_once = 100; + /// Zero means no minimum. Can be overridden by the same merge tree setting. + size_t min_parts_to_merge_at_once = 0; /// Some sort of a maximum number of parts in partition. Can be overridden by the same merge tree setting. size_t parts_to_throw_insert = 3000; diff --git a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 37b6539755c..488f4b2390d 100644 --- a/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -82,6 +82,7 @@ namespace MergeTreeSetting extern const MergeTreeSettingsMergeSelectorAlgorithm merge_selector_algorithm; extern const MergeTreeSettingsBool merge_selector_enable_heuristic_to_remove_small_parts_at_right; extern const MergeTreeSettingsFloat merge_selector_base; + extern const MergeTreeSettingsUInt64 min_parts_to_merge_at_once; } namespace ErrorCodes @@ -566,6 +567,7 @@ SelectPartsDecision MergeTreeDataMergerMutator::selectPartsToMergeFromRanges( simple_merge_settings.max_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::max_parts_to_merge_at_once]; simple_merge_settings.enable_heuristic_to_remove_small_parts_at_right = (*data_settings)[MergeTreeSetting::merge_selector_enable_heuristic_to_remove_small_parts_at_right]; simple_merge_settings.base = (*data_settings)[MergeTreeSetting::merge_selector_base]; + simple_merge_settings.min_parts_to_merge_at_once = (*data_settings)[MergeTreeSetting::min_parts_to_merge_at_once]; if (!(*data_settings)[MergeTreeSetting::min_age_to_force_merge_on_partition_only]) simple_merge_settings.min_age_to_force_merge = (*data_settings)[MergeTreeSetting::min_age_to_force_merge_seconds]; diff --git a/src/Storages/MergeTree/MergeTreeSettings.cpp b/src/Storages/MergeTree/MergeTreeSettings.cpp index 33910d1048d..fcd4e05cf00 100644 --- a/src/Storages/MergeTree/MergeTreeSettings.cpp +++ b/src/Storages/MergeTree/MergeTreeSettings.cpp @@ -102,6 +102,7 @@ namespace ErrorCodes DECLARE(MergeSelectorAlgorithm, merge_selector_algorithm, MergeSelectorAlgorithm::SIMPLE, "The algorithm to select parts for merges assignment", EXPERIMENTAL) \ DECLARE(Bool, merge_selector_enable_heuristic_to_remove_small_parts_at_right, true, "Enable heuristic for selecting parts for merge which removes parts from right side of range, if their size is less than specified ratio (0.01) of sum_size. Works for Simple and StochasticSimple merge selectors", 0) \ DECLARE(Float, merge_selector_base, 5.0, "Affects write amplification of assigned merges (expert level setting, don't change if you don't understand what it is doing). Works for Simple and StochasticSimple merge selectors", 0) \ + DECLARE(UInt64, min_parts_to_merge_at_once, 0, "Minimal amount of data parts which merge selector can pick to merge at once (expert level setting, don't change if you don't understand what it is doing). 0 - disabled. Works for Simple and StochasticSimple merge selectors.", 0) \ \ /** Inserts settings. */ \ DECLARE(UInt64, parts_to_delay_insert, 1000, "If table contains at least that many active parts in single partition, artificially slow down insert into table. Disabled if set to 0", 0) \ From b6cad9c913b304052939cd100ba4e9d35b44c47a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2024 12:25:26 +0100 Subject: [PATCH 52/91] Add test --- ...03267_min_parts_to_merge_at_once.reference | 4 ++ .../03267_min_parts_to_merge_at_once.sh | 43 +++++++++++++++++++ 2 files changed, 47 insertions(+) create mode 100644 tests/queries/0_stateless/03267_min_parts_to_merge_at_once.reference create mode 100755 tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh diff --git a/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.reference b/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.reference new file mode 100644 index 00000000000..966a0980e59 --- /dev/null +++ b/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.reference @@ -0,0 +1,4 @@ +2 +3 +4 +1 diff --git a/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh b/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh new file mode 100755 index 00000000000..e069b57bf86 --- /dev/null +++ b/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t;" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE t (key UInt64) ENGINE = MergeTree() ORDER BY tuple() SETTINGS min_parts_to_merge_at_once=5, merge_selector_base=1" + +$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (1)" +$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (2);" + +# doesn't make test flaky +sleep 1 + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'" + +$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (3)" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'" + +$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (4)" + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'" + +$CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (5)" + +counter=0 retries=60 + +I=0 +while [[ $counter -lt $retries ]]; do + result=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'") + if [ "$result" -eq "1" ];then + break; + fi + sleep 0.5 + counter=$((counter + 1)) +done + +$CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'" + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS t" From 4c644a98f5985a540ee75dc5a1f5ae31be39cc15 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Fri, 8 Nov 2024 12:29:04 +0100 Subject: [PATCH 53/91] Fix broken 03247_ghdata_string_to_json_alter --- .../queries/0_stateless/03247_ghdata_string_to_json_alter.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh b/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh index 931d106120c..a2d1788cb5d 100755 --- a/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh +++ b/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh @@ -18,12 +18,12 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)" ${CLICKHOUSE_CLIENT} -q \ "SELECT data.repo.name, count() AS stars FROM ghdata \ - WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" + WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" --allow_suspicious_types_in_group_by=1, --allow_suspicious_types_in_order_by=1 ${CLICKHOUSE_CLIENT} --enable_analyzer=1 -q \ "SELECT data.payload.commits[].author.name AS name, count() AS c FROM ghdata \ ARRAY JOIN data.payload.commits[].author.name \ - GROUP BY name ORDER BY c DESC, name LIMIT 5" + GROUP BY name ORDER BY c DESC, name LIMIT 5" --allow_suspicious_types_in_group_by=1, --allow_suspicious_types_in_order_by=1 ${CLICKHOUSE_CLIENT} -q "SELECT max(data.payload.pull_request.assignees[].size0) FROM ghdata" From 1bd6b9df95792e8917e1da744a0d8e7d586949ed Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2024 12:47:48 +0100 Subject: [PATCH 54/91] Fix style check --- tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh b/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh index e069b57bf86..90b9d0339cf 100755 --- a/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh +++ b/tests/queries/0_stateless/03267_min_parts_to_merge_at_once.sh @@ -28,7 +28,6 @@ $CLICKHOUSE_CLIENT --query "INSERT INTO t VALUES (5)" counter=0 retries=60 -I=0 while [[ $counter -lt $retries ]]; do result=$($CLICKHOUSE_CLIENT --query "SELECT count() FROM system.parts WHERE active and database = currentDatabase() and table = 't'") if [ "$result" -eq "1" ];then From 10329cbbf2da51925e5a4580a8ba9faf3315cd02 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Nov 2024 12:07:30 +0100 Subject: [PATCH 55/91] Generate clickhouse/clickhouse-server README as in docker-library --- docker/server/README.md | 8 ++ docker/server/README.sh | 38 +++++ docker/server/README.src/README-short.txt | 1 + docker/server/README.src/content.md | 166 ++++++++++++++++++++++ docker/server/README.src/github-repo | 1 + docker/server/README.src/license.md | 1 + docker/server/README.src/logo.svg | 43 ++++++ docker/server/README.src/maintainer.md | 1 + docker/server/README.src/metadata.json | 7 + 9 files changed, 266 insertions(+) create mode 100755 docker/server/README.sh create mode 100644 docker/server/README.src/README-short.txt create mode 100644 docker/server/README.src/content.md create mode 100644 docker/server/README.src/github-repo create mode 100644 docker/server/README.src/license.md create mode 100644 docker/server/README.src/logo.svg create mode 100644 docker/server/README.src/maintainer.md create mode 100644 docker/server/README.src/metadata.json diff --git a/docker/server/README.md b/docker/server/README.md index 1dc636414ac..e8c60204c96 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -1,3 +1,11 @@ + + # ClickHouse Server Docker Image ## What is ClickHouse? diff --git a/docker/server/README.sh b/docker/server/README.sh new file mode 100755 index 00000000000..42fa72404d1 --- /dev/null +++ b/docker/server/README.sh @@ -0,0 +1,38 @@ +#!/usr/bin/env bash +set -ueo pipefail + +# A script to generate README.sh close to as it done in https://github.com/docker-library/docs + +WORKDIR=$(dirname "$0") +SCRIPT_NAME=$(basename "$0") +CONTENT=README.src/content.md +LICENSE=README.src/license.md +cd "$WORKDIR" + +R=README.md + +cat > "$R" < + +EOD + +cat "$CONTENT" >> "$R" + +cat >> "$R" <=2, Azure and GCP instances. Examples for unsupported devices are Raspberry Pi 4 (ARMv8.0-A) and Jetson AGX Xavier/Orin (ARMv8.2-A). +- Since the Clickhouse 24.11 Ubuntu images started using `ubuntu:22.04` as its base image. It requires docker version >= `20.10.10` containing [patch](https://github.com/moby/moby/commit/977283509f75303bc6612665a04abf76ff1d2468). As a workaround you could use `docker run [--privileged | --security-opt seccomp=unconfined]` instead, however that has security implications. + +## How to use this image + +### start server instance + +```bash +docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +``` + +By default, ClickHouse will be accessible only via the Docker network. See the [networking section below](#networking). + +By default, starting above server instance will be run as the `default` user without password. + +### connect to it from a native client + +```bash +docker run -it --rm --link some-clickhouse-server:clickhouse-server --entrypoint clickhouse-client clickhouse/clickhouse-server --host clickhouse-server +# OR +docker exec -it some-clickhouse-server clickhouse-client +``` + +More information about the [ClickHouse client](https://clickhouse.com/docs/en/interfaces/cli/). + +### connect to it using curl + +```bash +echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server curlimages/curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- +``` + +More information about the [ClickHouse HTTP Interface](https://clickhouse.com/docs/en/interfaces/http/). + +### stopping / removing the container + +```bash +docker stop some-clickhouse-server +docker rm some-clickhouse-server +``` + +### networking + +You can expose your ClickHouse running in docker by [mapping a particular port](https://docs.docker.com/config/containers/container-networking/) from inside the container using host ports: + +```bash +docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @- +``` + +`22.6.3.35` + +or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance): + +```bash +docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- +``` + +`22.6.3.35` + +### Volumes + +Typically you may want to mount the following folders inside your container to achieve persistency: + +- `/var/lib/clickhouse/` - main folder where ClickHouse stores the data +- `/var/log/clickhouse-server/` - logs + +```bash +docker run -d \ + -v $(realpath ./ch_data):/var/lib/clickhouse/ \ + -v $(realpath ./ch_logs):/var/log/clickhouse-server/ \ + --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +``` + +You may also want to mount: + +- `/etc/clickhouse-server/config.d/*.xml` - files with server configuration adjustments +- `/etc/clickhouse-server/users.d/*.xml` - files with user settings adjustments +- `/docker-entrypoint-initdb.d/` - folder with database initialization scripts (see below). + +### Linux capabilities + +ClickHouse has some advanced functionality, which requires enabling several [Linux capabilities](https://man7.org/linux/man-pages/man7/capabilities.7.html). + +They are optional and can be enabled using the following [docker command-line arguments](https://docs.docker.com/engine/reference/run/#runtime-privilege-and-linux-capabilities): + +```bash +docker run -d \ + --cap-add=SYS_NICE --cap-add=NET_ADMIN --cap-add=IPC_LOCK \ + --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +``` + +## Configuration + +The container exposes port 8123 for the [HTTP interface](https://clickhouse.com/docs/en/interfaces/http_interface/) and port 9000 for the [native client](https://clickhouse.com/docs/en/interfaces/tcp/). + +ClickHouse configuration is represented with a file "config.xml" ([documentation](https://clickhouse.com/docs/en/operations/configuration_files/)) + +### Start server instance with custom configuration + +```bash +docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /path/to/your/config.xml:/etc/clickhouse-server/config.xml clickhouse/clickhouse-server +``` + +### Start server as custom user + +```bash +# $(pwd)/data/clickhouse should exist and be owned by current user +docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +``` + +When you use the image with local directories mounted, you probably want to specify the user to maintain the proper file ownership. Use the `--user` argument and mount `/var/lib/clickhouse` and `/var/log/clickhouse-server` inside the container. Otherwise, the image will complain and not start. + +### Start server from root (useful in case of enabled user namespace) + +```bash +docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-server-userns -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +``` + +### How to create default database and user on starting + +Sometimes you may want to create a user (user named `default` is used by default) and database on a container start. You can do it using environment variables `CLICKHOUSE_DB`, `CLICKHOUSE_USER`, `CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT` and `CLICKHOUSE_PASSWORD`: + +```bash +docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT=1 -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp clickhouse/clickhouse-server +``` + +## How to extend this image + +To perform additional initialization in an image derived from this one, add one or more `*.sql`, `*.sql.gz`, or `*.sh` scripts under `/docker-entrypoint-initdb.d`. After the entrypoint calls `initdb`, it will run any `*.sql` files, run any executable `*.sh` scripts, and source any non-executable `*.sh` scripts found in that directory to do further initialization before starting the service. +Also, you can provide environment variables `CLICKHOUSE_USER` & `CLICKHOUSE_PASSWORD` that will be used for clickhouse-client during initialization. + +For example, to add an additional user and database, add the following to `/docker-entrypoint-initdb.d/init-db.sh`: + +```bash +#!/bin/bash +set -e + +clickhouse client -n <<-EOSQL + CREATE DATABASE docker; + CREATE TABLE docker.docker (x Int32) ENGINE = Log; +EOSQL +``` diff --git a/docker/server/README.src/github-repo b/docker/server/README.src/github-repo new file mode 100644 index 00000000000..dc2b6635325 --- /dev/null +++ b/docker/server/README.src/github-repo @@ -0,0 +1 @@ +https://github.com/ClickHouse/docker-library diff --git a/docker/server/README.src/license.md b/docker/server/README.src/license.md new file mode 100644 index 00000000000..6be024edcde --- /dev/null +++ b/docker/server/README.src/license.md @@ -0,0 +1 @@ +View [license information](https://github.com/ClickHouse/ClickHouse/blob/master/LICENSE) for the software contained in this image. diff --git a/docker/server/README.src/logo.svg b/docker/server/README.src/logo.svg new file mode 100644 index 00000000000..a50dd81a164 --- /dev/null +++ b/docker/server/README.src/logo.svg @@ -0,0 +1,43 @@ + + + + + + + + + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/docker/server/README.src/maintainer.md b/docker/server/README.src/maintainer.md new file mode 100644 index 00000000000..26c7db1a293 --- /dev/null +++ b/docker/server/README.src/maintainer.md @@ -0,0 +1 @@ +[ClickHouse Inc.](%%GITHUB-REPO%%) diff --git a/docker/server/README.src/metadata.json b/docker/server/README.src/metadata.json new file mode 100644 index 00000000000..3d3937b21fb --- /dev/null +++ b/docker/server/README.src/metadata.json @@ -0,0 +1,7 @@ +{ + "hub": { + "categories": [ + "databases-and-storage" + ] + } +} From aa15b912df09bb95e400a50aa007266948f75697 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Nov 2024 12:58:50 +0100 Subject: [PATCH 56/91] Apply review comments from docker-library/docs --- docker/server/README.md | 20 +++++++++------- docker/server/README.src/content.md | 36 ++++++++++++++++------------- 2 files changed, 32 insertions(+), 24 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index e8c60204c96..7403d5b0b2a 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -16,6 +16,7 @@ ClickHouse works 100-1000x faster than traditional database management systems, For more information and documentation see https://clickhouse.com/. + ## Versions - The `latest` tag points to the latest release of the latest stable branch. @@ -24,6 +25,7 @@ For more information and documentation see https://clickhouse.com/. - The tag `head` is built from the latest commit to the default branch. - Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`. + ### Compatibility - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. @@ -38,7 +40,7 @@ For more information and documentation see https://clickhouse.com/. docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server ``` -By default, ClickHouse will be accessible only via the Docker network. See the [networking section below](#networking). +By default, ClickHouse will be accessible only via the Docker network. See the **networking** section below. By default, starting above server instance will be run as the `default` user without password. @@ -55,7 +57,7 @@ More information about the [ClickHouse client](https://clickhouse.com/docs/en/in ### connect to it using curl ```bash -echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server curlimages/curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- +echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server buildpack-deps:curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- ``` More information about the [ClickHouse HTTP Interface](https://clickhouse.com/docs/en/interfaces/http/). @@ -78,7 +80,7 @@ echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @- `22.6.3.35` -or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance): +Or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance): ```bash docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server @@ -96,8 +98,8 @@ Typically you may want to mount the following folders inside your container to a ```bash docker run -d \ - -v $(realpath ./ch_data):/var/lib/clickhouse/ \ - -v $(realpath ./ch_logs):/var/log/clickhouse-server/ \ + -v "$PWD/ch_data:/var/lib/clickhouse/" \ + -v "$PWD/ch_logs:/var/log/clickhouse-server/" \ --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server ``` @@ -119,6 +121,8 @@ docker run -d \ --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server ``` +Read more in [knowledge base](https://clickhouse.com/docs/knowledgebase/configure_cap_ipc_lock_and_cap_sys_nice_in_docker). + ## Configuration The container exposes port 8123 for the [HTTP interface](https://clickhouse.com/docs/en/interfaces/http_interface/) and port 9000 for the [native client](https://clickhouse.com/docs/en/interfaces/tcp/). @@ -134,8 +138,8 @@ docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /pa ### Start server as custom user ```bash -# $(pwd)/data/clickhouse should exist and be owned by current user -docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +# $PWD/data/clickhouse should exist and be owned by current user +docker run --rm --user "${UID}:${GID}" --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$PWD/logs/clickhouse:/var/log/clickhouse-server" -v "$PWD/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server ``` When you use the image with local directories mounted, you probably want to specify the user to maintain the proper file ownership. Use the `--user` argument and mount `/var/lib/clickhouse` and `/var/log/clickhouse-server` inside the container. Otherwise, the image will complain and not start. @@ -143,7 +147,7 @@ When you use the image with local directories mounted, you probably want to spec ### Start server from root (useful in case of enabled user namespace) ```bash -docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-server-userns -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +docker run --rm -e CLICKHOUSE_RUN_AS_ROOT=1 --name clickhouse-server-userns -v "$PWD/logs/clickhouse:/var/log/clickhouse-server" -v "$PWD/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server ``` ### How to create default database and user on starting diff --git a/docker/server/README.src/content.md b/docker/server/README.src/content.md index e790de41236..bfc1a271546 100644 --- a/docker/server/README.src/content.md +++ b/docker/server/README.src/content.md @@ -10,6 +10,7 @@ ClickHouse works 100-1000x faster than traditional database management systems, For more information and documentation see https://clickhouse.com/. + ## Versions - The `latest` tag points to the latest release of the latest stable branch. @@ -18,6 +19,7 @@ For more information and documentation see https://clickhouse.com/. - The tag `head` is built from the latest commit to the default branch. - Each tag has optional `-alpine` suffix to reflect that it's built on top of `alpine`. + ### Compatibility - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. @@ -29,17 +31,17 @@ For more information and documentation see https://clickhouse.com/. ### start server instance ```bash -docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 %%IMAGE%% ``` -By default, ClickHouse will be accessible only via the Docker network. See the [networking section below](#networking). +By default, ClickHouse will be accessible only via the Docker network. See the **networking** section below. By default, starting above server instance will be run as the `default` user without password. ### connect to it from a native client ```bash -docker run -it --rm --link some-clickhouse-server:clickhouse-server --entrypoint clickhouse-client clickhouse/clickhouse-server --host clickhouse-server +docker run -it --rm --link some-clickhouse-server:clickhouse-server --entrypoint clickhouse-client %%IMAGE%% --host clickhouse-server # OR docker exec -it some-clickhouse-server clickhouse-client ``` @@ -49,7 +51,7 @@ More information about the [ClickHouse client](https://clickhouse.com/docs/en/in ### connect to it using curl ```bash -echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server curlimages/curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- +echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server buildpack-deps:curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- ``` More information about the [ClickHouse HTTP Interface](https://clickhouse.com/docs/en/interfaces/http/). @@ -66,16 +68,16 @@ docker rm some-clickhouse-server You can expose your ClickHouse running in docker by [mapping a particular port](https://docs.docker.com/config/containers/container-networking/) from inside the container using host ports: ```bash -docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +docker run -d -p 18123:8123 -p19000:9000 --name some-clickhouse-server --ulimit nofile=262144:262144 %%IMAGE%% echo 'SELECT version()' | curl 'http://localhost:18123/' --data-binary @- ``` `22.6.3.35` -or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance): +Or by allowing the container to use [host ports directly](https://docs.docker.com/network/host/) using `--network=host` (also allows achieving better network performance): ```bash -docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server +docker run -d --network=host --name some-clickhouse-server --ulimit nofile=262144:262144 %%IMAGE%% echo 'SELECT version()' | curl 'http://localhost:8123/' --data-binary @- ``` @@ -90,9 +92,9 @@ Typically you may want to mount the following folders inside your container to a ```bash docker run -d \ - -v $(realpath ./ch_data):/var/lib/clickhouse/ \ - -v $(realpath ./ch_logs):/var/log/clickhouse-server/ \ - --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server + -v "$PWD/ch_data:/var/lib/clickhouse/" \ + -v "$PWD/ch_logs:/var/log/clickhouse-server/" \ + --name some-clickhouse-server --ulimit nofile=262144:262144 %%IMAGE%% ``` You may also want to mount: @@ -110,9 +112,11 @@ They are optional and can be enabled using the following [docker command-line ar ```bash docker run -d \ --cap-add=SYS_NICE --cap-add=NET_ADMIN --cap-add=IPC_LOCK \ - --name some-clickhouse-server --ulimit nofile=262144:262144 clickhouse/clickhouse-server + --name some-clickhouse-server --ulimit nofile=262144:262144 %%IMAGE%% ``` +Read more in [knowledge base](https://clickhouse.com/docs/knowledgebase/configure_cap_ipc_lock_and_cap_sys_nice_in_docker). + ## Configuration The container exposes port 8123 for the [HTTP interface](https://clickhouse.com/docs/en/interfaces/http_interface/) and port 9000 for the [native client](https://clickhouse.com/docs/en/interfaces/tcp/). @@ -122,14 +126,14 @@ ClickHouse configuration is represented with a file "config.xml" ([documentation ### Start server instance with custom configuration ```bash -docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /path/to/your/config.xml:/etc/clickhouse-server/config.xml clickhouse/clickhouse-server +docker run -d --name some-clickhouse-server --ulimit nofile=262144:262144 -v /path/to/your/config.xml:/etc/clickhouse-server/config.xml %%IMAGE%% ``` ### Start server as custom user ```bash -# $(pwd)/data/clickhouse should exist and be owned by current user -docker run --rm --user ${UID}:${GID} --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +# $PWD/data/clickhouse should exist and be owned by current user +docker run --rm --user "${UID}:${GID}" --name some-clickhouse-server --ulimit nofile=262144:262144 -v "$PWD/logs/clickhouse:/var/log/clickhouse-server" -v "$PWD/data/clickhouse:/var/lib/clickhouse" %%IMAGE%% ``` When you use the image with local directories mounted, you probably want to specify the user to maintain the proper file ownership. Use the `--user` argument and mount `/var/lib/clickhouse` and `/var/log/clickhouse-server` inside the container. Otherwise, the image will complain and not start. @@ -137,7 +141,7 @@ When you use the image with local directories mounted, you probably want to spec ### Start server from root (useful in case of enabled user namespace) ```bash -docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-server-userns -v "$(pwd)/logs/clickhouse:/var/log/clickhouse-server" -v "$(pwd)/data/clickhouse:/var/lib/clickhouse" clickhouse/clickhouse-server +docker run --rm -e CLICKHOUSE_RUN_AS_ROOT=1 --name clickhouse-server-userns -v "$PWD/logs/clickhouse:/var/log/clickhouse-server" -v "$PWD/data/clickhouse:/var/lib/clickhouse" %%IMAGE%% ``` ### How to create default database and user on starting @@ -145,7 +149,7 @@ docker run --rm -e CLICKHOUSE_UID=0 -e CLICKHOUSE_GID=0 --name clickhouse-server Sometimes you may want to create a user (user named `default` is used by default) and database on a container start. You can do it using environment variables `CLICKHOUSE_DB`, `CLICKHOUSE_USER`, `CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT` and `CLICKHOUSE_PASSWORD`: ```bash -docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT=1 -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp clickhouse/clickhouse-server +docker run --rm -e CLICKHOUSE_DB=my_database -e CLICKHOUSE_USER=username -e CLICKHOUSE_DEFAULT_ACCESS_MANAGEMENT=1 -e CLICKHOUSE_PASSWORD=password -p 9000:9000/tcp %%IMAGE%% ``` ## How to extend this image From da0e267278efa2f42e0f18bf5a4b78a5d16dbe99 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov <48961922+Avogar@users.noreply.github.com> Date: Fri, 8 Nov 2024 13:30:21 +0100 Subject: [PATCH 57/91] Fix typo --- .../queries/0_stateless/03247_ghdata_string_to_json_alter.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh b/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh index a2d1788cb5d..e8368b6702a 100755 --- a/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh +++ b/tests/queries/0_stateless/03247_ghdata_string_to_json_alter.sh @@ -18,12 +18,12 @@ ${CLICKHOUSE_CLIENT} -q "SELECT count() FROM ghdata WHERE NOT ignore(*)" ${CLICKHOUSE_CLIENT} -q \ "SELECT data.repo.name, count() AS stars FROM ghdata \ - WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" --allow_suspicious_types_in_group_by=1, --allow_suspicious_types_in_order_by=1 + WHERE data.type = 'WatchEvent' GROUP BY data.repo.name ORDER BY stars DESC, data.repo.name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1 ${CLICKHOUSE_CLIENT} --enable_analyzer=1 -q \ "SELECT data.payload.commits[].author.name AS name, count() AS c FROM ghdata \ ARRAY JOIN data.payload.commits[].author.name \ - GROUP BY name ORDER BY c DESC, name LIMIT 5" --allow_suspicious_types_in_group_by=1, --allow_suspicious_types_in_order_by=1 + GROUP BY name ORDER BY c DESC, name LIMIT 5" --allow_suspicious_types_in_group_by=1 --allow_suspicious_types_in_order_by=1 ${CLICKHOUSE_CLIENT} -q "SELECT max(data.payload.pull_request.assignees[].size0) FROM ghdata" From 298b172c49493a88c87fde4d0e09a6413102de55 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 7 Nov 2024 22:30:45 +0100 Subject: [PATCH 58/91] Add fallback to getgrgid_r and getpwuid_r for UID and GID arguments of clickhouse-su --- programs/su/su.cpp | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/programs/su/su.cpp b/programs/su/su.cpp index 33d929898f4..40242d0687f 100644 --- a/programs/su/su.cpp +++ b/programs/su/su.cpp @@ -59,7 +59,13 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid) throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid); if (!result) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group {} is not found in the system", arg_gid); + { + if (0 != getgrgid_r(gid, &entry, buf.get(), buf_size, &result)) + throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getgrnam_r' to obtain gid from group name ({})", arg_gid); + + if (!result) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Group {} is not found in the system", arg_gid); + } gid = entry.gr_gid; } @@ -84,7 +90,13 @@ void setUserAndGroup(std::string arg_uid, std::string arg_gid) throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getpwnam_r' to obtain uid from user name ({})", arg_uid); if (!result) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "User {} is not found in the system", arg_uid); + { + if (0 != getpwuid_r(uid, &entry, buf.get(), buf_size, &result)) + throw ErrnoException(ErrorCodes::SYSTEM_ERROR, "Cannot do 'getpwuid_r' to obtain uid from user name ({})", uid); + + if (!result) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "User {} is not found in the system", arg_uid); + } uid = entry.pw_uid; } From 69ae05210364cf03bddf62b13bd752857bcbbedc Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 8 Nov 2024 10:22:01 +0000 Subject: [PATCH 59/91] SimSIMD: Improve suppression for msan false positive --- contrib/SimSIMD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index ee3c9c9c00b..9e3cfc32d26 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit ee3c9c9c00b51645f62a1a9e99611b78c0052a21 +Subproject commit 9e3cfc32d26fbeece91e34df8668db28c0ca006a From ba20032987a042d45b4073e93eb5279222aff4ac Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 8 Nov 2024 14:08:36 +0000 Subject: [PATCH 60/91] Fix build --- contrib/SimSIMD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index 9e3cfc32d26..bb0bd2e7137 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit 9e3cfc32d26fbeece91e34df8668db28c0ca006a +Subproject commit bb0bd2e7137f02c555341d7c93124ed19f3c24fb From 5d2e1547a89cb43d545c5847cf47565f319bbd75 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 8 Nov 2024 11:34:06 -0500 Subject: [PATCH 61/91] use `/var/log/mysql/` instead of `/mysql/` --- tests/integration/compose/docker_compose_mysql.yml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/compose/docker_compose_mysql.yml b/tests/integration/compose/docker_compose_mysql.yml index f45410bde78..91df21165ea 100644 --- a/tests/integration/compose/docker_compose_mysql.yml +++ b/tests/integration/compose/docker_compose_mysql.yml @@ -5,7 +5,7 @@ services: environment: MYSQL_ROOT_PASSWORD: clickhouse MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} - DATADIR: /mysql/ + DATADIR: /var/log/mysql/ expose: - ${MYSQL_PORT:-3306} command: --server_id=100 @@ -14,11 +14,11 @@ services: --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 - --log-error=/mysql/error.log + --log-error=/var/log/mysql/error.log --general-log=ON - --general-log-file=/mysql/general.log + --general-log-file=/var/log/mysql/general.log volumes: - type: ${MYSQL_LOGS_FS:-tmpfs} source: ${MYSQL_LOGS:-} - target: /mysql/ + target: /var/log/mysql/ user: ${MYSQL_DOCKER_USER} From 97ec890b8e3e06d9914e573363965ef439f76d21 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Fri, 8 Nov 2024 11:36:36 -0500 Subject: [PATCH 62/91] use `/var/log/mysql/` instead of `/mysql/`, fix `MYSQL_ROOT_HOST` env initialization --- tests/integration/compose/docker_compose_mysql_8_0.yml | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/compose/docker_compose_mysql_8_0.yml b/tests/integration/compose/docker_compose_mysql_8_0.yml index e1ff1633bc7..e1e2e241443 100644 --- a/tests/integration/compose/docker_compose_mysql_8_0.yml +++ b/tests/integration/compose/docker_compose_mysql_8_0.yml @@ -4,8 +4,8 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} - DATADIR: /mysql/ + MYSQL_ROOT_HOST: ${MYSQL8_ROOT_HOST} + DATADIR: /var/log/mysql/ expose: - ${MYSQL8_PORT:-3306} command: --server_id=100 --log-bin='mysql-bin-1.log' @@ -13,11 +13,11 @@ services: --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 - --log-error=/mysql/error.log + --log-error=/var/log/mysql/error.log --general-log=ON - --general-log-file=/mysql/general.log + --general-log-file=/var/log/mysql/general.log volumes: - type: ${MYSQL8_LOGS_FS:-tmpfs} source: ${MYSQL8_LOGS:-} - target: /mysql/ + target: /var/log/mysql/ user: ${MYSQL8_DOCKER_USER} From fe39c4b65bfee09d9c7d5327963983fbd4cdd234 Mon Sep 17 00:00:00 2001 From: Tanya Bragin Date: Fri, 8 Nov 2024 08:55:20 -0800 Subject: [PATCH 63/91] Update README.md - Update meetups Add Stockholm --- README.md | 1 + 1 file changed, 1 insertion(+) diff --git a/README.md b/README.md index dcaeda13acd..abaf27abf11 100644 --- a/README.md +++ b/README.md @@ -47,6 +47,7 @@ Upcoming meetups * [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21 * [Paris Meetup](https://www.meetup.com/clickhouse-france-user-group/events/303096434) - November 26 * [Amsterdam Meetup](https://www.meetup.com/clickhouse-netherlands-user-group/events/303638814) - December 3 +* [Stockholm Meetup](https://www.meetup.com/clickhouse-stockholm-user-group/events/304382411) - December 9 * [New York Meetup](https://www.meetup.com/clickhouse-new-york-user-group/events/304268174) - December 9 * [San Francisco Meetup](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/304286951/) - December 12 From 9dc4046b897bd7cd185c0f5e0e221dea7481f8a9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2024 18:02:41 +0100 Subject: [PATCH 64/91] Add index granularity size column to system.parts --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 9 +++++++++ src/Storages/MergeTree/IMergeTreeDataPart.h | 2 ++ .../MergeTree/MergeTreeIndexGranularity.cpp | 10 ++++++++++ src/Storages/MergeTree/MergeTreeIndexGranularity.h | 3 +++ src/Storages/System/StorageSystemParts.cpp | 6 ++++++ .../03268_system_parts_index_granularity.reference | 1 + .../03268_system_parts_index_granularity.sql | 14 ++++++++++++++ 7 files changed, 45 insertions(+) create mode 100644 tests/queries/0_stateless/03268_system_parts_index_granularity.reference create mode 100644 tests/queries/0_stateless/03268_system_parts_index_granularity.sql diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 7453d609fa9..51c445945e6 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -624,6 +624,15 @@ UInt64 IMergeTreeDataPart::getIndexSizeInAllocatedBytes() const return res; } +UInt64 IMergeTreeDataPart::getIndexGranularityBytes() const +{ + return index_granularity.getBytesSize(); +} +UInt64 IMergeTreeDataPart::getIndexGranularityAllocatedBytes() const +{ + return index_granularity.getBytesAllocated(); +} + void IMergeTreeDataPart::assertState(const std::initializer_list & affordable_states) const { if (!checkState(affordable_states)) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index b41a1d840e1..55f1265318c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -380,6 +380,8 @@ public: /// For data in RAM ('index') UInt64 getIndexSizeInBytes() const; UInt64 getIndexSizeInAllocatedBytes() const; + UInt64 getIndexGranularityBytes() const; + UInt64 getIndexGranularityAllocatedBytes() const; UInt64 getMarksCount() const; UInt64 getIndexSizeFromFile() const; diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index c3e740bde84..bf0ba17d473 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -128,4 +128,14 @@ void MergeTreeIndexGranularity::shrinkToFitInMemory() marks_rows_partial_sums.shrink_to_fit(); } +uint64_t MergeTreeIndexGranularity::getBytesSize() const +{ + return marks_rows_partial_sums.size() * sizeof(size_t); +} +uint64_t MergeTreeIndexGranularity::getBytesAllocated() const +{ + return marks_rows_partial_sums.capacity() * sizeof(size_t); +} + + } diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/src/Storages/MergeTree/MergeTreeIndexGranularity.h index 9b8375dd2d8..c616d2ac49a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -102,6 +102,9 @@ public: std::string describe() const; void shrinkToFitInMemory(); + + uint64_t getBytesSize() const; + uint64_t getBytesAllocated() const; }; } diff --git a/src/Storages/System/StorageSystemParts.cpp b/src/Storages/System/StorageSystemParts.cpp index 56a45d7b51d..d0e34842198 100644 --- a/src/Storages/System/StorageSystemParts.cpp +++ b/src/Storages/System/StorageSystemParts.cpp @@ -75,6 +75,8 @@ StorageSystemParts::StorageSystemParts(const StorageID & table_id_) {"data_version", std::make_shared(), "Number that is used to determine which mutations should be applied to the data part (mutations with a version higher than data_version)."}, {"primary_key_bytes_in_memory", std::make_shared(), "The amount of memory (in bytes) used by primary key values."}, {"primary_key_bytes_in_memory_allocated", std::make_shared(), "The amount of memory (in bytes) reserved for primary key values."}, + {"index_granularity_bytes_in_memory", std::make_shared(), "The amount of memory (in bytes) used by index granularity values."}, + {"index_granularity_bytes_in_memory_allocated", std::make_shared(), "The amount of memory (in bytes) reserved for index granularity values."}, {"is_frozen", std::make_shared(), "Flag that shows that a partition data backup exists. 1, the backup exists. 0, the backup does not exist. "}, {"database", std::make_shared(), "Name of the database."}, @@ -216,6 +218,10 @@ void StorageSystemParts::processNextStorage( columns[res_index++]->insert(part->getIndexSizeInBytes()); if (columns_mask[src_index++]) columns[res_index++]->insert(part->getIndexSizeInAllocatedBytes()); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->getIndexGranularityBytes()); + if (columns_mask[src_index++]) + columns[res_index++]->insert(part->getIndexGranularityAllocatedBytes()); if (columns_mask[src_index++]) columns[res_index++]->insert(part->is_frozen.load(std::memory_order_relaxed)); diff --git a/tests/queries/0_stateless/03268_system_parts_index_granularity.reference b/tests/queries/0_stateless/03268_system_parts_index_granularity.reference new file mode 100644 index 00000000000..f301cd54ad2 --- /dev/null +++ b/tests/queries/0_stateless/03268_system_parts_index_granularity.reference @@ -0,0 +1 @@ +88 88 diff --git a/tests/queries/0_stateless/03268_system_parts_index_granularity.sql b/tests/queries/0_stateless/03268_system_parts_index_granularity.sql new file mode 100644 index 00000000000..009a15d0825 --- /dev/null +++ b/tests/queries/0_stateless/03268_system_parts_index_granularity.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS t; + +CREATE TABLE t ( + key UInt64, + value String +) +ENGINE MergeTree() +ORDER by key SETTINGS index_granularity = 10, index_granularity_bytes = '1024K'; + +INSERT INTO t SELECT number, toString(number) FROM numbers(100); + +SELECT index_granularity_bytes_in_memory, index_granularity_bytes_in_memory_allocated FROM system.parts where table = 't' and database = currentDatabase(); + +DROP TABLE IF EXISTS t; From 6c223c92bd852b56c713aff768b07c4adb90d5dc Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2024 18:13:29 +0100 Subject: [PATCH 65/91] btter --- .../queries/0_stateless/03268_system_parts_index_granularity.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/03268_system_parts_index_granularity.sql b/tests/queries/0_stateless/03268_system_parts_index_granularity.sql index 009a15d0825..1bab7840856 100644 --- a/tests/queries/0_stateless/03268_system_parts_index_granularity.sql +++ b/tests/queries/0_stateless/03268_system_parts_index_granularity.sql @@ -1,3 +1,4 @@ +-- Tags: no-random-settings, no-random-merge-tree-settings DROP TABLE IF EXISTS t; CREATE TABLE t ( From 6d2504662a45e0c35758698ec60ac265309c0f6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 01:01:46 +0100 Subject: [PATCH 66/91] Update tests --- tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql | 1 + tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql | 1 + tests/queries/0_stateless/01958_partial_hour_timezone.sql | 2 ++ tests/queries/0_stateless/02125_query_views_log.sql | 2 ++ 4 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql index 0154265ef72..907a8283396 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -1,5 +1,6 @@ SET session_timezone = 'UTC'; -- disable timezone randomization SET enable_analyzer = 1; -- The old path formats the result with different whitespaces +SET output_format_pretty_highlight_digit_groups = 0; SELECT '-- Negative tests'; SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql index 41e5beb9c16..1f62f3d36da 100644 --- a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql @@ -1,5 +1,6 @@ SET session_timezone = 'UTC'; -- disable timezone randomization SET enable_analyzer = 1; -- The old path formats the result with different whitespaces +SET output_format_pretty_highlight_digit_groups = 0; SELECT '-- Negative tests'; SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.sql b/tests/queries/0_stateless/01958_partial_hour_timezone.sql index 26350e55620..b72adfd9d58 100644 --- a/tests/queries/0_stateless/01958_partial_hour_timezone.sql +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.sql @@ -1,3 +1,5 @@ +SET output_format_pretty_highlight_digit_groups = 0; + -- Appeared in https://github.com/ClickHouse/ClickHouse/pull/26978#issuecomment-890889362 WITH toDateTime('1970-06-17 07:39:21', 'Africa/Monrovia') as t SELECT toUnixTimestamp(t), diff --git a/tests/queries/0_stateless/02125_query_views_log.sql b/tests/queries/0_stateless/02125_query_views_log.sql index ba50902ebea..96170efedd6 100644 --- a/tests/queries/0_stateless/02125_query_views_log.sql +++ b/tests/queries/0_stateless/02125_query_views_log.sql @@ -1,3 +1,5 @@ +SET output_format_pretty_highlight_digit_groups = 0; + drop table if exists src; drop table if exists dst; drop table if exists mv1; From 19ca58e95203ed2eed71a6ef0ab88677d9bb6b93 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 02:29:44 +0100 Subject: [PATCH 67/91] Fix #71677 --- src/Functions/nested.cpp | 19 +++++++++++++------ .../03268_nested_analyzer.reference | 3 +++ .../0_stateless/03268_nested_analyzer.sql | 16 ++++++++++++++++ 3 files changed, 32 insertions(+), 6 deletions(-) create mode 100644 tests/queries/0_stateless/03268_nested_analyzer.reference create mode 100644 tests/queries/0_stateless/03268_nested_analyzer.sql diff --git a/src/Functions/nested.cpp b/src/Functions/nested.cpp index 85c342b5e7c..29d99b8a6df 100644 --- a/src/Functions/nested.cpp +++ b/src/Functions/nested.cpp @@ -108,27 +108,29 @@ public: { size_t arguments_size = arguments.size(); - const auto * lhs_array = assert_cast(arguments.at(1).column.get()); + ColumnPtr first_array_materialized = arguments[1].column->convertToFullColumnIfConst(); + const ColumnArray & first_array = assert_cast(*first_array_materialized); Columns data_columns; data_columns.reserve(arguments_size); - data_columns.push_back(lhs_array->getDataPtr()); + data_columns.push_back(first_array.getDataPtr()); for (size_t i = 2; i < arguments_size; ++i) { - const auto * rhs_array = assert_cast(arguments[i].column.get()); + ColumnPtr other_array_materialized = arguments[i].column->convertToFullColumnIfConst(); + const ColumnArray & other_array = assert_cast(*other_array_materialized); - if (!lhs_array->hasEqualOffsets(*rhs_array)) + if (!first_array.hasEqualOffsets(other_array)) throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH, "The argument 2 and argument {} of function {} have different array offsets", i + 1, getName()); - data_columns.push_back(rhs_array->getDataPtr()); + data_columns.push_back(other_array.getDataPtr()); } auto tuple_column = ColumnTuple::create(std::move(data_columns)); - auto array_column = ColumnArray::create(std::move(tuple_column), lhs_array->getOffsetsPtr()); + auto array_column = ColumnArray::create(std::move(tuple_column), first_array.getOffsetsPtr()); return array_column; } @@ -168,7 +170,12 @@ REGISTER_FUNCTION(Nested) { factory.registerFunction(FunctionDocumentation{ .description=R"( +This is a function used internally by the ClickHouse engine and not meant to be used directly. + Returns the array of tuples from multiple arrays. + +The first argument must be a constant array of Strings determining the names of the resulting Tuple. +The other arguments must be arrays of the same size. )", .examples{{"nested", "SELECT nested(['keys', 'values'], ['key_1', 'key_2'], ['value_1','value_2'])", ""}}, .categories{"OtherFunctions"} diff --git a/tests/queries/0_stateless/03268_nested_analyzer.reference b/tests/queries/0_stateless/03268_nested_analyzer.reference new file mode 100644 index 00000000000..01dabfe4ba7 --- /dev/null +++ b/tests/queries/0_stateless/03268_nested_analyzer.reference @@ -0,0 +1,3 @@ +[(1,3),(2,4)] +0 0 +0 0 1 diff --git a/tests/queries/0_stateless/03268_nested_analyzer.sql b/tests/queries/0_stateless/03268_nested_analyzer.sql new file mode 100644 index 00000000000..920cf2b3174 --- /dev/null +++ b/tests/queries/0_stateless/03268_nested_analyzer.sql @@ -0,0 +1,16 @@ +SELECT nested(['a', 'b'], [1, 2], materialize([3, 4])); + +DROP TABLE IF EXISTS test; +CREATE TABLE test +( + x UInt8, + “struct.x” DEFAULT [0], + “struct.y” ALIAS [1], +) +ENGINE = Memory; + +insert into test (x) values (0); +select * from test array join struct; +select x, struct.x, struct.y from test array join struct; + +DROP TABLE test; From b5237313adaac770c95b8c9415a01c23b1372f66 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 02:49:27 +0100 Subject: [PATCH 68/91] Fix tests --- tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql | 2 +- tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql | 2 +- tests/queries/0_stateless/01958_partial_hour_timezone.sql | 2 +- tests/queries/0_stateless/02125_query_views_log.sql | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql index 907a8283396..aeaf48716dc 100644 --- a/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql +++ b/tests/queries/0_stateless/01942_dateTimeToSnowflakeID.sql @@ -1,6 +1,6 @@ SET session_timezone = 'UTC'; -- disable timezone randomization SET enable_analyzer = 1; -- The old path formats the result with different whitespaces -SET output_format_pretty_highlight_digit_groups = 0; +SET output_format_pretty_single_large_number_tip_threshold = 0; SELECT '-- Negative tests'; SELECT dateTimeToSnowflakeID(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} diff --git a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql index 1f62f3d36da..e9b32607837 100644 --- a/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql +++ b/tests/queries/0_stateless/01942_snowflakeIDToDateTime.sql @@ -1,6 +1,6 @@ SET session_timezone = 'UTC'; -- disable timezone randomization SET enable_analyzer = 1; -- The old path formats the result with different whitespaces -SET output_format_pretty_highlight_digit_groups = 0; +SET output_format_pretty_single_large_number_tip_threshold = 0; SELECT '-- Negative tests'; SELECT snowflakeIDToDateTime(); -- {serverError NUMBER_OF_ARGUMENTS_DOESNT_MATCH} diff --git a/tests/queries/0_stateless/01958_partial_hour_timezone.sql b/tests/queries/0_stateless/01958_partial_hour_timezone.sql index b72adfd9d58..3eecaaf97e6 100644 --- a/tests/queries/0_stateless/01958_partial_hour_timezone.sql +++ b/tests/queries/0_stateless/01958_partial_hour_timezone.sql @@ -1,4 +1,4 @@ -SET output_format_pretty_highlight_digit_groups = 0; +SET output_format_pretty_single_large_number_tip_threshold = 0; -- Appeared in https://github.com/ClickHouse/ClickHouse/pull/26978#issuecomment-890889362 WITH toDateTime('1970-06-17 07:39:21', 'Africa/Monrovia') as t diff --git a/tests/queries/0_stateless/02125_query_views_log.sql b/tests/queries/0_stateless/02125_query_views_log.sql index 96170efedd6..08e9c73a165 100644 --- a/tests/queries/0_stateless/02125_query_views_log.sql +++ b/tests/queries/0_stateless/02125_query_views_log.sql @@ -1,4 +1,4 @@ -SET output_format_pretty_highlight_digit_groups = 0; +SET output_format_pretty_single_large_number_tip_threshold = 0; drop table if exists src; drop table if exists dst; From 959c4633f9e8cbc3f41def853fa62618fba604c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 02:53:32 +0100 Subject: [PATCH 69/91] Apply review suggestion --- src/Formats/PrettyFormatHelpers.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Formats/PrettyFormatHelpers.cpp b/src/Formats/PrettyFormatHelpers.cpp index 6e2af036651..4ee4b49521d 100644 --- a/src/Formats/PrettyFormatHelpers.cpp +++ b/src/Formats/PrettyFormatHelpers.cpp @@ -5,6 +5,11 @@ #include +static constexpr const char * GRAY_COLOR = "\033[90m"; +static constexpr const char * UNDERSCORE = "\033[4m"; +static constexpr const char * RESET_COLOR = "\033[0m"; + + namespace DB { @@ -25,11 +30,11 @@ void writeReadableNumberTip(WriteBuffer & out, const IColumn & column, size_t ro if (threshold && isFinite(value) && abs(value) > threshold) { if (color) - writeCString("\033[90m", out); + writeCString(GRAY_COLOR, out); writeCString(" -- ", out); formatReadableQuantity(value, out, 2); if (color) - writeCString("\033[0m", out); + writeCString(RESET_COLOR, out); } } @@ -76,9 +81,9 @@ String highlightDigitGroups(String source) size_t offset = num_digits_before_decimal - digit_num; if (offset && offset % 3 == 0) { - result += "\033[4m"; + result += UNDERSCORE; result += c; - result += "\033[0m"; + result += RESET_COLOR; } else { From ef0ec74d2bfe8ae61a06e0c3fa4e33fb3c094ef6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 04:50:18 +0100 Subject: [PATCH 70/91] Fix build --- src/Processors/Formats/Impl/PrettyBlockOutputFormat.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 824a2fd2e6f..81bd0e6632d 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -55,8 +55,6 @@ protected: } bool color; - -protected: bool readable_number_tip = false; private: From e50bbc433e5c57c96bbf71e22b900a28eb5be6c5 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Fri, 8 Nov 2024 22:08:08 +0100 Subject: [PATCH 71/91] Another review round for docker-library/docs --- docker/server/README.md | 4 ++-- docker/server/README.src/content.md | 4 ++-- docker/server/README.src/github-repo | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/server/README.md b/docker/server/README.md index 7403d5b0b2a..5f6144d0633 100644 --- a/docker/server/README.md +++ b/docker/server/README.md @@ -30,7 +30,7 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A) and additionally the Load-Acquire RCpc register. The register is optional in version ARMv8.2-A and mandatory in [ARMv8.3-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.3-A). Supported in Graviton >=2, Azure and GCP instances. Examples for unsupported devices are Raspberry Pi 4 (ARMv8.0-A) and Jetson AGX Xavier/Orin (ARMv8.2-A). -- Since the Clickhouse 24.11 Ubuntu images started using `ubuntu:22.04` as its base image. It requires docker version >= `20.10.10` containing [patch](https://github.com/moby/moby/commit/977283509f75303bc6612665a04abf76ff1d2468). As a workaround you could use `docker run [--privileged | --security-opt seccomp=unconfined]` instead, however that has security implications. +- Since the Clickhouse 24.11 Ubuntu images started using `ubuntu:22.04` as its base image. It requires docker version >= `20.10.10` containing [patch](https://github.com/moby/moby/commit/977283509f75303bc6612665a04abf76ff1d2468). As a workaround you could use `docker run --security-opt seccomp=unconfined` instead, however that has security implications. ## How to use this image @@ -57,7 +57,7 @@ More information about the [ClickHouse client](https://clickhouse.com/docs/en/in ### connect to it using curl ```bash -echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server buildpack-deps:curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- +echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server buildpack-deps:curl curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- ``` More information about the [ClickHouse HTTP Interface](https://clickhouse.com/docs/en/interfaces/http/). diff --git a/docker/server/README.src/content.md b/docker/server/README.src/content.md index bfc1a271546..df0b6718d69 100644 --- a/docker/server/README.src/content.md +++ b/docker/server/README.src/content.md @@ -24,7 +24,7 @@ For more information and documentation see https://clickhouse.com/. - The amd64 image requires support for [SSE3 instructions](https://en.wikipedia.org/wiki/SSE3). Virtually all x86 CPUs after 2005 support SSE3. - The arm64 image requires support for the [ARMv8.2-A architecture](https://en.wikipedia.org/wiki/AArch64#ARMv8.2-A) and additionally the Load-Acquire RCpc register. The register is optional in version ARMv8.2-A and mandatory in [ARMv8.3-A](https://en.wikipedia.org/wiki/AArch64#ARMv8.3-A). Supported in Graviton >=2, Azure and GCP instances. Examples for unsupported devices are Raspberry Pi 4 (ARMv8.0-A) and Jetson AGX Xavier/Orin (ARMv8.2-A). -- Since the Clickhouse 24.11 Ubuntu images started using `ubuntu:22.04` as its base image. It requires docker version >= `20.10.10` containing [patch](https://github.com/moby/moby/commit/977283509f75303bc6612665a04abf76ff1d2468). As a workaround you could use `docker run [--privileged | --security-opt seccomp=unconfined]` instead, however that has security implications. +- Since the Clickhouse 24.11 Ubuntu images started using `ubuntu:22.04` as its base image. It requires docker version >= `20.10.10` containing [patch](https://github.com/moby/moby/commit/977283509f75303bc6612665a04abf76ff1d2468). As a workaround you could use `docker run --security-opt seccomp=unconfined` instead, however that has security implications. ## How to use this image @@ -51,7 +51,7 @@ More information about the [ClickHouse client](https://clickhouse.com/docs/en/in ### connect to it using curl ```bash -echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server buildpack-deps:curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- +echo "SELECT 'Hello, ClickHouse!'" | docker run -i --rm --link some-clickhouse-server:clickhouse-server buildpack-deps:curl curl 'http://clickhouse-server:8123/?query=' -s --data-binary @- ``` More information about the [ClickHouse HTTP Interface](https://clickhouse.com/docs/en/interfaces/http/). diff --git a/docker/server/README.src/github-repo b/docker/server/README.src/github-repo index dc2b6635325..70a009ec958 100644 --- a/docker/server/README.src/github-repo +++ b/docker/server/README.src/github-repo @@ -1 +1 @@ -https://github.com/ClickHouse/docker-library +https://github.com/ClickHouse/ClickHouse From aa4d37f72cbea7834c8a8c8d6668f3b3b01b80a7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Sat, 9 Nov 2024 13:41:08 +0100 Subject: [PATCH 72/91] Fix test --- .../0_stateless/02117_show_create_table_system.reference | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference index 2ea62444cff..ef5a2c6665f 100644 --- a/tests/queries/0_stateless/02117_show_create_table_system.reference +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -485,6 +485,8 @@ CREATE TABLE system.parts `data_version` UInt64, `primary_key_bytes_in_memory` UInt64, `primary_key_bytes_in_memory_allocated` UInt64, + `index_granularity_bytes_in_memory` UInt64, + `index_granularity_bytes_in_memory_allocated` UInt64, `is_frozen` UInt8, `database` String, `table` String, From 7b1c72729a4f8e37d7c6ddf4fc8894149085e3e3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 14:22:43 +0100 Subject: [PATCH 73/91] Fix upgrade check --- tests/docker_scripts/upgrade_runner.sh | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/docker_scripts/upgrade_runner.sh b/tests/docker_scripts/upgrade_runner.sh index ece75ebf782..92484f88ece 100755 --- a/tests/docker_scripts/upgrade_runner.sh +++ b/tests/docker_scripts/upgrade_runner.sh @@ -135,7 +135,7 @@ IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM if [ "${IS_SANITIZED}" -eq "0" ] then save_settings_clean 'new_settings.native' - clickhouse-local -nmq " + clickhouse-local --implicit-select 0 -nmq " CREATE TABLE old_settings AS file('old_settings.native'); CREATE TABLE old_version AS file('old_version.native'); CREATE TABLE new_settings AS file('new_settings.native'); @@ -147,7 +147,6 @@ then FROM new_settings LEFT JOIN old_settings ON new_settings.name = old_settings.name WHERE (new_value != old_value) - AND NOT (startsWith(new_value, 'auto(') AND old_value LIKE '%auto(%') AND (name NOT IN ( SELECT arrayJoin(tupleElement(changes, 'name')) FROM From 93d586876092ae662f510d67e1ad00e6d1d55bdf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 14:30:46 +0100 Subject: [PATCH 74/91] Fix tests --- .../02751_ip_types_aggregate_functions_states.sql.j2 | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 index 7d030d4be2d..602b98e576b 100644 --- a/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 +++ b/tests/queries/0_stateless/02751_ip_types_aggregate_functions_states.sql.j2 @@ -1,5 +1,7 @@ -- Tags: no-parallel, no-fasttest +SET output_format_pretty_single_large_number_tip_threshold = 0; + {# this test checks backward compatibility of aggregate functions States against IPv4, IPv6 types #} {% set ip4_generator = "select num::UInt32::IPv4 ip from (select arrayJoin(range(999999999, number)) as num from numbers(999999999,50)) order by ip" %} From 016c122af9d85da32726d9bb0d2b318eda06c2e8 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 16:33:37 +0100 Subject: [PATCH 75/91] Update PULL_REQUEST_TEMPLATE.md --- .github/PULL_REQUEST_TEMPLATE.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/PULL_REQUEST_TEMPLATE.md b/.github/PULL_REQUEST_TEMPLATE.md index 3dcce68ab46..976c69d3c34 100644 --- a/.github/PULL_REQUEST_TEMPLATE.md +++ b/.github/PULL_REQUEST_TEMPLATE.md @@ -12,7 +12,7 @@ tests/ci/cancel_and_rerun_workflow_lambda/app.py - Backward Incompatible Change - Build/Testing/Packaging Improvement - Documentation (changelog entry is not required) -- Critical Bug Fix (crash, LOGICAL_ERROR, data loss, RBAC) +- Critical Bug Fix (crash, data loss, RBAC) - Bug Fix (user-visible misbehavior in an official stable release) - CI Fix or Improvement (changelog entry is not required) - Not for changelog (changelog entry is not required) From a898f163546b3cb3d607443e9881c88337867e83 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 17:21:53 +0100 Subject: [PATCH 76/91] Fix tests --- tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql index e7d1909cae6..22b59a16255 100644 --- a/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql +++ b/tests/queries/0_stateless/02184_hash_functions_and_ip_types.sql @@ -1,5 +1,6 @@ -- Tags: no-fasttest +SET output_format_pretty_single_large_number_tip_threshold = 0; SET enable_analyzer = 1; SELECT From 7849a9ce16d8a8ed9d97e3b57541c108bb00d044 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 18:12:11 +0100 Subject: [PATCH 77/91] Fix error --- tests/docker_scripts/upgrade_runner.sh | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/docker_scripts/upgrade_runner.sh b/tests/docker_scripts/upgrade_runner.sh index 92484f88ece..15c5ab69521 100755 --- a/tests/docker_scripts/upgrade_runner.sh +++ b/tests/docker_scripts/upgrade_runner.sh @@ -63,7 +63,7 @@ install_packages previous_release_package_folder function save_settings_clean() { local out=$1 && shift - script -q -c "clickhouse-local -q \"select * from system.settings into outfile '$out'\"" --log-out /dev/null + script -q -c "clickhouse-local --implicit-select 0 -q \"select * from system.settings into outfile '$out'\"" --log-out /dev/null } # We save the (numeric) version of the old server to compare setting changes between the 2 @@ -135,7 +135,7 @@ IS_SANITIZED=$(clickhouse-local --query "SELECT value LIKE '%-fsanitize=%' FROM if [ "${IS_SANITIZED}" -eq "0" ] then save_settings_clean 'new_settings.native' - clickhouse-local --implicit-select 0 -nmq " + clickhouse-local -nmq " CREATE TABLE old_settings AS file('old_settings.native'); CREATE TABLE old_version AS file('old_version.native'); CREATE TABLE new_settings AS file('new_settings.native'); From 979b2128067e44e92bb738b491ebabaa0f41cbeb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jan 2024 19:36:35 +0100 Subject: [PATCH 78/91] Make higher order functions constant expressions --- src/Functions/FunctionsMiscellaneous.h | 14 +++++++++++++- ...961_higher_order_constant_expressions.reference | 8 ++++++++ .../02961_higher_order_constant_expressions.sql | 11 +++++++++++ 3 files changed, 32 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02961_higher_order_constant_expressions.reference create mode 100644 tests/queries/0_stateless/02961_higher_order_constant_expressions.sql diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index fb5109eaa88..4b189279651 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -6,6 +6,7 @@ #include #include #include +#include #include @@ -122,12 +123,19 @@ public: String getName() const override { return "FunctionCapture"; } bool useDefaultImplementationForNulls() const override { return false; } + /// It's possible if expression_actions contains function that don't use /// default implementation for Nothing and one of captured columns can be Nothing /// Example: SELECT arrayMap(x -> [x, arrayElement(y, 0)], []), [] as y bool useDefaultImplementationForNothing() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } + /// If all the captured arguments are constant, let's also return ColumnConst (with ColumnFunction inside it). + /// Consequently, it allows to treat higher order functions with constant arrays and constant captured columns + /// as constant expressions. + /// Consequently, it allows its usage in contexts requiring constants, such as the right hand side of IN. + bool useDefaultImplementationForConstants() const override { return true; } + ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { Names names; @@ -148,7 +156,11 @@ public: auto function = std::make_unique(expression_actions, types, names, capture->return_type, capture->return_name); - return ColumnFunction::create(input_rows_count, std::move(function), arguments); + /// If there are no captured columns, the result is constant. + if (arguments.empty()) + return ColumnConst::create(ColumnFunction::create(1, std::move(function), arguments), input_rows_count); + else + return ColumnFunction::create(input_rows_count, std::move(function), arguments); } private: diff --git a/tests/queries/0_stateless/02961_higher_order_constant_expressions.reference b/tests/queries/0_stateless/02961_higher_order_constant_expressions.reference new file mode 100644 index 00000000000..058d23ad850 --- /dev/null +++ b/tests/queries/0_stateless/02961_higher_order_constant_expressions.reference @@ -0,0 +1,8 @@ +[1,2,3] 1 +[2,3,4] 1 +[2,4,6] 1 +[5,7,9] 1 +[1,1,1] 1 +[1,2,3] 0 +[0,0,0] 0 +3 1 diff --git a/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql b/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql new file mode 100644 index 00000000000..47480010751 --- /dev/null +++ b/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql @@ -0,0 +1,11 @@ +SELECT arrayMap(x -> x, [1, 2, 3]) AS x, isConstant(x); +SELECT arrayMap(x -> x + 1, [1, 2, 3]) AS x, isConstant(x); +SELECT arrayMap(x -> x + x, [1, 2, 3]) AS x, isConstant(x); +SELECT arrayMap((x, y) -> x + y, [1, 2, 3], [4, 5, 6]) AS x, isConstant(x); +SELECT arrayMap(x -> 1, [1, 2, 3]) AS x, isConstant(x); +SELECT arrayMap(x -> x + number, [1, 2, 3]) AS x, isConstant(x) FROM numbers(1); +SELECT arrayMap(x -> number, [1, 2, 3]) AS x, isConstant(x) FROM numbers(1); +SELECT arrayMax([1, 2, 3]) AS x, isConstant(x); + +-- Does not work yet: +-- SELECT [1, 2, 3] IN arrayMap(x -> x, [1, 2, 3]); From 3f2f358fb9c7e8b01e32632684c1dea24c1fc67a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 1 Jan 2024 20:32:27 +0100 Subject: [PATCH 79/91] Support constant lambda functions --- src/Functions/array/FunctionArrayMapped.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/array/FunctionArrayMapped.h b/src/Functions/array/FunctionArrayMapped.h index f4832431f04..e51c465f883 100644 --- a/src/Functions/array/FunctionArrayMapped.h +++ b/src/Functions/array/FunctionArrayMapped.h @@ -282,7 +282,9 @@ public: if (!column_with_type_and_name.column) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function.", getName()); - const auto * column_function = typeid_cast(column_with_type_and_name.column.get()); + auto column_function_materialized = column_with_type_and_name.column->convertToFullColumnIfConst(); + + const auto * column_function = typeid_cast(column_function_materialized.get()); if (!column_function) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function.", getName()); From 6c1016568c4e76e2285a5a73d5bbfc7c3d0824a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 03:35:21 +0100 Subject: [PATCH 80/91] Better implementation --- src/Functions/FunctionsMiscellaneous.h | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 4b189279651..6e89c4dd65d 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -130,12 +130,6 @@ public: bool useDefaultImplementationForNothing() const override { return false; } bool useDefaultImplementationForLowCardinalityColumns() const override { return false; } - /// If all the captured arguments are constant, let's also return ColumnConst (with ColumnFunction inside it). - /// Consequently, it allows to treat higher order functions with constant arrays and constant captured columns - /// as constant expressions. - /// Consequently, it allows its usage in contexts requiring constants, such as the right hand side of IN. - bool useDefaultImplementationForConstants() const override { return true; } - ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { Names names; @@ -156,8 +150,13 @@ public: auto function = std::make_unique(expression_actions, types, names, capture->return_type, capture->return_name); - /// If there are no captured columns, the result is constant. - if (arguments.empty()) + /// If all the captured arguments are constant, let's also return ColumnConst (with ColumnFunction inside it). + /// Consequently, it allows to treat higher order functions with constant arrays and constant captured columns + /// as constant expressions. + /// Consequently, it allows its usage in contexts requiring constants, such as the right hand side of IN. + bool all_arguments_are_constant = std::all_of(arguments.begin(), arguments.end(), [](const auto & arg) { return arg.column->isConst(); }); + + if (all_arguments_are_constant) return ColumnConst::create(ColumnFunction::create(1, std::move(function), arguments), input_rows_count); else return ColumnFunction::create(input_rows_count, std::move(function), arguments); From f1777b957946a5ab32851d8e1c7000ee3657e3d7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 03:55:18 +0100 Subject: [PATCH 81/91] Fix error --- src/Functions/FunctionsMiscellaneous.h | 10 +++++++++- src/Processors/Formats/IOutputFormat.cpp | 2 +- src/Processors/Transforms/DistinctTransform.cpp | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 4 files changed, 12 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 6e89c4dd65d..62b43386db5 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -157,9 +157,17 @@ public: bool all_arguments_are_constant = std::all_of(arguments.begin(), arguments.end(), [](const auto & arg) { return arg.column->isConst(); }); if (all_arguments_are_constant) - return ColumnConst::create(ColumnFunction::create(1, std::move(function), arguments), input_rows_count); + { + ColumnsWithTypeAndName arguments_resized = arguments; + for (auto & elem : arguments_resized) + elem.column = elem.column->cloneResized(1); + + return ColumnConst::create(ColumnFunction::create(1, std::move(function), arguments_resized), input_rows_count); + } else + { return ColumnFunction::create(input_rows_count, std::move(function), arguments); + } } private: diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 97628778adb..947de45f852 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -55,7 +55,7 @@ static Chunk prepareTotals(Chunk chunk) /// Skip rows except the first one. auto columns = chunk.detachColumns(); for (auto & column : columns) - column = column->cut(0, 1); + column = column->cloneResized(1); chunk.setColumns(std::move(columns), 1); } diff --git a/src/Processors/Transforms/DistinctTransform.cpp b/src/Processors/Transforms/DistinctTransform.cpp index d528303a642..53ee2c52884 100644 --- a/src/Processors/Transforms/DistinctTransform.cpp +++ b/src/Processors/Transforms/DistinctTransform.cpp @@ -64,7 +64,7 @@ void DistinctTransform::transform(Chunk & chunk) if (unlikely(key_columns_pos.empty())) { for (auto & column : columns) - column = column->cut(0, 1); + column = column->cloneResized(1); chunk.setColumns(std::move(columns), 1); stopReading(); diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 5faae03bc8f..401b3d36f83 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -327,7 +327,7 @@ static Block adaptBlockStructure(const Block & block, const Block & header) /// TODO: check that column contains the same value. /// TODO: serialize const columns. auto col = block.getByName(elem.name); - col.column = block.getByName(elem.name).column->cut(0, 1); + col.column = block.getByName(elem.name).column->cloneResized(1); column = castColumn(col, elem.type); From 4ae7d589f7f09db1488618f47c8137ac6aca0d01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 03:56:23 +0100 Subject: [PATCH 82/91] Fix error --- src/Processors/Formats/IOutputFormat.cpp | 2 +- src/Processors/Transforms/DistinctTransform.cpp | 2 +- src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 947de45f852..97628778adb 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -55,7 +55,7 @@ static Chunk prepareTotals(Chunk chunk) /// Skip rows except the first one. auto columns = chunk.detachColumns(); for (auto & column : columns) - column = column->cloneResized(1); + column = column->cut(0, 1); chunk.setColumns(std::move(columns), 1); } diff --git a/src/Processors/Transforms/DistinctTransform.cpp b/src/Processors/Transforms/DistinctTransform.cpp index 53ee2c52884..d528303a642 100644 --- a/src/Processors/Transforms/DistinctTransform.cpp +++ b/src/Processors/Transforms/DistinctTransform.cpp @@ -64,7 +64,7 @@ void DistinctTransform::transform(Chunk & chunk) if (unlikely(key_columns_pos.empty())) { for (auto & column : columns) - column = column->cloneResized(1); + column = column->cut(0, 1); chunk.setColumns(std::move(columns), 1); stopReading(); diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 401b3d36f83..5faae03bc8f 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -327,7 +327,7 @@ static Block adaptBlockStructure(const Block & block, const Block & header) /// TODO: check that column contains the same value. /// TODO: serialize const columns. auto col = block.getByName(elem.name); - col.column = block.getByName(elem.name).column->cloneResized(1); + col.column = block.getByName(elem.name).column->cut(0, 1); column = castColumn(col, elem.type); From 1e64b56a0f7f9cf0e7d209db4af33511d440954d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 04:18:32 +0100 Subject: [PATCH 83/91] Support constexpr functions in arrayFold --- src/Functions/array/arrayFold.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Functions/array/arrayFold.cpp b/src/Functions/array/arrayFold.cpp index 483a5d6404b..a9635f82db4 100644 --- a/src/Functions/array/arrayFold.cpp +++ b/src/Functions/array/arrayFold.cpp @@ -87,7 +87,9 @@ public: if (!lambda_function_with_type_and_name.column) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName()); - const auto * lambda_function = typeid_cast(lambda_function_with_type_and_name.column.get()); + auto lambda_function_materialized = lambda_function_with_type_and_name.column->convertToFullColumnIfConst(); + + const auto * lambda_function = typeid_cast(lambda_function_materialized.get()); if (!lambda_function) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "First argument for function {} must be a function", getName()); From 4334a149735742c154e9381ea9b997bb25f03dbd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 04:20:13 +0100 Subject: [PATCH 84/91] Fix test --- tests/queries/0_stateless/01284_fuzz_bits.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01284_fuzz_bits.sql b/tests/queries/0_stateless/01284_fuzz_bits.sql index 95a07c7bd44..1055d2aa580 100644 --- a/tests/queries/0_stateless/01284_fuzz_bits.sql +++ b/tests/queries/0_stateless/01284_fuzz_bits.sql @@ -18,7 +18,7 @@ FROM reinterpretAsUInt8( substring( fuzzBits( - arrayStringConcat(arrayMap(x -> toString('\0'), range(10000))), + materialize(arrayStringConcat(arrayMap(x -> toString('\0'), range(10000)))), 0.3 ), id + 1, From f2d45ba43b1d846dfb27d9f1b15b15dddb59c930 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 04:35:42 +0100 Subject: [PATCH 85/91] Fix tests --- src/Storages/MergeTree/KeyCondition.cpp | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 17723d341fb..c6497660386 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -597,12 +597,15 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( case (ActionsDAG::ActionType::COLUMN): { String name; - if (const auto * column_const = typeid_cast(node.column.get())) + if (const auto * column_const = typeid_cast(node.column.get()); + column_const && column_const->getDataType() != TypeIndex::Function) + { /// Re-generate column name for constant. - /// DAG form query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. - /// DAG from PK does not use it. This breaks matching by column name sometimes. + /// DAG from the query (with enabled analyzer) uses suffixes for constants, like 1_UInt8. + /// DAG from the PK does not use it. This breaks matching by column name sometimes. /// Ideally, we should not compare names, but DAG subtrees instead. - name = ASTLiteral(column_const->getDataColumn()[0]).getColumnName(); + name = ASTLiteral(column_const->getField()).getColumnName(); + } else name = node.result_name; From ce8ffaf5c344c924f01697c543632068895d3bb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 9 Nov 2024 23:54:37 +0100 Subject: [PATCH 86/91] Miscellaneous --- src/Analyzer/Resolve/QueryAnalyzer.cpp | 16 +++++++--------- src/Columns/ColumnFunction.cpp | 20 ++++++++++++++++++++ src/Columns/ColumnFunction.h | 10 ++-------- src/Functions/IFunction.h | 2 +- 4 files changed, 30 insertions(+), 18 deletions(-) diff --git a/src/Analyzer/Resolve/QueryAnalyzer.cpp b/src/Analyzer/Resolve/QueryAnalyzer.cpp index 4bb283cbf3e..390418494e7 100644 --- a/src/Analyzer/Resolve/QueryAnalyzer.cpp +++ b/src/Analyzer/Resolve/QueryAnalyzer.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include @@ -3023,9 +3022,10 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi argument_column.name = arguments_projection_names[function_argument_index]; /** If function argument is lambda, save lambda argument index and initialize argument type as DataTypeFunction - * where function argument types are initialized with empty array of lambda arguments size. + * where function argument types are initialized with empty arrays of lambda arguments size. */ - if (const auto * lambda_node = function_argument->as()) + const auto * lambda_node = function_argument->as(); + if (lambda_node) { size_t lambda_arguments_size = lambda_node->getArguments().getNodes().size(); argument_column.type = std::make_shared(DataTypes(lambda_arguments_size, nullptr), nullptr); @@ -3497,15 +3497,11 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi else function_base = function->build(argument_columns); - /// Do not constant fold get scalar functions - // bool disable_constant_folding = function_name == "__getScalar" || function_name == "shardNum" || - // function_name == "shardCount" || function_name == "hostName" || function_name == "tcpPort"; - /** If function is suitable for constant folding try to convert it to constant. * Example: SELECT plus(1, 1); * Result: SELECT 2; */ - if (function_base->isSuitableForConstantFolding()) // && !disable_constant_folding) + if (function_base->isSuitableForConstantFolding()) { auto result_type = function_base->getResultType(); auto executable_function = function_base->prepare(argument_columns); @@ -3514,7 +3510,9 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi if (all_arguments_constants) { - size_t num_rows = function_arguments.empty() ? 0 : argument_columns.front().column->size(); + size_t num_rows = 0; + if (!argument_columns.empty()) + num_rows = argument_columns.front().column->size(); column = executable_function->execute(argument_columns, result_type, num_rows, true); } else diff --git a/src/Columns/ColumnFunction.cpp b/src/Columns/ColumnFunction.cpp index 18c343c6ca6..cc80d04444e 100644 --- a/src/Columns/ColumnFunction.cpp +++ b/src/Columns/ColumnFunction.cpp @@ -72,6 +72,26 @@ ColumnPtr ColumnFunction::cut(size_t start, size_t length) const return ColumnFunction::create(length, function, capture, is_short_circuit_argument, is_function_compiled); } +Field ColumnFunction::operator[](size_t n) const +{ + Field res; + get(n, res); + return res; +} + +void ColumnFunction::get(size_t n, Field & res) const +{ + const size_t tuple_size = captured_columns.size(); + + res = Tuple(); + Tuple & res_tuple = res.safeGet(); + res_tuple.reserve(tuple_size); + + for (size_t i = 0; i < tuple_size; ++i) + res_tuple.push_back((*captured_columns[i].column)[n]); +} + + #if !defined(DEBUG_OR_SANITIZER_BUILD) void ColumnFunction::insertFrom(const IColumn & src, size_t n) #else diff --git a/src/Columns/ColumnFunction.h b/src/Columns/ColumnFunction.h index b62c6bf70eb..8df9e23c0e8 100644 --- a/src/Columns/ColumnFunction.h +++ b/src/Columns/ColumnFunction.h @@ -60,15 +60,9 @@ public: void appendArguments(const ColumnsWithTypeAndName & columns); ColumnWithTypeAndName reduce() const; - Field operator[](size_t) const override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName()); - } + Field operator[](size_t n) const override; - void get(size_t, Field &) const override - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot get value from {}", getName()); - } + void get(size_t n, Field & res) const override; StringRef getDataAt(size_t) const override { diff --git a/src/Functions/IFunction.h b/src/Functions/IFunction.h index c3ba4be7419..d0d6b02e69d 100644 --- a/src/Functions/IFunction.h +++ b/src/Functions/IFunction.h @@ -184,7 +184,7 @@ public: /** If function isSuitableForConstantFolding then, this method will be called during query analysis * if some arguments are constants. For example logical functions (AndFunction, OrFunction) can - * return they result based on some constant arguments. + * return the result based on some constant arguments. * Arguments are passed without modifications, useDefaultImplementationForNulls, useDefaultImplementationForNothing, * useDefaultImplementationForConstants, useDefaultImplementationForLowCardinality are not applied. */ From bc79d9bad3569a94b2755ab9ea3549ff8202148a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Nov 2024 00:01:42 +0100 Subject: [PATCH 87/91] Only with analyzer --- src/Functions/FunctionsMiscellaneous.h | 11 ++++++++--- src/Interpreters/ActionsVisitor.cpp | 2 +- src/Planner/PlannerActionsVisitor.cpp | 2 +- 3 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/Functions/FunctionsMiscellaneous.h b/src/Functions/FunctionsMiscellaneous.h index 62b43386db5..cea11cfe677 100644 --- a/src/Functions/FunctionsMiscellaneous.h +++ b/src/Functions/FunctionsMiscellaneous.h @@ -113,6 +113,7 @@ public: NamesAndTypesList lambda_arguments; String return_name; DataTypePtr return_type; + bool allow_constant_folding; }; using CapturePtr = std::shared_ptr; @@ -154,9 +155,11 @@ public: /// Consequently, it allows to treat higher order functions with constant arrays and constant captured columns /// as constant expressions. /// Consequently, it allows its usage in contexts requiring constants, such as the right hand side of IN. - bool all_arguments_are_constant = std::all_of(arguments.begin(), arguments.end(), [](const auto & arg) { return arg.column->isConst(); }); + bool constant_folding = capture->allow_constant_folding + && std::all_of(arguments.begin(), arguments.end(), + [](const auto & arg) { return arg.column->isConst(); }); - if (all_arguments_are_constant) + if (constant_folding) { ColumnsWithTypeAndName arguments_resized = arguments; for (auto & elem : arguments_resized) @@ -222,7 +225,8 @@ public: const Names & captured_names, const NamesAndTypesList & lambda_arguments, const DataTypePtr & function_return_type, - const String & expression_return_name) + const String & expression_return_name, + bool allow_constant_folding) : expression_actions(std::move(expression_actions_)) { /// Check that expression does not contain unusual actions that will break columns structure. @@ -265,6 +269,7 @@ public: .lambda_arguments = lambda_arguments, .return_name = expression_return_name, .return_type = function_return_type, + .allow_constant_folding = allow_constant_folding, }); } diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 65c3fe8cfcf..696021b418c 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -1308,7 +1308,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & String lambda_name = data.getUniqueName("__lambda"); auto function_capture = std::make_shared( - lambda_actions, captured, lambda_arguments, result_type, result_name); + lambda_actions, captured, lambda_arguments, result_type, result_name, false); data.addFunction(function_capture, captured, lambda_name); argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); diff --git a/src/Planner/PlannerActionsVisitor.cpp b/src/Planner/PlannerActionsVisitor.cpp index aa233109fa9..2cb2a242c35 100644 --- a/src/Planner/PlannerActionsVisitor.cpp +++ b/src/Planner/PlannerActionsVisitor.cpp @@ -804,7 +804,7 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi auto lambda_node_name = calculateActionNodeName(node, *planner_context); auto function_capture = std::make_shared( - lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name); + lambda_actions, captured_column_names, lambda_arguments_names_and_types, lambda_node.getExpression()->getResultType(), lambda_expression_node_name, true); // TODO: Pass IFunctionBase here not FunctionCaptureOverloadResolver. const auto * actions_node = actions_stack[level].addFunctionIfNecessary(lambda_node_name, std::move(lambda_children), function_capture); From b70f39879d3740e671d5111854daad2b9397adc1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Nov 2024 00:02:24 +0100 Subject: [PATCH 88/91] Only with analyzer --- .../0_stateless/02961_higher_order_constant_expressions.sql | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql b/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql index 47480010751..23b0b72f48f 100644 --- a/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql +++ b/tests/queries/0_stateless/02961_higher_order_constant_expressions.sql @@ -1,3 +1,5 @@ +SET enable_analyzer = 1; + SELECT arrayMap(x -> x, [1, 2, 3]) AS x, isConstant(x); SELECT arrayMap(x -> x + 1, [1, 2, 3]) AS x, isConstant(x); SELECT arrayMap(x -> x + x, [1, 2, 3]) AS x, isConstant(x); From f2d6b1db7fb8b8eee52e2a33ce6f88648fe1c863 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Nov 2024 12:39:10 +0000 Subject: [PATCH 89/91] Better --- contrib/SimSIMD | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/SimSIMD b/contrib/SimSIMD index bb0bd2e7137..fa60f1b8e35 160000 --- a/contrib/SimSIMD +++ b/contrib/SimSIMD @@ -1 +1 @@ -Subproject commit bb0bd2e7137f02c555341d7c93124ed19f3c24fb +Subproject commit fa60f1b8e3582c50978f0ae86c2ebb6c9af957f3 From b6b850a2f11301272ee28fe2274733c2cdb0c7c6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Nov 2024 17:03:35 +0000 Subject: [PATCH 90/91] Docs: Add row and byte sizes of tables --- docs/en/getting-started/example-datasets/tpch.md | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/docs/en/getting-started/example-datasets/tpch.md b/docs/en/getting-started/example-datasets/tpch.md index 5fa0d779ecd..379b92cbe9a 100644 --- a/docs/en/getting-started/example-datasets/tpch.md +++ b/docs/en/getting-started/example-datasets/tpch.md @@ -33,6 +33,21 @@ Then, generate the data. Parameter `-s` specifies the scale factor. For example, ./dbgen -s 100 ``` +Detailed table sizes with scale factor 100: + +| Table | size (in rows) | size (compressed in ClickHouse) | +|----------|----------------|---------------------------------| +| nation | 25 | 2 kB | +| region | 5 | 1 kB | +| part | 20.000.000 | 895 MB | +| supplier | 1.000.000 | 75 MB | +| partsupp | 80.000.000 | 4.37 GB | +| customer | 15.000.000 | 1.19 GB | +| orders | 150.000.000 | 6.15 GB | +| lineitem | 600.00.00 | 26.69 GB | + +(The table sizes in ClickHouse are taken from `system.tables.total_bytes` and based on below table definitions. + Now create tables in ClickHouse. We stick as closely as possible to the rules of the TPC-H specification: From 3668a78589821d89f8f7cce92e6c2bc54fff6ea3 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Sun, 10 Nov 2024 17:24:00 +0000 Subject: [PATCH 91/91] Fix spelling --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index a08143467cd..a58b5e9ff58 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -186,7 +186,6 @@ ComplexKeyCache ComplexKeyDirect ComplexKeyHashed Composable -composable ConcurrencyControlAcquired ConcurrencyControlSoftLimit Config @@ -405,12 +404,12 @@ ITION Identifiant IdentifierQuotingRule IdentifierQuotingStyle -Incrementing -IndexesAreNeighbors -InfluxDB InJodaSyntax InJodaSyntaxOrNull InJodaSyntaxOrZero +Incrementing +IndexesAreNeighbors +InfluxDB Instana IntN Integrations @@ -1475,6 +1474,7 @@ combinator combinators comparising composable +composable compressability concat concatAssumeInjective @@ -2355,6 +2355,7 @@ parsedatetime parsers partitionID partitionId +partsupp pathFull pclmulqdq pcre