From 06e203d00420197d264492c42c67539171646c71 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 14 Feb 2020 17:28:33 +0300 Subject: [PATCH 01/40] StorageStripeLog S3 support. --- .../CompressedReadBufferFromFile.cpp | 19 +++++--- .../CompressedReadBufferFromFile.h | 8 ++-- dbms/src/Disks/DiskLocal.cpp | 2 +- dbms/src/Disks/DiskLocal.h | 3 +- dbms/src/Disks/DiskMemory.cpp | 39 +++++++++++++++- dbms/src/Disks/DiskMemory.h | 30 ++++++++++++- dbms/src/Disks/DiskS3.cpp | 44 ++++++++++++------- dbms/src/Disks/DiskS3.h | 2 +- dbms/src/Disks/IDisk.h | 4 +- .../IO/MMapReadBufferFromFileDescriptor.cpp | 2 +- .../src/IO/MMapReadBufferFromFileDescriptor.h | 4 +- dbms/src/IO/ReadBufferAIO.cpp | 8 ++-- dbms/src/IO/ReadBufferAIO.h | 4 +- dbms/src/IO/ReadBufferFromFileBase.h | 8 +--- dbms/src/IO/ReadBufferFromFileDescriptor.h | 4 +- dbms/src/IO/ReadBufferFromMemory.cpp | 5 +++ dbms/src/IO/ReadBufferFromMemory.h | 2 + dbms/src/IO/ReadBufferFromS3.cpp | 6 +++ dbms/src/IO/ReadBufferFromS3.h | 1 + dbms/src/IO/SeekableReadBuffer.h | 5 +++ dbms/src/Storages/StorageLog.cpp | 25 +++++++++-- dbms/src/Storages/StorageStripeLog.cpp | 28 +++++++++--- dbms/src/Storages/StorageTinyLog.cpp | 26 ++++++++--- dbms/tests/config/disks.xml | 9 ++++ .../integration/test_disk_memory/__init__.py | 0 .../test_disk_memory/configs/config.xml | 19 -------- .../integration/test_disk_memory/test.py | 30 ------------- .../integration/test_log_family_s3/test.py | 12 ++--- .../01079_log_family_disk_memory.reference | 18 ++++++++ .../01079_log_family_disk_memory.sql | 40 +++++++++++++++++ docker/test/stateless/Dockerfile | 1 + 31 files changed, 286 insertions(+), 122 deletions(-) create mode 100644 dbms/tests/config/disks.xml delete mode 100644 dbms/tests/integration/test_disk_memory/__init__.py delete mode 100644 dbms/tests/integration/test_disk_memory/configs/config.xml delete mode 100644 dbms/tests/integration/test_disk_memory/test.py create mode 100644 dbms/tests/queries/0_stateless/01079_log_family_disk_memory.reference create mode 100644 dbms/tests/queries/0_stateless/01079_log_family_disk_memory.sql diff --git a/dbms/src/Compression/CompressedReadBufferFromFile.cpp b/dbms/src/Compression/CompressedReadBufferFromFile.cpp index 63bacde6d78..b87ab5f1528 100644 --- a/dbms/src/Compression/CompressedReadBufferFromFile.cpp +++ b/dbms/src/Compression/CompressedReadBufferFromFile.cpp @@ -1,14 +1,13 @@ #include "CompressedReadBufferFromFile.h" -#include -#include #include #include +#include +#include namespace DB { - namespace ErrorCodes { extern const int SEEK_POSITION_OUT_OF_BOUND; @@ -31,12 +30,18 @@ bool CompressedReadBufferFromFile::nextImpl() return true; } +CompressedReadBufferFromFile::CompressedReadBufferFromFile(std::unique_ptr buf) + : BufferWithOwnMemory(0), p_file_in(std::move(buf)), file_in(*p_file_in) +{ + compressed_in = &file_in; +} + CompressedReadBufferFromFile::CompressedReadBufferFromFile( const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size) - : BufferWithOwnMemory(0), - p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, mmap_threshold, buf_size)), - file_in(*p_file_in) + : BufferWithOwnMemory(0) + , p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, mmap_threshold, buf_size)) + , file_in(*p_file_in) { compressed_in = &file_in; } @@ -45,7 +50,7 @@ CompressedReadBufferFromFile::CompressedReadBufferFromFile( void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t offset_in_decompressed_block) { if (size_compressed && - offset_in_compressed_file == file_in.getPositionInFile() - size_compressed && + offset_in_compressed_file == file_in.getPosition() - size_compressed && offset_in_decompressed_block <= working_buffer.size()) { bytes += offset(); diff --git a/dbms/src/Compression/CompressedReadBufferFromFile.h b/dbms/src/Compression/CompressedReadBufferFromFile.h index 641e3d6ed1b..33727909a69 100644 --- a/dbms/src/Compression/CompressedReadBufferFromFile.h +++ b/dbms/src/Compression/CompressedReadBufferFromFile.h @@ -1,10 +1,10 @@ #pragma once -#include "CompressedReadBufferBase.h" -#include -#include #include +#include +#include #include +#include "CompressedReadBufferBase.h" namespace DB @@ -29,6 +29,8 @@ private: bool nextImpl() override; public: + CompressedReadBufferFromFile(std::unique_ptr buf); + CompressedReadBufferFromFile( const std::string & path, size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Disks/DiskLocal.cpp b/dbms/src/Disks/DiskLocal.cpp index 421583a4258..74300cf61b5 100644 --- a/dbms/src/Disks/DiskLocal.cpp +++ b/dbms/src/Disks/DiskLocal.cpp @@ -200,7 +200,7 @@ void DiskLocal::copyFile(const String & from_path, const String & to_path) Poco::File(disk_path + from_path).copyTo(disk_path + to_path); } -std::unique_ptr DiskLocal::readFile(const String & path, size_t buf_size) const +std::unique_ptr DiskLocal::readFile(const String & path, size_t buf_size) const { return std::make_unique(disk_path + path, buf_size); } diff --git a/dbms/src/Disks/DiskLocal.h b/dbms/src/Disks/DiskLocal.h index 1a9a2e8f6cd..8d9d51fc384 100644 --- a/dbms/src/Disks/DiskLocal.h +++ b/dbms/src/Disks/DiskLocal.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -66,7 +67,7 @@ public: void copyFile(const String & from_path, const String & to_path) override; - std::unique_ptr readFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) const override; + std::unique_ptr readFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) const override; std::unique_ptr writeFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite) override; diff --git a/dbms/src/Disks/DiskMemory.cpp b/dbms/src/Disks/DiskMemory.cpp index cc61b0f870d..30509d42b43 100644 --- a/dbms/src/Disks/DiskMemory.cpp +++ b/dbms/src/Disks/DiskMemory.cpp @@ -15,6 +15,7 @@ namespace ErrorCodes extern const int FILE_ALREADY_EXISTS; extern const int DIRECTORY_DOESNT_EXIST; extern const int CANNOT_DELETE_DIRECTORY; + extern const int CANNOT_SEEK_THROUGH_FILE; } @@ -37,8 +38,42 @@ private: std::vector::iterator iter; }; +bool ReadIndirectBuffer::nextImpl() +{ + if (!initialized) + { + initialized = true; + + internal_buffer = buf.buffer(); + working_buffer = internal_buffer; + + return true; + } + + return false; +} + +off_t ReadIndirectBuffer::seek(off_t off, int whence) +{ + if (whence != SEEK_SET) + throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + + off_t result = buf.seek(off, whence); + pos = buf.position(); + + return result; +} + +off_t ReadIndirectBuffer::getPosition() +{ + return pos - working_buffer.begin(); +} + void WriteIndirectBuffer::finalize() { + if (isFinished()) + return; + next(); WriteBufferFromVector::finalize(); @@ -249,7 +284,7 @@ void DiskMemory::copyFile(const String & /*from_path*/, const String & /*to_path throw Exception("Method copyFile is not implemented for memory disks", ErrorCodes::NOT_IMPLEMENTED); } -std::unique_ptr DiskMemory::readFile(const String & path, size_t /*buf_size*/) const +std::unique_ptr DiskMemory::readFile(const String & path, size_t /*buf_size*/) const { std::lock_guard lock(mutex); @@ -257,7 +292,7 @@ std::unique_ptr DiskMemory::readFile(const String & path, si if (iter == files.end()) throw Exception("File '" + path + "' does not exist", ErrorCodes::FILE_DOESNT_EXIST); - return std::make_unique(iter->second.data); + return std::make_unique(path, iter->second.data); } std::unique_ptr DiskMemory::writeFile(const String & path, size_t /*buf_size*/, WriteMode mode) diff --git a/dbms/src/Disks/DiskMemory.h b/dbms/src/Disks/DiskMemory.h index 6bbd13d2988..dace7c47da3 100644 --- a/dbms/src/Disks/DiskMemory.h +++ b/dbms/src/Disks/DiskMemory.h @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include namespace DB @@ -13,7 +15,31 @@ class DiskMemory; class ReadBuffer; class WriteBuffer; -// This class is responsible to update files metadata after buffer is finalized. +/// Adapter with actual behaviour as ReadBufferFromString. +class ReadIndirectBuffer : public ReadBufferFromFileBase +{ +public: + ReadIndirectBuffer(String path_, const String & data_) + : ReadBufferFromFileBase(), buf(ReadBufferFromString(data_)), path(std::move(path_)) + { + } + + std::string getFileName() const override { return path; } + +private: + bool nextImpl() override; + +public: + off_t seek(off_t off, int whence) override; + off_t getPosition() override; + +private: + bool initialized = false; + ReadBufferFromString buf; + String path; +}; + +/// This class is responsible to update files metadata after buffer is finalized. class WriteIndirectBuffer : public WriteBufferFromOwnString { public: @@ -76,7 +102,7 @@ public: void copyFile(const String & from_path, const String & to_path) override; - std::unique_ptr readFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) const override; + std::unique_ptr readFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) const override; std::unique_ptr writeFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite) override; diff --git a/dbms/src/Disks/DiskS3.cpp b/dbms/src/Disks/DiskS3.cpp index 25701507884..7142b6a3643 100644 --- a/dbms/src/Disks/DiskS3.cpp +++ b/dbms/src/Disks/DiskS3.cpp @@ -137,17 +137,17 @@ namespace // Reads data from S3. // It supports reading from multiple S3 paths that resides in Metadata. - class ReadIndirectBufferFromS3 : public BufferWithOwnMemory + class ReadIndirectBufferFromS3 : public ReadBufferFromFileBase { public: ReadIndirectBufferFromS3( std::shared_ptr client_ptr_, const String & bucket_, Metadata metadata_, size_t buf_size_) - : BufferWithOwnMemory(buf_size_) + : ReadBufferFromFileBase() , client_ptr(std::move(client_ptr_)) , bucket(bucket_) , metadata(std::move(metadata_)) , buf_size(buf_size_) - , offset(0) + , absolute_position(0) , initialized(false) , current_buf_idx(0) , current_buf(nullptr) @@ -156,9 +156,6 @@ namespace off_t seek(off_t offset_, int whence) override { - if (initialized) - throw Exception("Seek is allowed only before first read attempt from the buffer.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - if (whence != SEEK_SET) throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); @@ -169,14 +166,23 @@ namespace + std::to_string(offset_) + ", Max: " + std::to_string(metadata.total_size), ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); - offset = offset_; + absolute_position = offset_; - return offset; + /// TODO: Do not re-initialize buffer if current position within working buffer. + current_buf = initialize(); + pos = working_buffer.end(); + + return absolute_position; } + off_t getPosition() override { return absolute_position - available(); } + + std::string getFileName() const override { return metadata.metadata_file_path; } + private: std::unique_ptr initialize() { + size_t offset = absolute_position; for (UInt32 i = 0; i < metadata.s3_objects_count; ++i) { current_buf_idx = i; @@ -190,6 +196,7 @@ namespace } offset -= size; } + initialized = true; return nullptr; } @@ -199,14 +206,13 @@ namespace if (!initialized) { current_buf = initialize(); - - initialized = true; } // If current buffer has remaining data - use it. if (current_buf && current_buf->next()) { working_buffer = current_buf->buffer(); + absolute_position += working_buffer.size(); return true; } @@ -219,6 +225,7 @@ namespace current_buf = std::make_unique(client_ptr, bucket, path, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); + absolute_position += working_buffer.size(); return true; } @@ -229,7 +236,7 @@ namespace Metadata metadata; size_t buf_size; - size_t offset; + size_t absolute_position = 0; bool initialized; UInt32 current_buf_idx; std::unique_ptr current_buf; @@ -337,8 +344,13 @@ private: }; -DiskS3::DiskS3(String name_, std::shared_ptr client_, String bucket_, String s3_root_path_, - String metadata_path_, size_t min_upload_part_size_) +DiskS3::DiskS3( + String name_, + std::shared_ptr client_, + String bucket_, + String s3_root_path_, + String metadata_path_, + size_t min_upload_part_size_) : name(std::move(name_)) , client(std::move(client_)) , bucket(std::move(bucket_)) @@ -445,7 +457,7 @@ void DiskS3::copyFile(const String & from_path, const String & to_path) to.save(); } -std::unique_ptr DiskS3::readFile(const String & path, size_t buf_size) const +std::unique_ptr DiskS3::readFile(const String & path, size_t buf_size) const { Metadata metadata(metadata_path + path); @@ -628,8 +640,8 @@ void registerDiskS3(DiskFactory & factory) String metadata_path = context.getPath() + "disks/" + name + "/"; - auto s3disk = std::make_shared(name, client, uri.bucket, uri.key, metadata_path, - context.getSettingsRef().s3_min_upload_part_size); + auto s3disk + = std::make_shared(name, client, uri.bucket, uri.key, metadata_path, context.getSettingsRef().s3_min_upload_part_size); /// This code is used only to check access to the corresponding disk. checkWriteAccess(s3disk); diff --git a/dbms/src/Disks/DiskS3.h b/dbms/src/Disks/DiskS3.h index bafeb41a261..759e2f347d9 100644 --- a/dbms/src/Disks/DiskS3.h +++ b/dbms/src/Disks/DiskS3.h @@ -62,7 +62,7 @@ public: void copyFile(const String & from_path, const String & to_path) override; - std::unique_ptr readFile(const String & path, size_t buf_size) const override; + std::unique_ptr readFile(const String & path, size_t buf_size) const override; std::unique_ptr writeFile(const String & path, size_t buf_size, WriteMode mode) override; diff --git a/dbms/src/Disks/IDisk.h b/dbms/src/Disks/IDisk.h index 76e09dda983..ff920897ebf 100644 --- a/dbms/src/Disks/IDisk.h +++ b/dbms/src/Disks/IDisk.h @@ -25,7 +25,7 @@ using DiskDirectoryIteratorPtr = std::unique_ptr; class IReservation; using ReservationPtr = std::unique_ptr; -class SeekableReadBuffer; +class ReadBufferFromFileBase; class WriteBuffer; /** @@ -122,7 +122,7 @@ public: virtual void copyFile(const String & from_path, const String & to_path) = 0; /// Open the file for read and return SeekableReadBuffer object. - virtual std::unique_ptr readFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) const = 0; + virtual std::unique_ptr readFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) const = 0; /// Open the file for write and return WriteBuffer object. virtual std::unique_ptr writeFile(const String & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, WriteMode mode = WriteMode::Rewrite) = 0; diff --git a/dbms/src/IO/MMapReadBufferFromFileDescriptor.cpp b/dbms/src/IO/MMapReadBufferFromFileDescriptor.cpp index 2d1ddba5f58..027b95bc022 100644 --- a/dbms/src/IO/MMapReadBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/MMapReadBufferFromFileDescriptor.cpp @@ -100,7 +100,7 @@ int MMapReadBufferFromFileDescriptor::getFD() const return fd; } -off_t MMapReadBufferFromFileDescriptor::getPositionInFile() +off_t MMapReadBufferFromFileDescriptor::getPosition() { return count(); } diff --git a/dbms/src/IO/MMapReadBufferFromFileDescriptor.h b/dbms/src/IO/MMapReadBufferFromFileDescriptor.h index fb6d3651b41..e409e9d2d0c 100644 --- a/dbms/src/IO/MMapReadBufferFromFileDescriptor.h +++ b/dbms/src/IO/MMapReadBufferFromFileDescriptor.h @@ -32,9 +32,9 @@ public: /// unmap memory before call to destructor void finish(); - off_t getPositionInFile() override; + off_t getPosition() override; std::string getFileName() const override; - int getFD() const override; + int getFD() const; private: size_t length = 0; diff --git a/dbms/src/IO/ReadBufferAIO.cpp b/dbms/src/IO/ReadBufferAIO.cpp index f7d3cd475af..37a6aae5c05 100644 --- a/dbms/src/IO/ReadBufferAIO.cpp +++ b/dbms/src/IO/ReadBufferAIO.cpp @@ -163,17 +163,17 @@ off_t ReadBufferAIO::seek(off_t off, int whence) { if (off >= 0) { - if (off > (std::numeric_limits::max() - getPositionInFile())) + if (off > (std::numeric_limits::max() - getPosition())) throw Exception("SEEK_CUR overflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } - else if (off < -getPositionInFile()) + else if (off < -getPosition()) throw Exception("SEEK_CUR underflow", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - new_pos_in_file = getPositionInFile() + off; + new_pos_in_file = getPosition() + off; } else throw Exception("ReadBufferAIO::seek expects SEEK_SET or SEEK_CUR as whence", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - if (new_pos_in_file != getPositionInFile()) + if (new_pos_in_file != getPosition()) { off_t first_read_pos_in_file = first_unread_pos_in_file - static_cast(working_buffer.size()); if (hasPendingData() && (new_pos_in_file >= first_read_pos_in_file) && (new_pos_in_file <= first_unread_pos_in_file)) diff --git a/dbms/src/IO/ReadBufferAIO.h b/dbms/src/IO/ReadBufferAIO.h index 446034e4733..2dd11b44095 100644 --- a/dbms/src/IO/ReadBufferAIO.h +++ b/dbms/src/IO/ReadBufferAIO.h @@ -36,9 +36,9 @@ public: ReadBufferAIO & operator=(const ReadBufferAIO &) = delete; void setMaxBytes(size_t max_bytes_read_); - off_t getPositionInFile() override { return first_unread_pos_in_file - (working_buffer.end() - pos); } + off_t getPosition() override { return first_unread_pos_in_file - (working_buffer.end() - pos); } std::string getFileName() const override { return filename; } - int getFD() const override { return fd; } + int getFD() const { return fd; } off_t seek(off_t off, int whence) override; diff --git a/dbms/src/IO/ReadBufferFromFileBase.h b/dbms/src/IO/ReadBufferFromFileBase.h index d9e26d80405..58f54904e85 100644 --- a/dbms/src/IO/ReadBufferFromFileBase.h +++ b/dbms/src/IO/ReadBufferFromFileBase.h @@ -1,17 +1,16 @@ #pragma once -#include #include #include +#include #include -#include #include +#include #include #include "SeekableReadBuffer.h" namespace DB { - class ReadBufferFromFileBase : public BufferWithOwnMemory { public: @@ -19,9 +18,7 @@ public: ReadBufferFromFileBase(size_t buf_size, char * existing_memory, size_t alignment); ReadBufferFromFileBase(ReadBufferFromFileBase &&) = default; ~ReadBufferFromFileBase() override; - virtual off_t getPositionInFile() = 0; virtual std::string getFileName() const = 0; - virtual int getFD() const = 0; /// It is possible to get information about the time of each reading. struct ProfileInfo @@ -43,7 +40,6 @@ public: protected: ProfileCallback profile_callback; clockid_t clock_type{}; - }; } diff --git a/dbms/src/IO/ReadBufferFromFileDescriptor.h b/dbms/src/IO/ReadBufferFromFileDescriptor.h index b19a04aef18..dc2d581ebf1 100644 --- a/dbms/src/IO/ReadBufferFromFileDescriptor.h +++ b/dbms/src/IO/ReadBufferFromFileDescriptor.h @@ -27,12 +27,12 @@ public: ReadBufferFromFileDescriptor(ReadBufferFromFileDescriptor &&) = default; - int getFD() const override + int getFD() const { return fd; } - off_t getPositionInFile() override + off_t getPosition() override { return pos_in_file - (working_buffer.end() - pos); } diff --git a/dbms/src/IO/ReadBufferFromMemory.cpp b/dbms/src/IO/ReadBufferFromMemory.cpp index 2097c78c34e..98c39c833b0 100644 --- a/dbms/src/IO/ReadBufferFromMemory.cpp +++ b/dbms/src/IO/ReadBufferFromMemory.cpp @@ -43,4 +43,9 @@ off_t ReadBufferFromMemory::seek(off_t offset, int whence) throw Exception("Only SEEK_SET and SEEK_CUR seek modes allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); } +off_t ReadBufferFromMemory::getPosition() +{ + return pos - working_buffer.begin(); +} + } diff --git a/dbms/src/IO/ReadBufferFromMemory.h b/dbms/src/IO/ReadBufferFromMemory.h index ba79c9e56bf..c4330f312f9 100644 --- a/dbms/src/IO/ReadBufferFromMemory.h +++ b/dbms/src/IO/ReadBufferFromMemory.h @@ -25,6 +25,8 @@ public: } off_t seek(off_t off, int whence) override; + + off_t getPosition() override; }; } diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index a77d742370c..1fa318cf629 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -58,6 +58,12 @@ off_t ReadBufferFromS3::seek(off_t offset_, int whence) return offset; } + +off_t ReadBufferFromS3::getPosition() +{ + return offset + count(); +} + std::unique_ptr ReadBufferFromS3::initialize() { LOG_TRACE(log, "Read S3 object. Bucket: " + bucket + ", Key: " + key + ", Offset: " + std::to_string(offset)); diff --git a/dbms/src/IO/ReadBufferFromS3.h b/dbms/src/IO/ReadBufferFromS3.h index f04794d4458..15d3b4f0beb 100644 --- a/dbms/src/IO/ReadBufferFromS3.h +++ b/dbms/src/IO/ReadBufferFromS3.h @@ -45,6 +45,7 @@ public: bool nextImpl() override; off_t seek(off_t off, int whence) override; + off_t getPosition() override; private: std::unique_ptr initialize(); diff --git a/dbms/src/IO/SeekableReadBuffer.h b/dbms/src/IO/SeekableReadBuffer.h index 7deb30ed28f..be06887e61f 100644 --- a/dbms/src/IO/SeekableReadBuffer.h +++ b/dbms/src/IO/SeekableReadBuffer.h @@ -20,6 +20,11 @@ public: * @return New position from the begging of underlying buffer / file. */ virtual off_t seek(off_t off, int whence) = 0; + + /** + * @return Offset from the begging of underlying buffer / file corresponds to buffer current position. + */ + virtual off_t getPosition() = 0; }; } diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 549dd352a48..da3023a5776 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -5,6 +5,9 @@ #include #include +#include + +#include #include #include #include @@ -18,6 +21,7 @@ #include #include +#include #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" @@ -88,7 +92,7 @@ private: plain->seek(offset, SEEK_SET); } - std::unique_ptr plain; + std::unique_ptr plain; CompressedReadBuffer compressed; }; @@ -625,13 +629,26 @@ void registerStorageLog(StorageFactory & factory) { factory.registerStorage("Log", [](const StorageFactory::Arguments & args) { - if (!args.engine_args.empty()) + ASTs & engine_args = args.engine_args; + + if (engine_args.size() > 1) throw Exception( - "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + "Engine " + args.engine_name + " requires 0 or 1 arguments: [disk_name] (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + for (size_t i = 0; i < engine_args.size(); ++i) + engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); + + DiskPtr disk = args.context.getDefaultDisk(); + + if (engine_args.size() == 1) + { + String disk_name = engine_args[0]->as().value.safeGet(); + disk = args.context.getDisk(disk_name); + } + return StorageLog::create( - args.context.getDefaultDisk(), args.relative_data_path, args.table_id, args.columns, args.constraints, + disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.context.getSettings().max_compress_block_size); }); } diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 5aa3b52f969..9912cf47a89 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -8,7 +8,6 @@ #include #include -#include #include #include #include @@ -26,8 +25,10 @@ #include -#include +#include +#include #include +#include namespace DB @@ -120,7 +121,7 @@ private: String data_file_path = storage.table_path + "data.bin"; size_t buffer_size = std::min(max_read_buffer_size, storage.disk->getFileSize(data_file_path)); - data_in.emplace(fullPath(storage.disk, data_file_path), 0, 0, buffer_size); + data_in.emplace(storage.disk->readFile(data_file_path, buffer_size)); block_in.emplace(*data_in, 0, index_begin, index_end); } } @@ -253,7 +254,7 @@ BlockInputStreams StorageStripeLog::read( if (!disk->exists(index_file)) return { std::make_shared(getSampleBlockForColumns(column_names)) }; - CompressedReadBufferFromFile index_in(fullPath(disk, index_file), 0, 0, 0, INDEX_BUFFER_SIZE); + CompressedReadBufferFromFile index_in(disk->readFile(index_file, INDEX_BUFFER_SIZE)); std::shared_ptr index{std::make_shared(index_in, column_names_set)}; BlockInputStreams res; @@ -307,13 +308,26 @@ void registerStorageStripeLog(StorageFactory & factory) { factory.registerStorage("StripeLog", [](const StorageFactory::Arguments & args) { - if (!args.engine_args.empty()) + ASTs & engine_args = args.engine_args; + + if (engine_args.size() > 1) throw Exception( - "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + "Engine " + args.engine_name + " requires 0 or 1 arguments: [disk_name] (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + for (size_t i = 0; i < engine_args.size(); ++i) + engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); + + DiskPtr disk = args.context.getDefaultDisk(); + + if (engine_args.size() == 1) + { + String disk_name = engine_args[0]->as().value.safeGet(); + disk = args.context.getDisk(disk_name); + } + return StorageStripeLog::create( - args.context.getDefaultDisk(), args.relative_data_path, args.table_id, args.columns, args.constraints, + disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size); }); } diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index aeb90399816..db90ba3e41d 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include #include @@ -25,9 +26,11 @@ #include -#include -#include +#include +#include #include +#include +#include #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" @@ -436,13 +439,26 @@ void registerStorageTinyLog(StorageFactory & factory) { factory.registerStorage("TinyLog", [](const StorageFactory::Arguments & args) { - if (!args.engine_args.empty()) + ASTs & engine_args = args.engine_args; + + if (engine_args.size() > 1) throw Exception( - "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", + "Engine " + args.engine_name + " requires 0 or 1 arguments: [disk_name] (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + for (size_t i = 0; i < engine_args.size(); ++i) + engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); + + DiskPtr disk = args.context.getDefaultDisk(); + + if (engine_args.size() == 1) + { + String disk_name = engine_args[0]->as().value.safeGet(); + disk = args.context.getDisk(disk_name); + } + return StorageTinyLog::create( - args.context.getDefaultDisk(), args.relative_data_path, args.table_id, args.columns, args.constraints, + disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size); }); } diff --git a/dbms/tests/config/disks.xml b/dbms/tests/config/disks.xml new file mode 100644 index 00000000000..1387c7f76f1 --- /dev/null +++ b/dbms/tests/config/disks.xml @@ -0,0 +1,9 @@ + + + + + memory + + + + \ No newline at end of file diff --git a/dbms/tests/integration/test_disk_memory/__init__.py b/dbms/tests/integration/test_disk_memory/__init__.py deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/tests/integration/test_disk_memory/configs/config.xml b/dbms/tests/integration/test_disk_memory/configs/config.xml deleted file mode 100644 index 4573ddf9cfd..00000000000 --- a/dbms/tests/integration/test_disk_memory/configs/config.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - trace - /var/log/clickhouse-server/clickhouse-server.log - /var/log/clickhouse-server/clickhouse-server.err.log - 1000M - 10 - /var/log/clickhouse-server/stderr.log - /var/log/clickhouse-server/stdout.log - - - - - - memory - - - - diff --git a/dbms/tests/integration/test_disk_memory/test.py b/dbms/tests/integration/test_disk_memory/test.py deleted file mode 100644 index ea9309fcbe1..00000000000 --- a/dbms/tests/integration/test_disk_memory/test.py +++ /dev/null @@ -1,30 +0,0 @@ -import pytest - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV - -cluster = ClickHouseCluster(__file__) -node = cluster.add_instance("node", main_configs=['configs/config.xml']) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - yield cluster - finally: - cluster.shutdown() - - -def test_tinylog(started_cluster): - node.query('''CREATE DATABASE IF NOT EXISTS test''') - - node.query('''CREATE TABLE test.tinylog (s String, n UInt8) ENGINE = TinyLog''') - - node.query('''INSERT INTO test.tinylog SELECT toString(number), number * 2 FROM system.numbers LIMIT 5''') - assert TSV(node.query('''SELECT * FROM test.tinylog''')) == TSV('0\t0\n1\t2\n2\t4\n3\t6\n4\t8') - - node.query('''TRUNCATE TABLE test.tinylog''') - assert TSV(node.query('''SELECT * FROM test.tinylog''')) == TSV('') - - node.query('''DROP TABLE test.tinylog''') diff --git a/dbms/tests/integration/test_log_family_s3/test.py b/dbms/tests/integration/test_log_family_s3/test.py index 09002a95f2c..63272a7c58e 100644 --- a/dbms/tests/integration/test_log_family_s3/test.py +++ b/dbms/tests/integration/test_log_family_s3/test.py @@ -34,8 +34,10 @@ def cluster(): cluster.shutdown() -@pytest.mark.parametrize("log_engine,files_overhead", [("TinyLog", 1), ("Log", 2)]) -def test_log_family_s3(cluster, log_engine, files_overhead): +@pytest.mark.parametrize( + "log_engine,files_overhead,files_overhead_per_insert", + [("TinyLog", 1, 1), ("Log", 2, 1), ("StripeLog", 1, 2)]) +def test_log_family_s3(cluster, log_engine, files_overhead, files_overhead_per_insert): node = cluster.instances["node"] minio = cluster.minio_client @@ -43,15 +45,15 @@ def test_log_family_s3(cluster, log_engine, files_overhead): node.query("INSERT INTO s3_test SELECT number FROM numbers(5)") assert node.query("SELECT * FROM s3_test") == "0\n1\n2\n3\n4\n" - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 1 + files_overhead + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == files_overhead_per_insert + files_overhead node.query("INSERT INTO s3_test SELECT number + 5 FROM numbers(3)") assert node.query("SELECT * FROM s3_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n" - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 2 + files_overhead + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == files_overhead_per_insert * 2 + files_overhead node.query("INSERT INTO s3_test SELECT number + 8 FROM numbers(1)") assert node.query("SELECT * FROM s3_test order by id") == "0\n1\n2\n3\n4\n5\n6\n7\n8\n" - assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 3 + files_overhead + assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == files_overhead_per_insert * 3 + files_overhead node.query("TRUNCATE TABLE s3_test") assert len(list(minio.list_objects(cluster.minio_bucket, 'data/'))) == 0 diff --git a/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.reference b/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.reference new file mode 100644 index 00000000000..9d8e6d18e1d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.reference @@ -0,0 +1,18 @@ +0 +0 +1 +0 +1 +2 +0 +0 +1 +0 +1 +2 +0 +0 +1 +0 +1 +2 diff --git a/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.sql b/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.sql new file mode 100644 index 00000000000..bbc75f1d3bd --- /dev/null +++ b/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.sql @@ -0,0 +1,40 @@ +DROP TABLE IF EXISTS log; + +CREATE TABLE log (x UInt8) ENGINE = StripeLog ('disk_memory'); + +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (0); +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (1); +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (2); +SELECT * FROM log ORDER BY x; + +TRUNCATE TABLE log; +DROP TABLE log; + +CREATE TABLE log (x UInt8) ENGINE = TinyLog ('disk_memory'); + +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (0); +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (1); +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (2); +SELECT * FROM log ORDER BY x; + +TRUNCATE TABLE log; +DROP TABLE log; + +CREATE TABLE log (x UInt8) ENGINE = Log ('disk_memory'); + +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (0); +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (1); +SELECT * FROM log ORDER BY x; +INSERT INTO log VALUES (2); +SELECT * FROM log ORDER BY x; + +TRUNCATE TABLE log; +DROP TABLE log; diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile index 51e92bfa5a3..c30f63c0c13 100644 --- a/docker/test/stateless/Dockerfile +++ b/docker/test/stateless/Dockerfile @@ -56,5 +56,6 @@ CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \ ln -s /usr/share/clickhouse-test/config/strings_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/decimals_dictionary.xml /etc/clickhouse-server/; \ ln -s /usr/share/clickhouse-test/config/macros.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/disks.xml /etc/clickhouse-server/config.d/; \ service zookeeper start; sleep 5; \ service clickhouse-server start && sleep 5 && clickhouse-test --testname --shard --zookeeper $ADDITIONAL_OPTIONS $SKIP_TESTS_OPTION 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt From f3bec50d2298d01a1ae4ff24b8cc8838d79a881c Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 17 Feb 2020 13:49:36 +0300 Subject: [PATCH 02/40] Fixed getPositionInFile() method usages. --- dbms/src/IO/tests/read_buffer_aio.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/IO/tests/read_buffer_aio.cpp b/dbms/src/IO/tests/read_buffer_aio.cpp index adb2f7d5458..04aafd1fee2 100644 --- a/dbms/src/IO/tests/read_buffer_aio.cpp +++ b/dbms/src/IO/tests/read_buffer_aio.cpp @@ -283,14 +283,14 @@ bool test6(const std::string & filename, const std::string & buf) DB::ReadBufferAIO in(filename, 3 * DEFAULT_AIO_FILE_BLOCK_SIZE); - if (in.getPositionInFile() != 0) + if (in.getPosition() != 0) return false; size_t count = in.read(newbuf.data(), newbuf.length()); if (count != newbuf.length()) return false; - if (static_cast(in.getPositionInFile()) != buf.length()) + if (static_cast(in.getPosition()) != buf.length()) return false; return true; @@ -646,7 +646,7 @@ bool test20(const std::string & filename, const std::string & buf) return false; } - (void) in.getPositionInFile(); + (void) in.getPosition(); { std::string newbuf; From cceaf69264b043958a9a3c4bf081eb4ae3ade4ed Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 17 Feb 2020 13:51:16 +0300 Subject: [PATCH 03/40] Change log_family_disk_memory test number. --- ...sk_memory.reference => 01082_log_family_disk_memory.reference} | 0 ...og_family_disk_memory.sql => 01082_log_family_disk_memory.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename dbms/tests/queries/0_stateless/{01079_log_family_disk_memory.reference => 01082_log_family_disk_memory.reference} (100%) rename dbms/tests/queries/0_stateless/{01079_log_family_disk_memory.sql => 01082_log_family_disk_memory.sql} (100%) diff --git a/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.reference b/dbms/tests/queries/0_stateless/01082_log_family_disk_memory.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01079_log_family_disk_memory.reference rename to dbms/tests/queries/0_stateless/01082_log_family_disk_memory.reference diff --git a/dbms/tests/queries/0_stateless/01079_log_family_disk_memory.sql b/dbms/tests/queries/0_stateless/01082_log_family_disk_memory.sql similarity index 100% rename from dbms/tests/queries/0_stateless/01079_log_family_disk_memory.sql rename to dbms/tests/queries/0_stateless/01082_log_family_disk_memory.sql From cf474e1f09becd101c84baa20188237da1ef20fb Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 17 Feb 2020 18:00:39 +0300 Subject: [PATCH 04/40] Fixed DiskMemory->readFile() test. --- dbms/src/Disks/DiskMemory.cpp | 8 +++++--- dbms/src/Disks/tests/gtest_disk.cpp | 8 +++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/dbms/src/Disks/DiskMemory.cpp b/dbms/src/Disks/DiskMemory.cpp index 30509d42b43..776ea264940 100644 --- a/dbms/src/Disks/DiskMemory.cpp +++ b/dbms/src/Disks/DiskMemory.cpp @@ -55,11 +55,13 @@ bool ReadIndirectBuffer::nextImpl() off_t ReadIndirectBuffer::seek(off_t off, int whence) { - if (whence != SEEK_SET) - throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + /// Synchronize position in working buffer and string buffer. + buf.seek(offset(), SEEK_SET); + /// Seek string buffer position. off_t result = buf.seek(off, whence); - pos = buf.position(); + + pos = working_buffer.begin() + result; return result; } diff --git a/dbms/src/Disks/tests/gtest_disk.cpp b/dbms/src/Disks/tests/gtest_disk.cpp index 7c487c9a428..ea67545e5fa 100644 --- a/dbms/src/Disks/tests/gtest_disk.cpp +++ b/dbms/src/Disks/tests/gtest_disk.cpp @@ -108,11 +108,13 @@ TYPED_TEST(DiskTest, readFile) // Test SEEK_SET { - DB::String data; + String buf(4, '0'); std::unique_ptr in = disk->readFile("test_file"); + in->seek(5, SEEK_SET); - readString(data, *in); - EXPECT_EQ("data", data); + + in->readStrict(buf.data(), 4); + EXPECT_EQ("data", buf); } // Test SEEK_CUR From e849654628dbd767a966fac3d622ee2868e18ff9 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Mon, 17 Feb 2020 23:08:35 +0300 Subject: [PATCH 05/40] Reworked ReadIndirectBuffer for DiskMemory. --- dbms/src/Disks/DiskMemory.cpp | 62 ++++++++++++++++++++++------------- dbms/src/Disks/DiskMemory.h | 11 +------ 2 files changed, 40 insertions(+), 33 deletions(-) diff --git a/dbms/src/Disks/DiskMemory.cpp b/dbms/src/Disks/DiskMemory.cpp index 776ea264940..2b249858f6b 100644 --- a/dbms/src/Disks/DiskMemory.cpp +++ b/dbms/src/Disks/DiskMemory.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes extern const int DIRECTORY_DOESNT_EXIST; extern const int CANNOT_DELETE_DIRECTORY; extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int SEEK_POSITION_OUT_OF_BOUND; } @@ -38,32 +39,39 @@ private: std::vector::iterator iter; }; -bool ReadIndirectBuffer::nextImpl() +off_t ReadIndirectBuffer::seek(off_t offset, int whence) { - if (!initialized) + if (whence == SEEK_SET) { - initialized = true; - - internal_buffer = buf.buffer(); - working_buffer = internal_buffer; - - return true; + if (offset >= 0 && working_buffer.begin() + offset < working_buffer.end()) + { + pos = working_buffer.begin() + offset; + return size_t(pos - working_buffer.begin()); + } + else + throw Exception( + "Seek position is out of bounds. " + "Offset: " + + std::to_string(offset) + ", Max: " + std::to_string(size_t(working_buffer.end() - working_buffer.begin())), + ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); } - - return false; -} - -off_t ReadIndirectBuffer::seek(off_t off, int whence) -{ - /// Synchronize position in working buffer and string buffer. - buf.seek(offset(), SEEK_SET); - - /// Seek string buffer position. - off_t result = buf.seek(off, whence); - - pos = working_buffer.begin() + result; - - return result; + else if (whence == SEEK_CUR) + { + Position new_pos = pos + offset; + if (new_pos >= working_buffer.begin() && new_pos < working_buffer.end()) + { + pos = new_pos; + return size_t(pos - working_buffer.begin()); + } + else + throw Exception( + "Seek position is out of bounds. " + "Offset: " + + std::to_string(offset) + ", Max: " + std::to_string(size_t(working_buffer.end() - working_buffer.begin())), + ErrorCodes::SEEK_POSITION_OUT_OF_BOUND); + } + else + throw Exception("Only SEEK_SET and SEEK_CUR seek modes allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); } off_t ReadIndirectBuffer::getPosition() @@ -71,6 +79,14 @@ off_t ReadIndirectBuffer::getPosition() return pos - working_buffer.begin(); } +ReadIndirectBuffer::ReadIndirectBuffer(String path_, const String & data_) + : ReadBufferFromFileBase(), buf(ReadBufferFromString(data_)), path(std::move(path_)) +{ + internal_buffer = buf.buffer(); + working_buffer = internal_buffer; + pos = working_buffer.begin(); +} + void WriteIndirectBuffer::finalize() { if (isFinished()) diff --git a/dbms/src/Disks/DiskMemory.h b/dbms/src/Disks/DiskMemory.h index dace7c47da3..2dd93e58549 100644 --- a/dbms/src/Disks/DiskMemory.h +++ b/dbms/src/Disks/DiskMemory.h @@ -19,22 +19,13 @@ class WriteBuffer; class ReadIndirectBuffer : public ReadBufferFromFileBase { public: - ReadIndirectBuffer(String path_, const String & data_) - : ReadBufferFromFileBase(), buf(ReadBufferFromString(data_)), path(std::move(path_)) - { - } + ReadIndirectBuffer(String path_, const String & data_); std::string getFileName() const override { return path; } - -private: - bool nextImpl() override; - -public: off_t seek(off_t off, int whence) override; off_t getPosition() override; private: - bool initialized = false; ReadBufferFromString buf; String path; }; From a8cffd55aab25b5cbed47f13bc66ee27e07c11ba Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 18 Feb 2020 12:51:22 +0300 Subject: [PATCH 06/40] Formatting issues. --- dbms/src/Disks/DiskMemory.cpp | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/dbms/src/Disks/DiskMemory.cpp b/dbms/src/Disks/DiskMemory.cpp index 2b249858f6b..ca79cab79af 100644 --- a/dbms/src/Disks/DiskMemory.cpp +++ b/dbms/src/Disks/DiskMemory.cpp @@ -39,6 +39,14 @@ private: std::vector::iterator iter; }; +ReadIndirectBuffer::ReadIndirectBuffer(String path_, const String & data_) + : ReadBufferFromFileBase(), buf(ReadBufferFromString(data_)), path(std::move(path_)) +{ + internal_buffer = buf.buffer(); + working_buffer = internal_buffer; + pos = working_buffer.begin(); +} + off_t ReadIndirectBuffer::seek(off_t offset, int whence) { if (whence == SEEK_SET) @@ -79,14 +87,6 @@ off_t ReadIndirectBuffer::getPosition() return pos - working_buffer.begin(); } -ReadIndirectBuffer::ReadIndirectBuffer(String path_, const String & data_) - : ReadBufferFromFileBase(), buf(ReadBufferFromString(data_)), path(std::move(path_)) -{ - internal_buffer = buf.buffer(); - working_buffer = internal_buffer; - pos = working_buffer.begin(); -} - void WriteIndirectBuffer::finalize() { if (isFinished()) From 0040ad32892b2072454f09bd09ca9d2eba241644 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 18 Feb 2020 15:24:20 +0300 Subject: [PATCH 07/40] Fixed includes. --- dbms/src/Compression/CompressedReadBufferFromFile.h | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/dbms/src/Compression/CompressedReadBufferFromFile.h b/dbms/src/Compression/CompressedReadBufferFromFile.h index 33727909a69..1729490f606 100644 --- a/dbms/src/Compression/CompressedReadBufferFromFile.h +++ b/dbms/src/Compression/CompressedReadBufferFromFile.h @@ -1,10 +1,9 @@ #pragma once -#include -#include -#include -#include #include "CompressedReadBufferBase.h" +#include +#include +#include namespace DB From 05ceef4ef889e503cde03159465a20cd44b9e47a Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 18 Feb 2020 17:41:30 +0300 Subject: [PATCH 08/40] Log family storages disk support via 'SETTINGS' clause. --- dbms/src/Storages/StorageLog.cpp | 26 +++++++------------ dbms/src/Storages/StorageLogSettings.cpp | 22 ++++++++++++++++ dbms/src/Storages/StorageLogSettings.h | 10 +++++++ dbms/src/Storages/StorageStripeLog.cpp | 25 +++++++----------- dbms/src/Storages/StorageTinyLog.cpp | 25 +++++++----------- ...=> 01083_log_family_disk_memory.reference} | 0 ...y.sql => 01083_log_family_disk_memory.sql} | 6 ++--- 7 files changed, 65 insertions(+), 49 deletions(-) create mode 100644 dbms/src/Storages/StorageLogSettings.cpp create mode 100644 dbms/src/Storages/StorageLogSettings.h rename dbms/tests/queries/0_stateless/{01082_log_family_disk_memory.reference => 01083_log_family_disk_memory.reference} (100%) rename dbms/tests/queries/0_stateless/{01082_log_family_disk_memory.sql => 01083_log_family_disk_memory.sql} (75%) diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index da3023a5776..53adbf47203 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -22,7 +22,7 @@ #include #include - +#include "StorageLogSettings.h" #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" #define DBMS_STORAGE_LOG_MARKS_FILE_NAME "__marks.mrk" @@ -627,30 +627,24 @@ CheckResults StorageLog::checkData(const ASTPtr & /* query */, const Context & / void registerStorageLog(StorageFactory & factory) { + StorageFactory::StorageFeatures features{ + .supports_settings = true + }; + factory.registerStorage("Log", [](const StorageFactory::Arguments & args) { - ASTs & engine_args = args.engine_args; - - if (engine_args.size() > 1) + if (!args.engine_args.empty()) throw Exception( - "Engine " + args.engine_name + " requires 0 or 1 arguments: [disk_name] (" + toString(args.engine_args.size()) + " given)", + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (size_t i = 0; i < engine_args.size(); ++i) - engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); - - DiskPtr disk = args.context.getDefaultDisk(); - - if (engine_args.size() == 1) - { - String disk_name = engine_args[0]->as().value.safeGet(); - disk = args.context.getDisk(disk_name); - } + String disk_name = getDiskName(*args.storage_def); + DiskPtr disk = args.context.getDisk(disk_name); return StorageLog::create( disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.context.getSettings().max_compress_block_size); - }); + }, features); } } diff --git a/dbms/src/Storages/StorageLogSettings.cpp b/dbms/src/Storages/StorageLogSettings.cpp new file mode 100644 index 00000000000..5472c10001c --- /dev/null +++ b/dbms/src/Storages/StorageLogSettings.cpp @@ -0,0 +1,22 @@ +#include "StorageLogSettings.h" +#include +#include + +namespace DB +{ +String getDiskName(ASTStorage & storage_def) +{ + if (storage_def.settings) + { + SettingsChanges changes = storage_def.settings->changes; + for (auto it = changes.begin(); it != changes.end(); ++it) + { + if (it->name == "disk_name") + return it->value.safeGet(); + } + } + + return "default"; +} + +} diff --git a/dbms/src/Storages/StorageLogSettings.h b/dbms/src/Storages/StorageLogSettings.h new file mode 100644 index 00000000000..c970cd6be37 --- /dev/null +++ b/dbms/src/Storages/StorageLogSettings.h @@ -0,0 +1,10 @@ +#pragma once + +#include + +namespace DB +{ + class ASTStorage; + + String getDiskName(ASTStorage & storage_def); +} diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 9912cf47a89..84e625c89cc 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -29,6 +29,7 @@ #include #include #include +#include "StorageLogSettings.h" namespace DB @@ -306,30 +307,24 @@ void StorageStripeLog::truncate(const ASTPtr &, const Context &, TableStructureW void registerStorageStripeLog(StorageFactory & factory) { + StorageFactory::StorageFeatures features{ + .supports_settings = true + }; + factory.registerStorage("StripeLog", [](const StorageFactory::Arguments & args) { - ASTs & engine_args = args.engine_args; - - if (engine_args.size() > 1) + if (!args.engine_args.empty()) throw Exception( - "Engine " + args.engine_name + " requires 0 or 1 arguments: [disk_name] (" + toString(args.engine_args.size()) + " given)", + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (size_t i = 0; i < engine_args.size(); ++i) - engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); - - DiskPtr disk = args.context.getDefaultDisk(); - - if (engine_args.size() == 1) - { - String disk_name = engine_args[0]->as().value.safeGet(); - disk = args.context.getDisk(disk_name); - } + String disk_name = getDiskName(*args.storage_def); + DiskPtr disk = args.context.getDisk(disk_name); return StorageStripeLog::create( disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size); - }); + }, features); } } diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index db90ba3e41d..f3d7750fd10 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -31,6 +31,7 @@ #include #include #include +#include "StorageLogSettings.h" #define DBMS_STORAGE_LOG_DATA_FILE_EXTENSION ".bin" @@ -437,30 +438,24 @@ void StorageTinyLog::drop(TableStructureWriteLockHolder &) void registerStorageTinyLog(StorageFactory & factory) { + StorageFactory::StorageFeatures features{ + .supports_settings = true + }; + factory.registerStorage("TinyLog", [](const StorageFactory::Arguments & args) { - ASTs & engine_args = args.engine_args; - - if (engine_args.size() > 1) + if (!args.engine_args.empty()) throw Exception( - "Engine " + args.engine_name + " requires 0 or 1 arguments: [disk_name] (" + toString(args.engine_args.size()) + " given)", + "Engine " + args.engine_name + " doesn't support any arguments (" + toString(args.engine_args.size()) + " given)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (size_t i = 0; i < engine_args.size(); ++i) - engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.local_context); - - DiskPtr disk = args.context.getDefaultDisk(); - - if (engine_args.size() == 1) - { - String disk_name = engine_args[0]->as().value.safeGet(); - disk = args.context.getDisk(disk_name); - } + String disk_name = getDiskName(*args.storage_def); + DiskPtr disk = args.context.getDisk(disk_name); return StorageTinyLog::create( disk, args.relative_data_path, args.table_id, args.columns, args.constraints, args.attach, args.context.getSettings().max_compress_block_size); - }); + }, features); } } diff --git a/dbms/tests/queries/0_stateless/01082_log_family_disk_memory.reference b/dbms/tests/queries/0_stateless/01083_log_family_disk_memory.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01082_log_family_disk_memory.reference rename to dbms/tests/queries/0_stateless/01083_log_family_disk_memory.reference diff --git a/dbms/tests/queries/0_stateless/01082_log_family_disk_memory.sql b/dbms/tests/queries/0_stateless/01083_log_family_disk_memory.sql similarity index 75% rename from dbms/tests/queries/0_stateless/01082_log_family_disk_memory.sql rename to dbms/tests/queries/0_stateless/01083_log_family_disk_memory.sql index bbc75f1d3bd..b5aa101faf3 100644 --- a/dbms/tests/queries/0_stateless/01082_log_family_disk_memory.sql +++ b/dbms/tests/queries/0_stateless/01083_log_family_disk_memory.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS log; -CREATE TABLE log (x UInt8) ENGINE = StripeLog ('disk_memory'); +CREATE TABLE log (x UInt8) ENGINE = StripeLog () SETTINGS disk_name = 'disk_memory'; SELECT * FROM log ORDER BY x; INSERT INTO log VALUES (0); @@ -13,7 +13,7 @@ SELECT * FROM log ORDER BY x; TRUNCATE TABLE log; DROP TABLE log; -CREATE TABLE log (x UInt8) ENGINE = TinyLog ('disk_memory'); +CREATE TABLE log (x UInt8) ENGINE = TinyLog () SETTINGS disk_name = 'disk_memory'; SELECT * FROM log ORDER BY x; INSERT INTO log VALUES (0); @@ -26,7 +26,7 @@ SELECT * FROM log ORDER BY x; TRUNCATE TABLE log; DROP TABLE log; -CREATE TABLE log (x UInt8) ENGINE = Log ('disk_memory'); +CREATE TABLE log (x UInt8) ENGINE = Log () SETTINGS disk_name = 'disk_memory'; SELECT * FROM log ORDER BY x; INSERT INTO log VALUES (0); From ff50a51757cfc4967cb1176575e33b4bf3a4e30e Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 18 Feb 2020 22:13:37 +0300 Subject: [PATCH 09/40] Formatting issues. --- dbms/src/Storages/StorageLogSettings.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/StorageLogSettings.cpp b/dbms/src/Storages/StorageLogSettings.cpp index 5472c10001c..d2d8e2138d8 100644 --- a/dbms/src/Storages/StorageLogSettings.cpp +++ b/dbms/src/Storages/StorageLogSettings.cpp @@ -15,7 +15,6 @@ String getDiskName(ASTStorage & storage_def) return it->value.safeGet(); } } - return "default"; } From 39123b25b4b529ad4ffb1a4d0418593a12173faf Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Wed, 19 Feb 2020 07:43:19 +0000 Subject: [PATCH 10/40] Bump commonmark from 0.5.4 to 0.9.1 in /docs/tools Bumps [commonmark](https://github.com/rtfd/commonmark.py) from 0.5.4 to 0.9.1. - [Release notes](https://github.com/rtfd/commonmark.py/releases) - [Changelog](https://github.com/readthedocs/commonmark.py/blob/master/CHANGELOG.md) - [Commits](https://github.com/rtfd/commonmark.py/commits/0.9.1) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 32b8e4426ee..f53d273c405 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -5,7 +5,7 @@ beautifulsoup4==4.8.2 certifi==2017.11.5 chardet==3.0.4 click==6.7 -CommonMark==0.5.4 +CommonMark==0.9.1 cssmin==0.2.0 docutils==0.16 futures==3.1.1 From d96255ccc4d86cdaa150a542ba3f53c27b0b0265 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Wed, 19 Feb 2020 07:44:01 +0000 Subject: [PATCH 11/40] Bump unidecode from 1.0.23 to 1.1.1 in /docs/tools Bumps [unidecode](https://github.com/kmike/text-unidecode) from 1.0.23 to 1.1.1. - [Release notes](https://github.com/kmike/text-unidecode/releases) - [Commits](https://github.com/kmike/text-unidecode/commits) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 32b8e4426ee..c117f7539f5 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -31,6 +31,6 @@ Sphinx==1.6.5 sphinxcontrib-websupport==1.0.1 tornado==5.1 typing==3.7.4.1 -Unidecode==1.0.23 +Unidecode==1.1.1 urllib3==1.25.8 gitpython==2.1.14 From 9d7d89676099fc46f6faa21c6b352594c4bb0342 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Wed, 19 Feb 2020 07:44:25 +0000 Subject: [PATCH 12/40] Bump markupsafe from 1.0 to 1.1.1 in /docs/tools Bumps [markupsafe](https://github.com/pallets/markupsafe) from 1.0 to 1.1.1. - [Release notes](https://github.com/pallets/markupsafe/releases) - [Changelog](https://github.com/pallets/markupsafe/blob/master/CHANGES.rst) - [Commits](https://github.com/pallets/markupsafe/compare/1.0...1.1.1) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 32b8e4426ee..b23b6cd1932 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -16,7 +16,7 @@ Jinja2==2.11.1 jsmin==2.2.2 livereload==2.5.1 Markdown==2.6.11 -MarkupSafe==1.0 +MarkupSafe==1.1.1 mkdocs==1.0.4 Pygments==2.5.2 python-slugify==1.2.6 From b2a52a2d6980e4ccf4cfd4f2d71b3da291616527 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Feb 2020 11:49:26 +0300 Subject: [PATCH 13/40] Remove useless code #9210 --- dbms/src/Compression/ICompressionCodec.cpp | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/dbms/src/Compression/ICompressionCodec.cpp b/dbms/src/Compression/ICompressionCodec.cpp index d55b50caf33..ea3a74c18a5 100644 --- a/dbms/src/Compression/ICompressionCodec.cpp +++ b/dbms/src/Compression/ICompressionCodec.cpp @@ -9,23 +9,12 @@ #include -namespace ProfileEvents -{ - extern const Event ReadCompressedBytes; - extern const Event CompressedReadBufferBlocks; - extern const Event CompressedReadBufferBytes; -} - namespace DB { namespace ErrorCodes { - extern const int CHECKSUM_DOESNT_MATCH; - extern const int TOO_LARGE_SIZE_COMPRESSED; - extern const int UNKNOWN_COMPRESSION_METHOD; extern const int CANNOT_DECOMPRESS; - extern const int SEEK_POSITION_OUT_OF_BOUND; extern const int CORRUPTED_DATA; } From 811197558cc2695e1ff352dafbd06f4bab78b087 Mon Sep 17 00:00:00 2001 From: Nicolae Vartolomei Date: Wed, 19 Feb 2020 13:33:31 +0000 Subject: [PATCH 14/40] Improve stack trace formatting for Poco and std exceptions Before: ``` Application: Caught exception while loading metadata: Poco::Exception. Code: 1000, e.code() = 0, e.displayText() = Access to file denied: boo0. /home/nv/clickhouse-master-clion/contrib/poco/Foundation/src/Exception.cpp:27: Poco::FileAccessDeniedException::FileAccessDeniedException(std::__1::basic_string, std::__1::allocator > const&, int) @ 0xbb598cc in /state/home/nv/clickhouse-builds/clickhouse-master-clion-gcc/dbms/programs/clickhous ``` After: ``` Application: Caught exception while loading metadata: Poco::Exception. Code: 1000, e.code() = 0, e.displayText() = Access to file denied: boo, Stack trace (when copying this message, always include the lines below): 0. /home/nv/clickhouse-master-clion/contrib/poco/Foundation/src/Exception.cpp:27: Poco::FileAccessDeniedException::FileAccessDeniedException(std::__1::basic_string, std::__1::allocator > const&, int) @ 0xbb5987c in /state/home/nv/clickhouse-builds/clickhouse-master-clion-gcc/dbms/programs/clickhous ``` --- dbms/src/Common/Exception.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/Exception.cpp b/dbms/src/Common/Exception.cpp index 318da1a27f2..00ef520f37c 100644 --- a/dbms/src/Common/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -193,7 +193,7 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded { stream << "Poco::Exception. Code: " << ErrorCodes::POCO_EXCEPTION << ", e.code() = " << e.code() << ", e.displayText() = " << e.displayText() - << (with_stacktrace ? getExceptionStackTraceString(e) : "") + << (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "") << (with_extra_info ? getExtraExceptionInfo(e) : "") << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")"; } @@ -210,9 +210,9 @@ std::string getCurrentExceptionMessage(bool with_stacktrace, bool check_embedded name += " (demangling status: " + toString(status) + ")"; stream << "std::exception. Code: " << ErrorCodes::STD_EXCEPTION << ", type: " << name << ", e.what() = " << e.what() - << (with_stacktrace ? getExceptionStackTraceString(e) : "") + << (with_stacktrace ? ", Stack trace (when copying this message, always include the lines below):\n\n" + getExceptionStackTraceString(e) : "") << (with_extra_info ? getExtraExceptionInfo(e) : "") - << ", version = " << VERSION_STRING << VERSION_OFFICIAL; + << " (version " << VERSION_STRING << VERSION_OFFICIAL << ")"; } catch (...) {} } From add281dfe31492f19fff0a651c02cccb028a174c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 19 Feb 2020 16:39:20 +0300 Subject: [PATCH 15/40] Enable metric_log by default --- dbms/programs/server/config.xml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 27a6876ca13..ae15a583fcd 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -403,15 +403,13 @@ --> - system metric_log
7500 1000
- --> - ## CREATE DICTIONARY {#create-dictionary-query} ```sql @@ -300,3 +298,5 @@ External dictionary structure consists of attributes. Dictionary attributes are Depending on dictionary [layout](dicts/external_dicts_dict_layout.md) one or more attributes can be specified as dictionary keys. For more information, see [External Dictionaries](dicts/external_dicts.md) section. + +[Original article](https://clickhouse.tech/docs/en/query_language/create/) diff --git a/docs/en/query_language/dicts/external_dicts.md b/docs/en/query_language/dicts/external_dicts.md index 19a45e40e73..46733654f68 100644 --- a/docs/en/query_language/dicts/external_dicts.md +++ b/docs/en/query_language/dicts/external_dicts.md @@ -6,7 +6,7 @@ ClickHouse: - Fully or partially stores dictionaries in RAM. - Periodically updates dictionaries and dynamically loads missing values. In other words, dictionaries can be loaded dynamically. -- Allows to create external dictionaries with xml-files or [DDL queries](../create.md#create-dictionary-query). +- Allows to create external dictionaries with xml files or [DDL queries](../create.md#create-dictionary-query). The configuration of external dictionaries can be located in one or more xml-files. The path to the configuration is specified in the [dictionaries_config](../../operations/server_settings/settings.md#server_settings-dictionaries_config) parameter. @@ -34,12 +34,16 @@ You can [configure](external_dicts_dict.md) any number of dictionaries in the sa [DDL queries for dictionaries](../create.md#create-dictionary-query) doesn't require any additional records in server configuration. They allow to work with dictionaries as first-class entities, like tables or views. -!!! attention +!!! attention "Attention" You can convert values for a small dictionary by describing it in a `SELECT` query (see the [transform](../functions/other_functions.md) function). This functionality is not related to external dictionaries. -**See also** +## See also {#ext-dicts-see-also} +- [Configuring an External Dictionary](external_dicts_dict.md) +- [Storing Dictionaries in Memory](external_dicts_dict_layout.md) +- [Dictionary Updates](external_dicts_dict_lifetime.md) +- [Sources of External Dictionaries](external_dicts_dict_sources.md) +- [Dictionary Key and Fields](external_dicts_dict_structure.md) - [Functions for Working with External Dictionaries](../functions/ext_dict_functions.md) - [Original article](https://clickhouse.tech/docs/en/query_language/dicts/external_dicts/) diff --git a/docs/en/query_language/dicts/external_dicts_dict.md b/docs/en/query_language/dicts/external_dicts_dict.md index 03963c2d3b2..fba76199784 100644 --- a/docs/en/query_language/dicts/external_dicts_dict.md +++ b/docs/en/query_language/dicts/external_dicts_dict.md @@ -1,6 +1,6 @@ # Configuring an External Dictionary {#dicts-external_dicts_dict} -If dictionary is configured using xml-file, than dictionary configuration has the following structure: +If dictionary is configured using xml file, than dictionary configuration has the following structure: ```xml @@ -37,7 +37,7 @@ LAYOUT(...) -- Memory layout configuration LIFETIME(...) -- Lifetime of dictionary in memory ``` -- name – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`. +- `name` – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`. - [source](external_dicts_dict_sources.md) — Source of the dictionary. - [layout](external_dicts_dict_layout.md) — Dictionary layout in memory. - [structure](external_dicts_dict_structure.md) — Structure of the dictionary . A key and attributes that can be retrieved by this key. diff --git a/docs/en/query_language/dicts/external_dicts_dict_layout.md b/docs/en/query_language/dicts/external_dicts_dict_layout.md index 5a587b5408c..87034a5aa14 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -34,7 +34,7 @@ The configuration looks like this: ``` -in case of [DDL-query](../create.md#create-dictionary-query), equal configuration will looks like +Corresponding [DDL-query](../create.md#create-dictionary-query): ```sql CREATE DICTIONARY (...) diff --git a/docs/en/query_language/dicts/external_dicts_dict_structure.md b/docs/en/query_language/dicts/external_dicts_dict_structure.md index 8c80c53561d..f5a0b0b6017 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/en/query_language/dicts/external_dicts_dict_structure.md @@ -47,10 +47,14 @@ Attributes are described in the query body: ClickHouse supports the following types of keys: -- Numeric key. UInt64. Defined in the `` tag or using `PRIMARY KEY` keyword. +- Numeric key. `UInt64`. Defined in the `` tag or using `PRIMARY KEY` keyword. - Composite key. Set of values of different types. Defined in the tag `` or `PRIMARY KEY` keyword. -A xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. +An xml structure can contain either `` or ``. DDL-query must contain single `PRIMARY KEY`. + +!!! warning "Warning" + You must not describe key as an attribute. + ### Numeric Key {#ext_dict-numeric-key} diff --git a/docs/en/query_language/show.md b/docs/en/query_language/show.md index 93a2a1a8bd0..f6a9cc6865b 100644 --- a/docs/en/query_language/show.md +++ b/docs/en/query_language/show.md @@ -81,7 +81,7 @@ SELECT name FROM system.dictionaries WHERE database = [AND name LIKE diff --git a/docs/ru/query_language/create.md b/docs/ru/query_language/create.md index 7008e8588db..42568305cd3 100644 --- a/docs/ru/query_language/create.md +++ b/docs/ru/query_language/create.md @@ -274,4 +274,28 @@ SELECT a, b, c FROM (SELECT ...) Отсутствует отдельный запрос для удаления представлений. Чтобы удалить представление, следует использовать `DROP TABLE`. +## CREATE DICTIONARY {#create-dictionary-query} + +```sql +CREATE DICTIONARY [IF NOT EXISTS] [db.]dictionary_name +( + key1 type1 [DEFAULT|EXPRESSION expr1] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], + key2 type2 [DEFAULT|EXPRESSION expr2] [HIERARCHICAL|INJECTIVE|IS_OBJECT_ID], + attr1 type2 [DEFAULT|EXPRESSION expr3], + attr2 type2 [DEFAULT|EXPRESSION expr4] +) +PRIMARY KEY key1, key2 +SOURCE(SOURCE_NAME([param1 value1 ... paramN valueN])) +LAYOUT(LAYOUT_NAME([param_name param_value])) +LIFETIME([MIN val1] MAX val2) +``` + +Создаёт [внешний словарь](dicts/external_dicts.md) с заданной [структурой](dicts/external_dicts_dict_structure.md), [источником](dicts/external_dicts_dict_sources.md), [способом размещения в памяти](dicts/external_dicts_dict_layout.md) и [периодом обновления](dicts/external_dicts_dict_lifetime.md). + +Структура внешнего словаря состоит из атрибутов. Атрибуты словаря задаются как столбцы таблицы. Единственным обязательным свойством атрибута является его тип, все остальные свойства могут иметь значения по умолчанию. + +В зависимости от [способа размещения словаря в памяти](dicts/external_dicts_dict_layout.md), ключами словаря могут быть один и более атрибутов. + +Смотрите [Внешние словари](dicts/external_dicts.md). + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/create/) diff --git a/docs/ru/query_language/dicts/external_dicts.md b/docs/ru/query_language/dicts/external_dicts.md index 4ebb8f5143d..7e38cd9221d 100644 --- a/docs/ru/query_language/dicts/external_dicts.md +++ b/docs/ru/query_language/dicts/external_dicts.md @@ -2,9 +2,12 @@ Существует возможность подключать собственные словари из различных источников данных. Источником данных для словаря может быть локальный текстовый/исполняемый файл, HTTP(s) ресурс или другая СУБД. Подробнее смотрите в разделе "[Источники внешних словарей](external_dicts_dict_sources.md)". -ClickHouse полностью или частично хранит словари в оперативной памяти. Словари можно подгружать динамически, ClickHouse периодически обновляет их и динамически подгружает отсутствующие значения. +ClickHouse: +- Полностью или частично хранит словари в оперативной памяти. +- Периодически обновляет их и динамически подгружает отсутствующие значения. +- Позволяет создавать внешние словари с помощью xml-файлов или [DDL-запросов](../create.md#create-dictionary-query). -Конфигурация внешних словарей находится в одном или нескольких файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../operations/server_settings/settings.md). +Конфигурация внешних словарей может находится в одном или нескольких xml-файлах. Путь к конфигурации указывается в параметре [dictionaries_config](../../operations/server_settings/settings.md). Словари могут загружаться при старте сервера или при первом использовании, в зависимости от настройки [dictionaries_lazy_load](../../operations/server_settings/settings.md). @@ -30,12 +33,15 @@ ClickHouse полностью или частично хранит словар ``` -В одном файле можно [сконфигурировать](external_dicts_dict.md) произвольное количество словарей. Формат файла сохраняется даже если словарь один (т.е. ` `). +В одном файле можно [сконфигурировать](external_dicts_dict.md) произвольное количество словарей. ->можете преобразовывать значения по небольшому словарю, описав его в запросе `SELECT` (см. функцию [transform](../functions/other_functions.md)). Эта функциональность не связана с внешними словарями. +Если вы создаёте внешние словари [DDL-запросами](../create.md#create-dictionary-query), то не задавайте конфигурацию словаря в конфигурации сервера. + +!!! attention "Внимание" + Можно преобразовывать значения по небольшому словарю, описав его в запросе `SELECT` (см. функцию [transform](../functions/other_functions.md)). Эта функциональность не связана с внешними словарями. -Смотрите также: +## Смотрите также {#ext-dicts-see-also} - [Настройка внешнего словаря](external_dicts_dict.md) - [Хранение словарей в памяти](external_dicts_dict_layout.md) diff --git a/docs/ru/query_language/dicts/external_dicts_dict.md b/docs/ru/query_language/dicts/external_dicts_dict.md index b004b2d8100..61946c10ee8 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict.md +++ b/docs/ru/query_language/dicts/external_dicts_dict.md @@ -1,11 +1,15 @@ # Настройка внешнего словаря {#dicts-external_dicts_dict} -Конфигурация словаря имеет следующую структуру: +XML-конфигурация словаря имеет следующую структуру: ```xml dict_name + + + + @@ -14,20 +18,29 @@ - - - - ``` -- name - Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`. -- [source](external_dicts_dict_sources.md) - Источник словаря. -- [layout](external_dicts_dict_layout.md) - Размещение словаря в памяти. -- [structure](external_dicts_dict_structure.md) - Структура словаря. Ключ и атрибуты, которые можно получить по ключу. -- [lifetime](external_dicts_dict_lifetime.md) - Периодичность обновления словарей. +Соответствующий [DDL-запрос](../create.md#create-dictionary-query) имеет следующий вид: + +```sql +CREATE DICTIONARY dict_name +( + ... -- attributes +) +PRIMARY KEY ... -- complex or single key configuration +SOURCE(...) -- Source configuration +LAYOUT(...) -- Memory layout configuration +LIFETIME(...) -- Lifetime of dictionary in memory +``` + +- `name` — Идентификатор, под которым словарь будет доступен для использования. Используйте символы `[a-zA-Z0-9_\-]`. +- [source](external_dicts_dict_sources.md) — Источник словаря. +- [layout](external_dicts_dict_layout.md) — Размещение словаря в памяти. +- [structure](external_dicts_dict_structure.md) — Структура словаря. Ключ и атрибуты, которые можно получить по ключу. +- [lifetime](external_dicts_dict_lifetime.md) — Периодичность обновления словарей. [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/dicts/external_dicts_dict/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_layout.md b/docs/ru/query_language/dicts/external_dicts_dict_layout.md index a1a35ae2c4d..0b0f8ab4b82 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_layout.md @@ -35,15 +35,25 @@ ``` +Соответствущий [DDL-запрос](../create.md#create-dictionary-query): + +```sql +CREATE DICTIONARY (...) +... +LAYOUT(LAYOUT_TYPE(param value)) -- layout settings +... +``` + ## Способы размещения словарей в памяти -- [flat](#flat) -- [hashed](#hashed) -- [cache](#cache) -- [range_hashed](#range-hashed) -- [complex_key_hashed](#complex-key-hashed) -- [complex_key_cache](#complex-key-cache) -- [ip_trie](#ip-trie) +- [flat](#flat) +- [hashed](#hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) +- [cache](#cache) +- [range_hashed](#range-hashed) +- [complex_key_hashed](#complex-key-hashed) +- [complex_key_cache](#complex-key-cache) +- [ip_trie](#ip-trie) ### flat @@ -63,6 +73,12 @@ ``` +или + +```sql +LAYOUT(FLAT()) +``` + ### hashed Словарь полностью хранится в оперативной памяти в виде хэш-таблиц. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике, количество ключей может достигать десятков миллионов элементов. @@ -77,6 +93,29 @@ ``` +или + +```sql +LAYOUT(HASHED()) +``` + +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} + +Аналогичен `hashed`, но при этом занимает меньше места в памяти и генерирует более высокую загрузку CPU. + +Пример конфигурации: + +```xml + + + +``` + +или + +```sql +LAYOUT(SPARSE_HASHED()) +``` ### complex_key_hashed @@ -90,6 +129,12 @@ ``` +или + +```sql +LAYOUT(COMPLEX_KEY_HASHED()) +``` + ### range_hashed @@ -131,6 +176,19 @@ ... ``` +или + +```sql +CREATE DICTIONARY somedict ( + id UInt64, + first Date, + last Date +) +PRIMARY KEY id +LAYOUT(RANGE_HASHED()) +RANGE(MIN first MAX last) +``` + Для работы с такими словарями в функцию `dictGetT` необходимо передавать дополнительный аргумент, для которого подбирается диапазон: dictGetT('dict_name', 'attr_name', id, date) @@ -178,6 +236,18 @@ ``` +или + +```sql +CREATE DICTIONARY somedict( + Abcdef UInt64, + StartTimeStamp UInt64, + EndTimeStamp UInt64, + XXXType String DEFAULT '' +) +PRIMARY KEY Abcdef +RANGE(MIN StartTimeStamp MAX EndTimeStamp) +``` ### cache @@ -204,6 +274,12 @@ ``` +или + +```sql +LAYOUT(CACHE(SIZE_IN_CELLS 1000000000)) +``` + Укажите достаточно большой размер кэша. Количество ячеек следует подобрать экспериментальным путём: 1. Выставить некоторое значение. @@ -265,6 +341,17 @@ ... ``` +или + +```sql +CREATE DICTIONARY somedict ( + prefix String, + asn UInt32, + cca2 String DEFAULT '??' +) +PRIMARY KEY prefix +``` + Этот ключ должен иметь только один атрибут типа `String`, содержащий допустимый префикс IP. Другие типы еще не поддерживаются. Для запросов необходимо использовать те же функции (`dictGetT` с кортежем), что и для словарей с составными ключами: diff --git a/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md b/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md index 2108d3e49ff..d18ac8b38af 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_lifetime.md @@ -14,6 +14,15 @@ ClickHouse периодически обновляет словари. Инте ... ``` +или + +```sql +CREATE DICTIONARY (...) +... +LIFETIME(300) +... +``` + Настройка `0` запрещает обновление словарей. @@ -32,6 +41,12 @@ ClickHouse периодически обновляет словари. Инте ``` +или + +```sql +LIFETIME(MIN 300 MAX 360) +``` + При обновлении словарей сервер ClickHouse применяет различную логику в зависимости от типа [источника](external_dicts_dict_sources.md): > - У текстового файла проверяется время модификации. Если время изменилось по отношению к запомненному ранее, то словарь обновляется. @@ -56,4 +71,12 @@ ClickHouse периодически обновляет словари. Инте ``` +или + +```sql +... +SOURCE(ODBC(... invalidate_query 'SELECT update_time FROM dictionary_source where id = 1')) +... +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/dicts/external_dicts_dict_lifetime/) diff --git a/docs/ru/query_language/dicts/external_dicts_dict_sources.md b/docs/ru/query_language/dicts/external_dicts_dict_sources.md index 8ef492e050f..58caf501847 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_sources.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_sources.md @@ -3,7 +3,7 @@ Внешний словарь можно подключить из множества источников. -Общий вид конфигурации: +Общий вид XML-конфигурации: ```xml @@ -20,6 +20,16 @@ ``` +Аналогичный [DDL-запрос](../create.md#create-dictionary-query): + +```sql +CREATE DICTIONARY dict_name (...) +... +SOURCE(SOURCE_TYPE(param1 val1 ... paramN valN)) -- Source configuration +... +``` + + Источник настраивается в разделе `source`. Типы источников (`source_type`): @@ -48,10 +58,16 @@ ``` +или + +```sql +SOURCE(FILE(path '/opt/dictionaries/os.tsv' format 'TabSeparated')) +``` + Поля настройки: -- `path` - Абсолютный путь к файлу. -- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)". +- `path` — Абсолютный путь к файлу. +- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)". ## Исполняемый файл {#dicts-external_dicts_dict_sources-executable} @@ -69,10 +85,16 @@ ``` +или + +```sql +SOURCE(EXECUTABLE(command 'cat /opt/dictionaries/os.tsv' format 'TabSeparated')) +``` + Поля настройки: -- `command` - Абсолютный путь к исполняемому файлу или имя файла (если каталог программы прописан в `PATH`). -- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)". +- `command` — Абсолютный путь к исполняемому файлу или имя файла (если каталог программы прописан в `PATH`). +- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)". ## HTTP(s) {#dicts-external_dicts_dict_sources-http} @@ -86,16 +108,37 @@ http://[::1]/os.tsv TabSeparated + + user + password + + +
+ API-KEY + key +
+
``` +или + +```sql +SOURCE(HTTP( + url 'http://[::1]/os.tsv' + format 'TabSeparated' + credentials(user 'user' password 'password') + headers(header(name 'API-KEY' value 'key')) +)) +``` + Чтобы ClickHouse смог обратиться к HTTPS-ресурсу, необходимо [настроить openSSL](../../operations/server_settings/settings.md) в конфигурации сервера. Поля настройки: -- `url` - URL источника. -- `format` - Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)". +- `url` — URL источника. +- `format` — Формат файла. Поддерживаются все форматы, описанные в разделе "[Форматы](../../interfaces/formats.md#formats)". ## ODBC {#dicts-external_dicts_dict_sources-odbc} @@ -105,20 +148,33 @@ Пример настройки: ```xml - - DatabaseName - ShemaName.TableName
- DSN=some_parameters - SQL_QUERY -
+ + + DatabaseName + ShemaName.TableName
+ DSN=some_parameters + SQL_QUERY +
+ +``` + +или + +```sql +SOURCE(ODBC( + db 'DatabaseName' + table 'SchemaName.TableName' + connection_string 'DSN=some_parameters' + invalidate_query 'SQL_QUERY' +)) ``` Поля настройки: -- `db` - имя базы данных. Не указывать, если имя базы задано в параметрах. ``. -- `table` - имя таблицы и схемы, если она есть. -- `connection_string` - строка соединения. -- `invalidate_query` - запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md). +- `db` — имя базы данных. Не указывать, если имя базы задано в параметрах. ``. +- `table` — имя таблицы и схемы, если она есть. +- `connection_string` — строка соединения. +- `invalidate_query` — запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md). ClickHouse получает от ODBC-драйвера информацию о квотировании и квотирует настройки в запросах к драйверу, поэтому имя таблицы нужно указывать в соответствии с регистром имени таблицы в базе данных. @@ -216,6 +272,18 @@ $ sudo apt-get install -y unixodbc odbcinst odbc-postgresql ``` +или + +```sql +CREATE DICTIONARY table_name ( + id UInt64, + some_column UInt64 DEFAULT 0 +) +PRIMARY KEY id +SOURCE(ODBC(connection_string 'DSN=myconnection' table 'postgresql_table')) +LAYOUT(HASHED()) +LIFETIME(MIN 300 MAX 360) + Может понадобиться в `odbc.ini` указать полный путь до библиотеки с драйвером `DRIVER=/usr/local/lib/psqlodbcw.so`. ### Пример подключения MS SQL Server @@ -299,6 +367,20 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh ``` +или + +```sql +CREATE DICTIONARY test ( + k UInt64, + s String DEFAULT '' +) +PRIMARY KEY k +SOURCE(ODBC(table 'dict' connection_string 'DSN=MSSQL;UID=test;PWD=test')) +LAYOUT(FLAT()) +LIFETIME(MIN 300 MAX 360) +``` + + ## СУБД @@ -328,6 +410,22 @@ $ sudo apt-get install tdsodbc freetds-bin sqsh ``` +или + +```sql +SOURCE(MYSQL( + port 3306 + user 'clickhouse' + password 'qwerty' + replica(host 'example01-1' priority 1) + replica(host 'example01-2' priority 1) + db 'db_name' + table 'table_name' + where 'id=10' + invalidate_query 'SQL_QUERY' +)) +``` + Поля настройки: - `port` — порт сервера MySQL. Можно указать для всех реплик или для каждой в отдельности (внутри ``). @@ -362,6 +460,21 @@ MySQL можно подключить на локальном хосте чер ``` +или + +```sql +SOURCE(MYSQL( + host 'localhost' + socket '/path/to/socket/file.sock' + user 'clickhouse' + password 'qwerty' + db 'db_name' + table 'table_name' + where 'id=10' + invalidate_query 'SQL_QUERY' +)) +``` + ### ClickHouse {#dicts-external_dicts_dict_sources-clickhouse} @@ -381,16 +494,30 @@ MySQL можно подключить на локальном хосте чер ``` +или + +```sql +SOURCE(CLICKHOUSE( + host 'example01-01-1' + port 9000 + user 'default' + password '' + db 'default' + table 'ids' + where 'id=10' +)) +``` + Поля настройки: -- `host` - хост ClickHouse. Если host локальный, то запрос выполняется без сетевого взаимодействия. Чтобы повысить отказоустойчивость решения, можно создать таблицу типа [Distributed](../../operations/table_engines/distributed.md) и прописать её в дальнейших настройках. -- `port` - порт сервера ClickHouse. -- `user` - имя пользователя ClickHouse. -- `password` - пароль пользователя ClickHouse. -- `db` - имя базы данных. -- `table` - имя таблицы. -- `where` - условие выбора. Может отсутствовать. -- `invalidate_query` - запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md). +- `host` — хост ClickHouse. Если host локальный, то запрос выполняется без сетевого взаимодействия. Чтобы повысить отказоустойчивость решения, можно создать таблицу типа [Distributed](../../operations/table_engines/distributed.md) и прописать её в дальнейших настройках. +- `port` — порт сервера ClickHouse. +- `user` — имя пользователя ClickHouse. +- `password` — пароль пользователя ClickHouse. +- `db` — имя базы данных. +- `table` — имя таблицы. +- `where` — условие выбора. Может отсутствовать. +- `invalidate_query` — запрос для проверки статуса словаря. Необязательный параметр. Читайте подробнее в разделе [Обновление словарей](external_dicts_dict_lifetime.md). ### MongoDB {#dicts-external_dicts_dict_sources-mongodb} @@ -410,14 +537,27 @@ MySQL можно подключить на локальном хосте чер ``` +или + +```sql +SOURCE(MONGO( + host 'localhost' + port 27017 + user '' + password '' + db 'test' + collection 'dictionary_source' +)) +``` + Поля настройки: -- `host` - хост MongoDB. -- `port` - порт сервера MongoDB. -- `user` - имя пользователя MongoDB. -- `password` - пароль пользователя MongoDB. -- `db` - имя базы данных. -- `collection` - имя коллекции. +- `host` — хост MongoDB. +- `port` — порт сервера MongoDB. +- `user` — имя пользователя MongoDB. +- `password` — пароль пользователя MongoDB. +- `db` — имя базы данных. +- `collection` — имя коллекции. ### Redis {#dicts-external_dicts_dict_sources-redis} @@ -434,6 +574,17 @@ MySQL можно подключить на локальном хосте чер ``` +или + +```sql +SOURCE(REDIS( + host 'localhost' + port 6379 + storage_type 'simple' + db_index 0 +)) +``` + Поля настройки: - `host` – хост Redis. diff --git a/docs/ru/query_language/dicts/external_dicts_dict_structure.md b/docs/ru/query_language/dicts/external_dicts_dict_structure.md index c0f76db60b2..f9ab9d30dac 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_structure.md @@ -24,10 +24,10 @@ Атрибуты описываются элементами: - `` — [столбец с ключом](external_dicts_dict_structure.md#ext_dict_structure-key). -- `` — [столбец данных](external_dicts_dict_structure.md#ext_dict_structure-attributes). Можно задать несколько столбцов. +- `` — [столбец данных](external_dicts_dict_structure.md#ext_dict_structure-attributes). Можно задать несколько атрибутов. -Запрос создания словаря: +Создание словаря запросом: ```sql CREATE DICTIONARY dict_name ( @@ -48,10 +48,10 @@ PRIMARY KEY Id ClickHouse поддерживает следующие виды ключей: -- Числовой ключ. `UInt64`. Описывается в теге ``. -- Составной ключ. Набор значений разного типа. Описывается в теге ``. +- Числовой ключ. `UInt64`. Описывается в теге `` или ключевым словом `PRIMARY KEY`. +- Составной ключ. Набор значений разного типа. Описывается в теге `` или ключевым словом `PRIMARY KEY`. -Структура может содержать либо `` либо ``. +Структура может содержать либо `` либо ``. DDL-запрос может содержать только `PRIMARY KEY`. !!! warning "Обратите внимание" Ключ не надо дополнительно описывать в атрибутах. @@ -72,6 +72,20 @@ ClickHouse поддерживает следующие виды ключей: - `name` — имя столбца с ключами. +Для DDL-запроса: + +```sql +CREATE DICTIONARY ( + Id UInt64, + ... +) +PRIMARY KEY Id +... +``` + +- `PRIMARY KEY` – имя столбца с ключами. + + ### Составной ключ Ключом может быть кортеж (`tuple`) из полей произвольных типов. В этом случае [layout](external_dicts_dict_layout.md) должен быть `complex_key_hashed` или `complex_key_cache`. @@ -97,6 +111,18 @@ ClickHouse поддерживает следующие виды ключей: ... ``` +или + +```sql +CREATE DICTIONARY ( + field1 String, + field2 String + ... +) +PRIMARY KEY field1, field2 +... +``` + При запросе в функции `dictGet*` в качестве ключа передаётся кортеж. Пример: `dictGetString('dict_name', 'attr_name', tuple('string for field1', num_for_field2))`. @@ -119,6 +145,15 @@ ClickHouse поддерживает следующие виды ключей: ``` +или + +```sql +CREATE DICTIONARY somename ( + Name ClickHouseDataType DEFAULT '' EXPRESSION rand64() HIERARCHICAL INJECTIVE IS_OBJECT_ID +) +``` + + Поля конфигурации: | Тег | Описание | Обязательный | diff --git a/docs/ru/query_language/show.md b/docs/ru/query_language/show.md index 4eec70a8002..680b42809c5 100644 --- a/docs/ru/query_language/show.md +++ b/docs/ru/query_language/show.md @@ -3,10 +3,10 @@ ## SHOW CREATE TABLE ```sql -SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] +SHOW CREATE [TEMPORARY] [TABLE|DICTIONARY] [db.]table [INTO OUTFILE filename] [FORMAT format] ``` -Возвращает один столбец типа `String` с именем statement, содержащий одно значение — запрос `CREATE TABLE`, с помощью которого была создана указанная таблица. +Возвращает один столбец типа `String` с именем statement, содержащий одно значение — запрос `CREATE TABLE`, с помощью которого был создан указанный объект. ## SHOW DATABASES {#show-databases} @@ -62,3 +62,35 @@ SHOW TABLES FROM system LIKE '%co%' LIMIT 2 │ collations │ └────────────────────────────────┘ ``` + +## SHOW DICTIONARIES + +Выводит список [внешних словарей](dicts/external_dicts.md). + +```sql +SHOW DICTIONARIES [FROM ] [LIKE ''] [LIMIT ] [INTO OUTFILE ] [FORMAT ] +``` + +Если секция `FROM` не указана, запрос возвращает список словарей из текущей базы данных. + +Аналогичный результат можно получить следующим запросом: + +```sql +SELECT name FROM system.dictionaries WHERE database = [AND name LIKE ] [LIMIT ] [INTO OUTFILE ] [FORMAT ] +``` + +**Example** + +Запрос выводит первые две стоки из списка таблиц в базе данных `system`, имена которых содержат `reg`. + +```sql +SHOW DICTIONARIES FROM db LIKE '%reg%' LIMIT 2 +``` +```text +┌─name─────────┐ +│ regions │ +│ region_names │ +└──────────────┘ +``` + +[Оригинальная статья](https://clickhouse.tech/docs/ru/query_language/show/) From 49b86a638d9a21afa75dbd23cf1f50fdb7b83954 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Thu, 20 Feb 2020 07:42:43 +0000 Subject: [PATCH 27/40] Bump requests from 2.21.0 to 2.23.0 in /docs/tools Bumps [requests](https://github.com/psf/requests) from 2.21.0 to 2.23.0. - [Release notes](https://github.com/psf/requests/releases) - [Changelog](https://github.com/psf/requests/blob/master/HISTORY.md) - [Commits](https://github.com/psf/requests/compare/v2.21.0...v2.23.0) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 078fa607d1f..2dbb65c2133 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -23,7 +23,7 @@ python-slugify==1.2.6 pytz==2017.3 PyYAML==5.3 recommonmark==0.4.0 -requests==2.21.0 +requests==2.23.0 singledispatch==3.4.0.3 six==1.11.0 snowballstemmer==1.2.1 From 694185c0c2413b58feb1867ba37e7bd9fda4108c Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Thu, 20 Feb 2020 07:42:59 +0000 Subject: [PATCH 28/40] Bump idna from 2.6 to 2.9 in /docs/tools Bumps [idna](https://github.com/kjd/idna) from 2.6 to 2.9. - [Release notes](https://github.com/kjd/idna/releases) - [Changelog](https://github.com/kjd/idna/blob/master/HISTORY.rst) - [Commits](https://github.com/kjd/idna/compare/v2.6...v2.9) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 078fa607d1f..63cfda54480 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -10,7 +10,7 @@ cssmin==0.2.0 docutils==0.16 futures==3.1.1 htmlmin==0.1.12 -idna==2.6 +idna==2.9 imagesize==1.2.0 Jinja2==2.11.1 jsmin==2.2.2 From 8230160a00997b8f0cd014a56b3078b98cf33929 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Thu, 20 Feb 2020 07:43:22 +0000 Subject: [PATCH 29/40] Bump six from 1.11.0 to 1.14.0 in /docs/tools Bumps [six](https://github.com/benjaminp/six) from 1.11.0 to 1.14.0. - [Release notes](https://github.com/benjaminp/six/releases) - [Changelog](https://github.com/benjaminp/six/blob/master/CHANGES) - [Commits](https://github.com/benjaminp/six/compare/1.11.0...1.14.0) Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 078fa607d1f..381011ed4d2 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -25,7 +25,7 @@ PyYAML==5.3 recommonmark==0.4.0 requests==2.21.0 singledispatch==3.4.0.3 -six==1.11.0 +six==1.14.0 snowballstemmer==1.2.1 Sphinx==1.6.5 sphinxcontrib-websupport==1.0.1 From 24153a5233ceceb44575f28726ea89907e7c8168 Mon Sep 17 00:00:00 2001 From: "dependabot-preview[bot]" <27856297+dependabot-preview[bot]@users.noreply.github.com> Date: Thu, 20 Feb 2020 07:43:43 +0000 Subject: [PATCH 30/40] Bump certifi from 2017.11.5 to 2019.11.28 in /docs/tools Bumps [certifi](https://certifi.io/) from 2017.11.5 to 2019.11.28. Signed-off-by: dependabot-preview[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 078fa607d1f..d1d6ef1a423 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -2,7 +2,7 @@ alabaster==0.7.12 Babel==2.5.1 backports-abc==0.5 beautifulsoup4==4.8.2 -certifi==2017.11.5 +certifi==2019.11.28 chardet==3.0.4 click==6.7 CommonMark==0.9.1 From 23ceddebbcea2101e3e76d6bfcf76e17c7835059 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 20 Feb 2020 12:06:00 +0300 Subject: [PATCH 31/40] codestyle final 2 --- dbms/programs/copier/ClusterPartition.h | 2 +- dbms/programs/copier/Internals.cpp | 2 +- dbms/programs/copier/Internals.h | 8 ++++---- dbms/programs/copier/TaskTableAndShard.h | 12 ++++++------ 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/dbms/programs/copier/ClusterPartition.h b/dbms/programs/copier/ClusterPartition.h index 89dd1c1d2a2..ed69bfa8c26 100644 --- a/dbms/programs/copier/ClusterPartition.h +++ b/dbms/programs/copier/ClusterPartition.h @@ -5,7 +5,7 @@ namespace DB { /// Contains info about all shards that contain a partition - struct ClusterPartition + struct ClusterPartition { double elapsed_time_seconds = 0; UInt64 bytes_copied = 0; diff --git a/dbms/programs/copier/Internals.cpp b/dbms/programs/copier/Internals.cpp index 84add0f2098..331bdeb25de 100644 --- a/dbms/programs/copier/Internals.cpp +++ b/dbms/programs/copier/Internals.cpp @@ -1,6 +1,6 @@ #include "Internals.h" -namespace DB +namespace DB { ConfigurationPtr getConfigurationFromXMLString(const std::string & xml_data) diff --git a/dbms/programs/copier/Internals.h b/dbms/programs/copier/Internals.h index 57fcd2bfb01..c1377a250d8 100644 --- a/dbms/programs/copier/Internals.h +++ b/dbms/programs/copier/Internals.h @@ -106,19 +106,19 @@ struct TaskStateWithOwner TaskState state{TaskState::Unknown}; String owner; - static String getData(TaskState state, const String &owner) + static String getData(TaskState state, const String &owner) { return TaskStateWithOwner(state, owner).toString(); } - String toString() + String toString() { WriteBufferFromOwnString wb; wb << static_cast(state) << "\n" << escape << owner; return wb.str(); } - static TaskStateWithOwner fromString(const String & data) + static TaskStateWithOwner fromString(const String & data) { ReadBufferFromString rb(data); TaskStateWithOwner res; @@ -142,7 +142,7 @@ struct ShardPriority size_t hostname_difference = 0; UInt8 random = 0; - static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) + static bool greaterPriority(const ShardPriority & current, const ShardPriority & other) { return std::forward_as_tuple(current.is_remote, current.hostname_difference, current.random) < std::forward_as_tuple(other.is_remote, other.hostname_difference, other.random); diff --git a/dbms/programs/copier/TaskTableAndShard.h b/dbms/programs/copier/TaskTableAndShard.h index 4e4aaf18a96..f08cb208ee2 100644 --- a/dbms/programs/copier/TaskTableAndShard.h +++ b/dbms/programs/copier/TaskTableAndShard.h @@ -84,8 +84,8 @@ struct TaskTable }; - struct TaskShard - { +struct TaskShard +{ TaskShard(TaskTable &parent, const ShardInfo &info_) : task_table(parent), info(info_) {} TaskTable & task_table; @@ -222,13 +222,13 @@ inline TaskTable::TaskTable(TaskCluster & parent, const Poco::Util::AbstractConf } template -inline void TaskTable::initShards(RandomEngine && random_engine) +inline void TaskTable::initShards(RandomEngine && random_engine) { const String & fqdn_name = getFQDNOrHostName(); std::uniform_int_distribution get_urand(0, std::numeric_limits::max()); // Compute the priority - for (auto & shard_info : cluster_pull->getShardsInfo()) + for (auto & shard_info : cluster_pull->getShardsInfo()) { TaskShardPtr task_shard = std::make_shared(*this, shard_info); const auto & replicas = cluster_pull->getShardsAddresses().at(task_shard->indexInCluster()); @@ -239,14 +239,14 @@ inline void TaskTable::initShards(RandomEngine && random_engine) // Sort by priority std::sort(all_shards.begin(), all_shards.end(), - [](const TaskShardPtr & lhs, const TaskShardPtr & rhs) + [](const TaskShardPtr & lhs, const TaskShardPtr & rhs) { return ShardPriority::greaterPriority(lhs->priority, rhs->priority); }); // Cut local shards auto it_first_remote = std::lower_bound(all_shards.begin(), all_shards.end(), 1, - [](const TaskShardPtr & lhs, UInt8 is_remote) + [](const TaskShardPtr & lhs, UInt8 is_remote) { return lhs->priority.is_remote < is_remote; }); From 5a67c02a5d18ed9e3df00760c96945d3400cf785 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 20 Feb 2020 12:53:02 +0300 Subject: [PATCH 32/40] In KeyCondition: Fixed execution of inversed predicates for non-strictly monotinic functional index (#9223) * Tests for functional index * Fixed execution of inversed predicates in functional index When non-strictly monotonic functional index is used inverted predicated may be executed incorrectly, which leads to multiple problems: #8821, #9034 --- dbms/src/Storages/MergeTree/KeyCondition.cpp | 114 +++++++++++++++--- dbms/src/Storages/MergeTree/KeyCondition.h | 4 +- ...83_functional_index_in_mergetree.reference | 33 +++++ .../01083_functional_index_in_mergetree.sql | 33 +++++ 4 files changed, 163 insertions(+), 21 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.reference create mode 100644 dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.sql diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index 4ce58c85809..f8c7db4a423 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -264,6 +264,78 @@ const KeyCondition::AtomMap KeyCondition::atom_map }; +static const std::map inverse_relations = { + {"equals", "notEquals"}, + {"notEquals", "equals"}, + {"less", "greaterOrEquals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + {"lessOrEquals", "greater"}, + {"in", "notIn"}, + {"notIn", "in"}, + {"like", "notLike"}, + {"notLike", "like"}, + {"empty", "notEmpty"}, + {"notEmpty", "empty"}, +}; + + +bool isLogicalOperator(const String & func_name) +{ + return (func_name == "and" || func_name == "or" || func_name == "not" || func_name == "indexHint"); +} + +/// The node can be one of: +/// - Logical operator (AND, OR, NOT and indexHint() - logical NOOP) +/// - An "atom" (relational operator, constant, expression) +/// - A logical constant expression +/// - Any other function +ASTPtr cloneASTWithInversionPushDown(const ASTPtr node, const bool need_inversion = false) +{ + const ASTFunction * func = node->as(); + + if (func && isLogicalOperator(func->name)) + { + if (func->name == "not") + { + return cloneASTWithInversionPushDown(func->arguments->children.front(), !need_inversion); + } + + const auto result_node = makeASTFunction(func->name); + + /// indexHint() is a special case - logical NOOP function + if (result_node->name != "indexHint" && need_inversion) + { + result_node->name = (result_node->name == "and") ? "or" : "and"; + } + + if (func->arguments) + { + for (const auto & child : func->arguments->children) + { + result_node->arguments->children.push_back(cloneASTWithInversionPushDown(child, need_inversion)); + } + } + + return result_node; + } + + const auto cloned_node = node->clone(); + + if (func && inverse_relations.find(func->name) != inverse_relations.cend()) + { + if (need_inversion) + { + cloned_node->as()->name = inverse_relations.at(func->name); + } + + return cloned_node; + } + + return need_inversion ? makeASTFunction("not", cloned_node) : cloned_node; +} + + inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); } inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); } @@ -345,21 +417,23 @@ KeyCondition::KeyCondition( */ Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context); - /// Trasform WHERE section to Reverse Polish notation - const auto & select = query_info.query->as(); - if (select.where()) + const ASTSelectQuery & select = query_info.query->as(); + if (select.where() || select.prewhere()) { - traverseAST(select.where(), context, block_with_constants); + ASTPtr filter_query; + if (select.where() && select.prewhere()) + filter_query = makeASTFunction("and", select.where(), select.prewhere()); + else + filter_query = select.where() ? select.where() : select.prewhere(); - if (select.prewhere()) - { - traverseAST(select.prewhere(), context, block_with_constants); - rpn.emplace_back(RPNElement::FUNCTION_AND); - } - } - else if (select.prewhere()) - { - traverseAST(select.prewhere(), context, block_with_constants); + /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)), + * the use of NOT operator in predicate will result in the indexing algorithm leave out some data. + * This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict + * when parsing the AST into internal RPN representation. + * To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's + * are pushed down and applied (when possible) to leaf nodes. + */ + traverseAST(cloneASTWithInversionPushDown(filter_query), context, block_with_constants); } else { @@ -432,9 +506,9 @@ void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Blo { RPNElement element; - if (auto * func = node->as()) + if (const auto * func = node->as()) { - if (operatorFromAST(func, element)) + if (tryParseLogicalOperatorFromAST(func, element)) { auto & args = func->arguments->children; for (size_t i = 0, size = args.size(); i < size; ++i) @@ -452,7 +526,7 @@ void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Blo } } - if (!atomFromAST(node, context, block_with_constants, element)) + if (!tryParseAtomFromAST(node, context, block_with_constants, element)) { element.function = RPNElement::FUNCTION_UNKNOWN; } @@ -680,7 +754,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value, } -bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out) +bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out) { /** Functions < > = != <= >= in `notIn`, where one argument is a constant, and the other is one of columns of key, * or itself, wrapped in a chain of possibly-monotonic functions, @@ -768,7 +842,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo func_name = "lessOrEquals"; else if (func_name == "lessOrEquals") func_name = "greaterOrEquals"; - else if (func_name == "in" || func_name == "notIn" || func_name == "like") + else if (func_name == "in" || func_name == "notIn" || + func_name == "like" || func_name == "notLike" || + func_name == "startsWith") { /// "const IN data_column" doesn't make sense (unlike "data_column IN const") return false; @@ -809,7 +885,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo return false; } -bool KeyCondition::operatorFromAST(const ASTFunction * func, RPNElement & out) +bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out) { /// Functions AND, OR, NOT. /** Also a special function `indexHint` - works as if instead of calling a function there are just parentheses diff --git a/dbms/src/Storages/MergeTree/KeyCondition.h b/dbms/src/Storages/MergeTree/KeyCondition.h index fd1d11c0ec8..004cfbc9ea8 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.h +++ b/dbms/src/Storages/MergeTree/KeyCondition.h @@ -369,8 +369,8 @@ private: BoolMask initial_mask) const; void traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants); - bool atomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out); - bool operatorFromAST(const ASTFunction * func, RPNElement & out); + bool tryParseAtomFromAST(const ASTPtr & node, const Context & context, Block & block_with_constants, RPNElement & out); + bool tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out); /** Is node the key column * or expression in which column of key is wrapped by chain of functions, diff --git a/dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.reference b/dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.reference new file mode 100644 index 00000000000..bff552df991 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.reference @@ -0,0 +1,33 @@ +TP1 +7.51 +7.42 +7.41 +7.42 +7.41 +7.42 +7.41 +7.42 +7.41 +7.51 +TP2 +7.42 +7.41 +7.42 +7.51 +7.42 +7.41 +7.51 +7.51 +TP3 +7.42 +7.41 +7.51 +TP4 +7.42 +7.41 +7.42 +7.42 +7.41 +TP5 +7.41 +7.51 diff --git a/dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.sql b/dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.sql new file mode 100644 index 00000000000..d0fbf3356c8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01083_functional_index_in_mergetree.sql @@ -0,0 +1,33 @@ +SET max_threads = 1; + +CREATE TABLE IF NOT EXISTS functional_index_mergetree (x Float64) ENGINE = MergeTree ORDER BY round(x); +INSERT INTO functional_index_mergetree VALUES (7.42)(7.41)(7.51); + +SELECT 'TP1'; +SELECT * FROM functional_index_mergetree WHERE x > 7.42; +SELECT * FROM functional_index_mergetree WHERE x < 7.49; +SELECT * FROM functional_index_mergetree WHERE x < 7.5; + +SELECT * FROM functional_index_mergetree WHERE NOT (NOT x < 7.49); +SELECT * FROM functional_index_mergetree WHERE NOT (NOT x < 7.5); +SELECT * FROM functional_index_mergetree WHERE NOT (NOT x > 7.42); + +SELECT 'TP2'; +SELECT * FROM functional_index_mergetree WHERE NOT x > 7.49; +SELECT * FROM functional_index_mergetree WHERE NOT x < 7.42; +SELECT * FROM functional_index_mergetree WHERE NOT x < 7.41; +SELECT * FROM functional_index_mergetree WHERE NOT x < 7.5; + +SELECT 'TP3'; +SELECT * FROM functional_index_mergetree WHERE x > 7.41 AND x < 7.51; +SELECT * FROM functional_index_mergetree WHERE NOT (x > 7.41 AND x < 7.51); + +SELECT 'TP4'; +SELECT * FROM functional_index_mergetree WHERE NOT x < 7.41 AND NOT x > 7.49; +SELECT * FROM functional_index_mergetree WHERE NOT x < 7.42 AND NOT x > 7.42; +SELECT * FROM functional_index_mergetree WHERE (NOT x < 7.4) AND (NOT x > 7.49); + +SELECT 'TP5'; +SELECT * FROM functional_index_mergetree WHERE NOT or(NOT x, toUInt64(x) AND NOT floor(x) > 6, x >= 7.42 AND round(x) <= 7); + +DROP TABLE functional_index_mergetree; From 7cf650653431d24e2920a586d1586ec428f0911a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 20 Feb 2020 11:01:31 +0100 Subject: [PATCH 33/40] Update documentation for system.replicas --- docs/en/operations/system_tables.md | 119 ++++++++++++++-------------- docs/ru/operations/system_tables.md | 119 ++++++++++++++-------------- 2 files changed, 116 insertions(+), 122 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 5244877c469..0eb1b8d67f6 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -694,76 +694,73 @@ FORMAT Vertical ```text Row 1: ────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -log_max_index: 596273 -log_pointer: 596274 -total_replicas: 2 -active_replicas: 2 +database: merge +table: visits +engine: ReplicatedCollapsingMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 1 +parts_to_check: 0 +zookeeper_path: /clickhouse/tables/01-06/visits +replica_name: example01-06-1.yandex.ru +replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru +columns_version: 9 +queue_size: 1 +inserts_in_queue: 0 +merges_in_queue: 1 +part_mutations_in_queue: 0 +queue_oldest_time: 2020-02-20 08:34:30 +inserts_oldest_time: 0000-00-00 00:00:00 +merges_oldest_time: 2020-02-20 08:34:30 +part_mutations_oldest_time: 0000-00-00 00:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: 20200220_20284_20840_7 +oldest_part_to_mutate_to: +log_max_index: 596273 +log_pointer: 596274 +last_queue_update: 2020-02-20 08:34:32 +absolute_delay: 0 +total_replicas: 2 +active_replicas: 2 ``` Columns: -```text -database: Database name -table: Table name -engine: Table engine name - -is_leader: Whether the replica is the leader. - -Only one replica at a time can be the leader. The leader is responsible for selecting background merges to perform. +- `database` (`String`) - Database name +- `table` (`String`) - Table name +- `engine` (`String`) - Table engine name +- `is_leader` (`UInt8`) - Whether the replica is the leader. +Only one replica at a time can be the leader. The leader is responsible for selecting background merges to perform. Note that writes can be performed to any replica that is available and has a session in ZK, regardless of whether it is a leader. - -is_readonly: Whether the replica is in read-only mode. +- `can_become_leader` (`UInt8`) - Whether the replica can be elected as a leader. +- `is_readonly` (`UInt8`) - Whether the replica is in read-only mode. This mode is turned on if the config doesn't have sections with ZooKeeper, if an unknown error occurred when reinitializing sessions in ZooKeeper, and during session reinitialization in ZooKeeper. - -is_session_expired: Whether the session with ZooKeeper has expired. -Basically the same as 'is_readonly'. - -future_parts: The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet. - -parts_to_check: The number of data parts in the queue for verification. -A part is put in the verification queue if there is suspicion that it might be damaged. - -zookeeper_path: Path to table data in ZooKeeper. -replica_name: Replica name in ZooKeeper. Different replicas of the same table have different names. -replica_path: Path to replica data in ZooKeeper. The same as concatenating 'zookeeper_path/replicas/replica_path'. - -columns_version: Version number of the table structure. -Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas haven't made all of the ALTERs yet. - -queue_size: Size of the queue for operations waiting to be performed. -Operations include inserting blocks of data, merges, and certain other actions. -It usually coincides with 'future_parts'. - -inserts_in_queue: Number of inserts of blocks of data that need to be made. -Insertions are usually replicated fairly quickly. If this number is large, it means something is wrong. - -merges_in_queue: The number of merges waiting to be made. -Sometimes merges are lengthy, so this value may be greater than zero for a long time. +- `is_session_expired` (`UInt8`) - the session with ZooKeeper has expired. Basically the same as `is_readonly`. +- `future_parts` (`UInt32`) - The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet. +- `parts_to_check` (`UInt32`) - The number of data parts in the queue for verification. A part is put in the verification queue if there is suspicion that it might be damaged. +- `zookeeper_path` (`String`) - Path to table data in ZooKeeper. +- `replica_name` (`String`) - Replica name in ZooKeeper. Different replicas of the same table have different names. +- `replica_path` (`String`) - Path to replica data in ZooKeeper. The same as concatenating 'zookeeper_path/replicas/replica_path'. +- `columns_version` (`Int32`) - Version number of the table structure. Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas haven't made all of the ALTERs yet. +- `queue_size` (`UInt32`) - Size of the queue for operations waiting to be performed. Operations include inserting blocks of data, merges, and certain other actions. It usually coincides with `future_parts`. +- `inserts_in_queue` (`UInt32`) - Number of inserts of blocks of data that need to be made. Insertions are usually replicated fairly quickly. If this number is large, it means something is wrong. +- `merges_in_queue` (`UInt32`) - The number of merges waiting to be made. Sometimes merges are lengthy, so this value may be greater than zero for a long time. +- `part_mutations_in_queue` (`UInt32`) - The number of mutations waiting to be made. +- `queue_oldest_time` (`DateTime`) - If `queue_size` greater than 0, shows when the oldest operation was added to the queue. +- `inserts_oldest_time` (`DateTime`) - See `queue_oldest_time` +- `merges_oldest_time` (`DateTime`) - See `queue_oldest_time` +- `part_mutations_oldest_time` (`DateTime`) - See `queue_oldest_time` The next 4 columns have a non-zero value only where there is an active session with ZK. -log_max_index: Maximum entry number in the log of general activity. -log_pointer: Maximum entry number in the log of general activity that the replica copied to its execution queue, plus one. -If log_pointer is much smaller than log_max_index, something is wrong. - -total_replicas: The total number of known replicas of this table. -active_replicas: The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). -``` +- `log_max_index` (`UInt64`) - Maximum entry number in the log of general activity. +- `log_pointer` (`UInt64`) - Maximum entry number in the log of general activity that the replica copied to its execution queue, plus one. If `log_pointer` is much smaller than `log_max_index`, something is wrong. +- `last_queue_update` (`DateTime`) - When the queue was updated last time. +- `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. +- `total_replicas` (`UInt8`) - The total number of known replicas of this table. +- `active_replicas` (`UInt8`) - The number of replicas of this table that have a session in ZooKeeper (i.e., the number of functioning replicas). If you request all the columns, the table may work a bit slowly, since several reads from ZooKeeper are made for each row. If you don't request the last 4 columns (log_max_index, log_pointer, total_replicas, active_replicas), the table works quickly. diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index 400412325d2..dad397462d9 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -741,76 +741,73 @@ FORMAT Vertical ```text Row 1: ────── -database: merge -table: visits -engine: ReplicatedCollapsingMergeTree -is_leader: 1 -is_readonly: 0 -is_session_expired: 0 -future_parts: 1 -parts_to_check: 0 -zookeeper_path: /clickhouse/tables/01-06/visits -replica_name: example01-06-1.yandex.ru -replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru -columns_version: 9 -queue_size: 1 -inserts_in_queue: 0 -merges_in_queue: 1 -log_max_index: 596273 -log_pointer: 596274 -total_replicas: 2 -active_replicas: 2 +database: merge +table: visits +engine: ReplicatedCollapsingMergeTree +is_leader: 1 +can_become_leader: 1 +is_readonly: 0 +is_session_expired: 0 +future_parts: 1 +parts_to_check: 0 +zookeeper_path: /clickhouse/tables/01-06/visits +replica_name: example01-06-1.yandex.ru +replica_path: /clickhouse/tables/01-06/visits/replicas/example01-06-1.yandex.ru +columns_version: 9 +queue_size: 1 +inserts_in_queue: 0 +merges_in_queue: 1 +part_mutations_in_queue: 0 +queue_oldest_time: 2020-02-20 08:34:30 +inserts_oldest_time: 0000-00-00 00:00:00 +merges_oldest_time: 2020-02-20 08:34:30 +part_mutations_oldest_time: 0000-00-00 00:00:00 +oldest_part_to_get: +oldest_part_to_merge_to: 20200220_20284_20840_7 +oldest_part_to_mutate_to: +log_max_index: 596273 +log_pointer: 596274 +last_queue_update: 2020-02-20 08:34:32 +absolute_delay: 0 +total_replicas: 2 +active_replicas: 2 ``` Столбцы: -```text -database: имя БД -table: имя таблицы -engine: имя движка таблицы - -is_leader: является ли реплика лидером - -В один момент времени, не более одной из реплик является лидером. Лидер отвечает за выбор фоновых слияний, которые следует произвести. +- `database` (`String`) - имя БД. +- `table` (`String`) - имя таблицы. +- `engine` (`String`) - имя движка таблицы. +- `is_leader` (`UInt8`) - является ли реплика лидером. +В один момент времени, не более одной из реплик является лидером. Лидер отвечает за выбор фоновых слияний, которые следует произвести. Замечу, что запись можно осуществлять на любую реплику (доступную и имеющую сессию в ZK), независимо от лидерства. - -is_readonly: находится ли реплика в режиме "только для чтения" +- `can_become_leader` (`UInt8`) - может ли реплика быть выбрана лидером. +- `is_readonly` (`UInt8`) - находится ли реплика в режиме "только для чтения" Этот режим включается, если в конфиге нет секции с ZK; если при переинициализации сессии в ZK произошла неизвестная ошибка; во время переинициализации сессии с ZK. - -is_session_expired: истекла ли сессия с ZK. -В основном, то же самое, что и is_readonly. - -future_parts: количество кусков с данными, которые появятся в результате INSERT-ов или слияний, которых ещё предстоит сделать - -parts_to_check: количество кусков с данными в очереди на проверку -Кусок помещается в очередь на проверку, если есть подозрение, что он может быть битым. - -zookeeper_path: путь к данным таблицы в ZK -replica_name: имя реплики в ZK; разные реплики одной таблицы имеют разное имя -replica_path: путь к данным реплики в ZK. То же самое, что конкатенация zookeeper_path/replicas/replica_path. - -columns_version: номер версии структуры таблицы -Обозначает, сколько раз был сделан ALTER. Если на репликах разные версии, значит некоторые реплики сделали ещё не все ALTER-ы. - -queue_size: размер очереди действий, которых предстоит сделать -К действиям относятся вставки блоков данных, слияния, и некоторые другие действия. -Как правило, совпадает с future_parts. - -inserts_in_queue: количество вставок блоков данных, которых предстоит сделать -Обычно вставки должны быстро реплицироваться. Если величина большая - значит что-то не так. - -merges_in_queue: количество слияний, которых предстоит сделать -Бывают длинные слияния - то есть, это значение может быть больше нуля продолжительное время. +- `is_session_expired` (`UInt8`) - истекла ли сессия с ZK. В основном, то же самое, что и `is_readonly`. +- `future_parts` (`UInt32`) - количество кусков с данными, которые появятся в результате INSERT-ов или слияний, которых ещё предстоит сделать +- `parts_to_check` (`UInt32`) - количество кусков с данными в очереди на проверку. Кусок помещается в очередь на проверку, если есть подозрение, что он может быть битым. +- `zookeeper_path` (`String`) - путь к данным таблицы в ZK. +- `replica_name` (`String`) - имя реплики в ZK; разные реплики одной таблицы имеют разное имя. +- `replica_path` (`String`) - путь к данным реплики в ZK. То же самое, что конкатенация zookeeper_path/replicas/replica_path. +- `columns_version` (`Int32`) - номер версии структуры таблицы. Обозначает, сколько раз был сделан ALTER. Если на репликах разные версии, значит некоторые реплики сделали ещё не все ALTER-ы. +- `queue_size` (`UInt32`) - размер очереди действий, которые предстоит сделать. К действиям относятся вставки блоков данных, слияния, и некоторые другие действия. Как правило, совпадает с future_parts. +- `inserts_in_queue` (`UInt32`) - количество вставок блоков данных, которые предстоит сделать. Обычно вставки должны быстро реплицироваться. Если величина большая - значит что-то не так. +- `merges_in_queue` (`UInt32`) - количество слияний, которые предстоит сделать. Бывают длинные слияния - то есть, это значение может быть больше нуля продолжительное время. +- `part_mutations_in_queue` (`UInt32`) - количество мутаций, которые предстоит сделать. +- `queue_oldest_time` (`DateTime`) - если `queue_size` больше 0, показывает, когда была добавлена в очередь самая старая операция. +- `inserts_oldest_time` (`DateTime`) - см. `queue_oldest_time`. +- `merges_oldest_time` (`DateTime`) - см. `queue_oldest_time`. +- `part_mutations_oldest_time` (`DateTime`) - см. `queue_oldest_time`. Следующие 4 столбца имеют ненулевое значение только если активна сессия с ZK. -log_max_index: максимальный номер записи в общем логе действий -log_pointer: максимальный номер записи из общего лога действий, которую реплика скопировала в свою очередь для выполнения, плюс единица -Если log_pointer сильно меньше log_max_index, значит что-то не так. - -total_replicas: общее число известных реплик этой таблицы -active_replicas: число реплик этой таблицы, имеющих сессию в ZK; то есть, число работающих реплик -``` +- `log_max_index` (`UInt64`) - максимальный номер записи в общем логе действий. +- `log_pointer` (`UInt64`) - максимальный номер записи из общего лога действий, которую реплика скопировала в свою очередь для выполнения, плюс единица. Если log_pointer сильно меньше log_max_index, значит что-то не так. +- `last_queue_update` (`DateTime`) - When the queue was updated last time. +- `absolute_delay` (`UInt64`) - How big lag in seconds the current replica has. +- `total_replicas` (`UInt8`) - общее число известных реплик этой таблицы. +- `active_replicas` (`UInt8`) - число реплик этой таблицы, имеющих сессию в ZK; то есть, число работающих реплик. Если запрашивать все столбцы, то таблица может работать слегка медленно, так как на каждую строчку делается несколько чтений из ZK. Если не запрашивать последние 4 столбца (log_max_index, log_pointer, total_replicas, active_replicas), то таблица работает быстро. From ee2cb00aa1bb254e343110af07d6846c5658136d Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 20 Feb 2020 15:46:27 +0300 Subject: [PATCH 34/40] Use getAllPhysical() --- dbms/src/Storages/StorageLog.cpp | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 0c44081d68f..e9c539ea9a5 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -541,15 +541,8 @@ void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLo const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const { /// There should be at least one physical column - auto begin = getColumns().begin(); - while (begin != getColumns().end() && begin->default_desc == ColumnDefaultKind::Alias) - ++begin; - - if (begin == getColumns().end()) - throw Exception("No physical columns found!", ErrorCodes::LOGICAL_ERROR); - - const String & column_name = begin->name; - const IDataType & column_type = *begin->type; + const String & column_name = getColumns().getAllPhysical().begin()->name; + const IDataType & column_type = *getColumns().getAllPhysical().begin()->type; String filename; /** We take marks from first column. From 87b32a283178c3b9b07c7e4b8b4ce1074b1bc8cc Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 20 Feb 2020 16:32:48 +0300 Subject: [PATCH 35/40] Create events.js --- website/workers/events.js | 34 ++++++++++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 website/workers/events.js diff --git a/website/workers/events.js b/website/workers/events.js new file mode 100644 index 00000000000..653139af9f9 --- /dev/null +++ b/website/workers/events.js @@ -0,0 +1,34 @@ +addEventListener('fetch', event => { + event.respondWith(handleRequest(event.request)) +}) + +async function handleRequest(request) { + let raw = await fetch('https://raw.githubusercontent.com/ClickHouse/ClickHouse/master/README.md'); + let text = await raw.text(); + let lines = text.split('\n'); + let skip = true; + let events = []; + for (let idx in lines) { + let line = lines[idx]; + if (skip) { + if (line.includes('Upcoming Events')) { + skip = false; + } + } else { + if (!line) { continue; }; + line = line.split(']('); + var tail = line[1].split(') '); + events.push({ + 'signup_link': tail[0], + 'event_name': line[0].replace('* [', ''), + 'event_date': tail[1].slice(0, -1).replace('on ', '') + }); + } + } + + let response = new Response(JSON.stringify({ + 'events': events + })); + response.headers.set('Content-Type', 'application/json'); + return response; +} From 9748f8dcf65eff3a35baf4e06e1671bb2f6e74dd Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 20 Feb 2020 16:33:14 +0300 Subject: [PATCH 36/40] fix bug with comma join and in --- .../Interpreters/CrossToInnerJoinVisitor.cpp | 23 ++++--------------- .../0_stateless/00863_comma_join_in.sql | 2 ++ ...01083_cross_to_inner_with_in_bug.reference | 1 + .../01083_cross_to_inner_with_in_bug.sql | 13 +++++++++++ 4 files changed, 21 insertions(+), 18 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.reference create mode 100644 dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.sql diff --git a/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp b/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp index 54d5205c4c2..740fe35e936 100644 --- a/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp +++ b/dbms/src/Interpreters/CrossToInnerJoinVisitor.cpp @@ -124,15 +124,12 @@ public: { /// leave other comparisons as is } - else if (functionIsLikeOperator(node.name)) /// LIKE, NOT LIKE + else if (functionIsLikeOperator(node.name) || /// LIKE, NOT LIKE + functionIsInOperator(node.name)) /// IN, NOT IN { - /// leave as is - } - else if (functionIsInOperator(node.name)) /// IN, NOT IN - { - if (auto ident = node.arguments->children.at(0)->as()) - if (size_t min_table = checkIdentifier(*ident)) - asts_to_join_on[min_table].push_back(ast); + /// leave as is. It's not possible to make push down here cause of unknown aliases and not implemented JOIN predicates. + /// select a as b form t1, t2 where t1.x = t2.x and b in(42) + /// select a as b form t1 inner join t2 on t1.x = t2.x and b in(42) } else { @@ -202,16 +199,6 @@ private: } return 0; } - - size_t checkIdentifier(const ASTIdentifier & identifier) - { - size_t best_table_pos = 0; - bool match = IdentifierSemantic::chooseTable(identifier, tables, best_table_pos); - - if (match && joined_tables[best_table_pos].canAttachOnExpression()) - return best_table_pos; - return 0; - } }; using CheckExpressionMatcher = ConstOneTypeMatcher; diff --git a/dbms/tests/queries/0_stateless/00863_comma_join_in.sql b/dbms/tests/queries/0_stateless/00863_comma_join_in.sql index 8cfc00627dc..ebccd351c8a 100644 --- a/dbms/tests/queries/0_stateless/00863_comma_join_in.sql +++ b/dbms/tests/queries/0_stateless/00863_comma_join_in.sql @@ -10,6 +10,8 @@ insert into test1_00863 (id, code) select number, toString(number) FROM numbers( insert into test3_00863 (id, code) select number, toString(number) FROM numbers(100000); insert into test2_00863 (id, code, test1_id, test3_id) select number, toString(number), number, number FROM numbers(100000); +SET max_memory_usage = 50000000; + select test2_00863.id from test1_00863, test2_00863, test3_00863 where test1_00863.code in ('1', '2', '3') diff --git a/dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.reference b/dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.reference new file mode 100644 index 00000000000..699fa0cd95c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.reference @@ -0,0 +1 @@ +2 2 1 diff --git a/dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.sql b/dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.sql new file mode 100644 index 00000000000..f6d788512f8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01083_cross_to_inner_with_in_bug.sql @@ -0,0 +1,13 @@ +drop table if exists ax; +drop table if exists bx; + +create table ax (A Int64, B Int64) Engine = Memory; +create table bx (A Int64) Engine = Memory; + +insert into ax values (1, 1), (2, 1); +insert into bx values (2), (4); + +select * from bx, ax where ax.A = bx.A and ax.B in (1,2); + +drop table ax; +drop table bx; From 0c686baf4f810f9aba2d3da9dd0c1e068ec4a93f Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 20 Feb 2020 16:33:19 +0300 Subject: [PATCH 37/40] Create repo.js --- website/workers/repo.js | 10 ++++++++++ 1 file changed, 10 insertions(+) create mode 100644 website/workers/repo.js diff --git a/website/workers/repo.js b/website/workers/repo.js new file mode 100644 index 00000000000..470391cf225 --- /dev/null +++ b/website/workers/repo.js @@ -0,0 +1,10 @@ +addEventListener('fetch', event => { + event.respondWith(handleRequest(event.request)) +}) + +async function handleRequest(request) { + let url = new URL(request.url); + url.hostname = 'repo.yandex.ru'; + url.pathname = '/clickhouse' + url.pathname; + return fetch(url) +} From 9191e5f17facdcdaefb6757575d48d8588457552 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 20 Feb 2020 18:26:20 +0300 Subject: [PATCH 38/40] Fix NDEBUG in PipelineExecutor. --- .../Processors/Executors/PipelineExecutor.cpp | 20 +++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 728554c80cf..f4be5a84518 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -263,7 +263,7 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue std::vector updated_direct_edges; { -#ifndef N_DEBUG +#ifndef NDEBUG Stopwatch watch; #endif @@ -279,7 +279,7 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, size_t thread_number, Queue return false; } -#ifndef N_DEBUG +#ifndef NDEBUG node.execution_state->preparation_time_ns += watch.elapsed(); #endif @@ -468,7 +468,7 @@ void PipelineExecutor::execute(size_t num_threads) } catch (...) { -#ifndef N_DEBUG +#ifndef NDEBUG LOG_TRACE(log, "Exception while executing query. Current state:\n" << dumpPipeline()); #endif throw; @@ -491,7 +491,7 @@ void PipelineExecutor::execute(size_t num_threads) void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads) { -#ifndef N_DEBUG +#ifndef NDEBUG UInt64 total_time_ns = 0; UInt64 execution_time_ns = 0; UInt64 processing_time_ns = 0; @@ -577,13 +577,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads addJob(state); { -#ifndef N_DEBUG +#ifndef NDEBUG Stopwatch execution_time_watch; #endif state->job(); -#ifndef N_DEBUG +#ifndef NDEBUG execution_time_ns += execution_time_watch.elapsed(); #endif } @@ -594,7 +594,7 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads if (finished) break; -#ifndef N_DEBUG +#ifndef NDEBUG Stopwatch processing_time_watch; #endif @@ -648,13 +648,13 @@ void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads doExpandPipeline(task, false); } -#ifndef N_DEBUG +#ifndef NDEBUG processing_time_ns += processing_time_watch.elapsed(); #endif } } -#ifndef N_DEBUG +#ifndef NDEBUG total_time_ns = total_time_watch.elapsed(); wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns; @@ -769,7 +769,7 @@ String PipelineExecutor::dumpPipeline() const WriteBufferFromOwnString buffer; buffer << "(" << node.execution_state->num_executed_jobs << " jobs"; -#ifndef N_DEBUG +#ifndef NDEBUG buffer << ", execution time: " << node.execution_state->execution_time_ns / 1e9 << " sec."; buffer << ", preparation time: " << node.execution_state->preparation_time_ns / 1e9 << " sec."; #endif From d211b1a4826155d7fcaea23c963d2040e701e067 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Thu, 20 Feb 2020 18:40:23 +0300 Subject: [PATCH 39/40] Don't use refs on vanishing objects --- dbms/src/Storages/StorageLog.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index e9c539ea9a5..b079d5d7a8a 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -541,8 +541,8 @@ void StorageLog::truncate(const ASTPtr &, const Context &, TableStructureWriteLo const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const { /// There should be at least one physical column - const String & column_name = getColumns().getAllPhysical().begin()->name; - const IDataType & column_type = *getColumns().getAllPhysical().begin()->type; + const String column_name = getColumns().getAllPhysical().begin()->name; + const auto column_type = getColumns().getAllPhysical().begin()->type; String filename; /** We take marks from first column. @@ -550,7 +550,7 @@ const StorageLog::Marks & StorageLog::getMarksWithRealRowCount() const * (Example: for Array data type, first stream is array sizes; and number of array sizes is the number of arrays). */ IDataType::SubstreamPath substream_root_path; - column_type.enumerateStreams([&](const IDataType::SubstreamPath & substream_path) + column_type->enumerateStreams([&](const IDataType::SubstreamPath & substream_path) { if (filename.empty()) filename = IDataType::getFileNameForStream(column_name, substream_path); From 3ae2282209ce504a6314d24204128aaaf73e7d40 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Thu, 20 Feb 2020 19:28:21 +0300 Subject: [PATCH 40/40] performance comparison --- docker/test/performance-comparison/compare.sh | 85 ++++++++++++++----- .../test/performance-comparison/entrypoint.sh | 2 +- docker/test/performance-comparison/perf.py | 2 +- 3 files changed, 67 insertions(+), 22 deletions(-) diff --git a/docker/test/performance-comparison/compare.sh b/docker/test/performance-comparison/compare.sh index eba6aba0637..c782a52592b 100755 --- a/docker/test/performance-comparison/compare.sh +++ b/docker/test/performance-comparison/compare.sh @@ -27,13 +27,16 @@ function download wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$left_pr/$left_sha/performance/performance.tgz" -O- | tar -C left --strip-components=1 -zxv & wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$right_pr/$right_sha/performance/performance.tgz" -O- | tar -C right --strip-components=1 -zxv & else - wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$left_pr/$left_sha/performance/performance.tgz" -O- | tar -C left --strip-components=1 -zxv && cp -al left right + wget -nv -nd -c "https://clickhouse-builds.s3.yandex.net/$left_pr/$left_sha/performance/performance.tgz" -O- | tar -C left --strip-components=1 -zxv && cp -a left right & fi cd db0 && wget -nv -nd -c "https://s3.mds.yandex.net/clickhouse-private-datasets/hits_10m_single/partitions/hits_10m_single.tar" -O- | tar -xv & cd db0 && wget -nv -nd -c "https://s3.mds.yandex.net/clickhouse-private-datasets/hits_100m_single/partitions/hits_100m_single.tar" -O- | tar -xv & cd db0 && wget -nv -nd -c "https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_v1.tar" -O- | tar -xv & cd db0 && wget -nv -nd -c "https://clickhouse-datasets.s3.yandex.net/values_with_expressions/partitions/test_values.tar" -O- | tar -xv & + + mkdir ~/fg ; cd ~/fg && wget -nv -nd -c "https://raw.githubusercontent.com/brendangregg/FlameGraph/master/flamegraph.pl" && chmod +x ~/fg/flamegraph.pl & + wait } @@ -223,13 +226,13 @@ function get_profiles function report { -for x in *.tsv +for x in {right,left}-{addresses,{query,trace}-log}.tsv do # FIXME This loop builds column definitons from TSVWithNamesAndTypes in an # absolutely atrocious way. This should be done by the file() function itself. paste -d' ' \ - <(sed -n '1s/\t/\n/gp' "$x" | sed 's/\(^.*$\)/"\1"/') \ - <(sed -n '2s/\t/\n/gp' "$x" ) \ + <(sed -n '1{s/\t/\n/g;p;q}' "$x" | sed 's/\(^.*$\)/"\1"/') \ + <(sed -n '2{s/\t/\n/g;p;q}' "$x" ) \ | tr '\n' ', ' | sed 's/,$//' > "$x.columns" done @@ -301,43 +304,85 @@ create view right_query_log as select * create view right_trace_log as select * from file('right-trace-log.tsv', TSVWithNamesAndTypes, '$(cat right-trace-log.tsv.columns)'); -create view right_addresses as select * +create view right_addresses_src as select * from file('right-addresses.tsv', TSVWithNamesAndTypes, '$(cat right-addresses.tsv.columns)'); -create table unstable_query_ids engine File(TSVWithNamesAndTypes, 'unstable-query-ids.rep') as - select query_id from right_query_log +create table right_addresses_join engine Join(any, left, address) as + select addr address, name from right_addresses_src; + +create table unstable_query_runs engine File(TSVWithNamesAndTypes, 'unstable-query-runs.rep') as + select query_id, query from right_query_log join unstable_queries_tsv using query + where query_id not like 'prewarm %' ; -create table unstable_query_metrics engine File(TSVWithNamesAndTypes, 'unstable-query-metrics.rep') as +create table unstable_query_log engine File(Vertical, 'unstable-query-log.rep') as + select * from right_query_log + where query_id in (select query_id from unstable_query_runs); + +create table unstable_run_metrics engine File(TSVWithNamesAndTypes, 'unstable-run-metrics.rep') as select ProfileEvents.Values value, ProfileEvents.Names metric, query_id, query from right_query_log array join ProfileEvents - where query_id in (unstable_query_ids) + where query_id in (select query_id from unstable_query_runs) ; -create table unstable_query_traces engine File(TSVWithNamesAndTypes, 'unstable-query-traces.rep') as - select count() value, right_addresses.name metric, - unstable_query_ids.query_id, any(right_query_log.query) query - from unstable_query_ids - join right_query_log on right_query_log.query_id = unstable_query_ids.query_id - join right_trace_log on right_trace_log.query_id = unstable_query_ids.query_id - join right_addresses on addr = arrayJoin(trace) - group by unstable_query_ids.query_id, metric +create table unstable_run_metrics_2 engine File(TSVWithNamesAndTypes, 'unstable-run-metrics-2.rep') as + select v, n, query_id, query + from + (select + ['memory_usage', 'read_bytes', 'written_bytes'] n, + [memory_usage, read_bytes, written_bytes] v, + query, + query_id + from right_query_log + where query_id in (select query_id from unstable_query_runs)) + array join n, v; + +create table unstable_run_traces engine File(TSVWithNamesAndTypes, 'unstable-run-traces.rep') as + select count() value, joinGet(right_addresses_join, 'name', arrayJoin(trace)) metric, + unstable_query_runs.query_id, any(unstable_query_runs.query) query + from unstable_query_runs + join right_trace_log on right_trace_log.query_id = unstable_query_runs.query_id + group by unstable_query_runs.query_id, metric order by count() desc ; create table metric_devation engine File(TSVWithNamesAndTypes, 'metric-deviation.rep') as select floor((q[3] - q[1])/q[2], 3) d, - quantilesExact(0.05, 0.5, 0.95)(value) q, metric, query - from (select * from unstable_query_metrics - union all select * from unstable_query_traces) + quantilesExact(0, 0.5, 1)(value) q, metric, query + from (select * from unstable_run_metrics + union all select * from unstable_run_traces + union all select * from unstable_run_metrics_2) join queries using query group by query, metric having d > 0.5 order by any(rd[3]) desc, d desc ; + +create table stacks engine File(TSV, 'stacks.rep') as + select + query, + arrayStringConcat( + arrayMap(x -> joinGet(right_addresses_join, 'name', x), + arrayReverse(trace) + ), + ';' + ) readable_trace, + count() + from right_trace_log + join unstable_query_runs using query_id + group by query, trace + ; " +IFS=$'\n' +for q in $(cut -d' ' -f1 stacks.rep | sort | uniq) +do + grep -F "$q" stacks.rep | cut -d' ' -f 2- | tee "$q.stacks.rep" | ~/fg/flamegraph.pl > "$q.svg" & +done +wait +unset IFS + # Remember that grep sets error code when nothing is found, hence the bayan # operator grep Exception:[^:] *-err.log > run-errors.log ||: diff --git a/docker/test/performance-comparison/entrypoint.sh b/docker/test/performance-comparison/entrypoint.sh index bb5c2f6dbee..1248492914e 100755 --- a/docker/test/performance-comparison/entrypoint.sh +++ b/docker/test/performance-comparison/entrypoint.sh @@ -59,5 +59,5 @@ set +m dmesg > dmesg.log -7z a /output/output.7z *.log *.tsv *.html *.txt *.rep +7z a /output/output.7z *.log *.tsv *.html *.txt *.rep *.svg cp compare.log /output diff --git a/docker/test/performance-comparison/perf.py b/docker/test/performance-comparison/perf.py index 1205fc97ffd..13ba6301444 100755 --- a/docker/test/performance-comparison/perf.py +++ b/docker/test/performance-comparison/perf.py @@ -116,7 +116,7 @@ for q in test_queries: # Prewarm: run once on both servers. Helps to bring the data into memory, # precompile the queries, etc. for conn_index, c in enumerate(connections): - res = c.execute(q) + res = c.execute(q, query_id = 'prewarm {} {}'.format(0, q)) print('prewarm\t' + tsv_escape(q) + '\t' + str(conn_index) + '\t' + str(c.last_query.elapsed)) # Now, perform measured runs.