From ae715a8d1b644179773dfa390f7b5d0aeb82df23 Mon Sep 17 00:00:00 2001 From: Oleg Ershov Date: Thu, 14 May 2020 18:45:30 +0300 Subject: [PATCH 0001/1060] Add base --- src/Disks/DiskHDFS.cpp | 705 +++++++++++++++++++++++++++++++++ src/Disks/DiskHDFS.h | 112 ++++++ src/Disks/S3/DiskS3.cpp | 2 +- src/Disks/tests/gtest_disk.cpp | 11 + src/Disks/tests/gtest_disk.h | 1 + src/Disks/ya.make | 1 + src/IO/ReadBufferFromHDFS.cpp | 35 +- src/IO/ReadBufferFromHDFS.h | 7 +- 8 files changed, 871 insertions(+), 3 deletions(-) create mode 100644 src/Disks/DiskHDFS.cpp create mode 100644 src/Disks/DiskHDFS.h diff --git a/src/Disks/DiskHDFS.cpp b/src/Disks/DiskHDFS.cpp new file mode 100644 index 00000000000..606896810d8 --- /dev/null +++ b/src/Disks/DiskHDFS.cpp @@ -0,0 +1,705 @@ +#include "DiskHDFS.h" + +#include "DiskFactory.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; + extern const int PATH_ACCESS_DENIED; + extern const int FILE_ALREADY_EXISTS; + extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int UNKNOWN_FORMAT; + extern const int CANNOT_REMOVE_FILE; +} + +namespace +{ + String getRandomName() + { + std::uniform_int_distribution distribution('a', 'z'); + String res(32, ' '); /// The number of bits of entropy should be not less than 128. + for (auto & c : res) + c = distribution(thread_local_rng); + return res; + } + + /* + template + void throwIfError(Aws::Utils::Outcome && response) + { + if (!response.IsSuccess()) + { + const auto & err = response.GetError(); + throw Exception(err.GetMessage(), static_cast(err.GetErrorType())); + } + } + */ + + struct Metadata + { + /// Metadata file version. + static constexpr UInt32 VERSION = 1; + + using PathAndSize = std::pair; + + /// Disk path. + const String & disk_path; + /// Relative path to metadata file on local FS. + String metadata_file_path; + /// Total size of all HDFS objects. + size_t total_size; + /// HDFS objects paths and their sizes. + std::vector s3_objects; + /// Number of references (hardlinks) to this metadata file. + UInt32 ref_count; + + /// Load metadata by path or create empty if `create` flag is set. + explicit Metadata(const String & disk_path_, const String & metadata_file_path_, bool create = false) + : disk_path(disk_path_), metadata_file_path(metadata_file_path_), total_size(0), s3_objects(0), ref_count(0) + { + if (create) + return; + + ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ + + UInt32 version; + readIntText(version, buf); + + if (version != VERSION) + throw Exception( + "Unknown metadata file version. Path: " + disk_path + metadata_file_path + + " Version: " + std::to_string(version) + ", Expected version: " + std::to_string(VERSION), + ErrorCodes::UNKNOWN_FORMAT); + + assertChar('\n', buf); + + UInt32 s3_objects_count; + readIntText(s3_objects_count, buf); + assertChar('\t', buf); + readIntText(total_size, buf); + assertChar('\n', buf); + s3_objects.resize(s3_objects_count); + for (UInt32 i = 0; i < s3_objects_count; ++i) + { + String s3_object_path; + size_t s3_object_size; + readIntText(s3_object_size, buf); + assertChar('\t', buf); + readEscapedString(s3_object_path, buf); + assertChar('\n', buf); + s3_objects[i] = {s3_object_path, s3_object_size}; + } + + readIntText(ref_count, buf); + assertChar('\n', buf); + } + + void addObject(const String & path, size_t size) + { + total_size += size; + s3_objects.emplace_back(path, size); + } + + /// Fsync metadata file if 'sync' flag is set. + void save(bool sync = false) + { + WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); + + writeIntText(VERSION, buf); + writeChar('\n', buf); + + writeIntText(s3_objects.size(), buf); + writeChar('\t', buf); + writeIntText(total_size, buf); + writeChar('\n', buf); + for (const auto & [s3_object_path, s3_object_size] : s3_objects) + { + writeIntText(s3_object_size, buf); + writeChar('\t', buf); + writeEscapedString(s3_object_path, buf); + writeChar('\n', buf); + } + + writeIntText(ref_count, buf); + writeChar('\n', buf); + + buf.finalize(); + if (sync) + buf.sync(); + } + }; + + /// Reads data from HDFS using stored paths in metadata. + class ReadIndirectBufferFromHDFS final : public ReadBufferFromFileBase + { + public: + ReadIndirectBufferFromHDFS( + const String& hdfs_name_, const String & bucket_, Metadata metadata_, size_t buf_size_) + : hdfs_name(hdfs_name_), bucket(bucket_), metadata(std::move(metadata_)), buf_size(buf_size_) + { + } + + off_t seek(off_t offset_, int whence) override + { + if (whence == SEEK_CUR) + { + /// If position within current working buffer - shift pos. + if (working_buffer.size() && size_t(getPosition() + offset_) < absolute_position) + { + pos += offset_; + return getPosition(); + } + else + { + absolute_position += offset_; + } + } + else if (whence == SEEK_SET) + { + /// If position within current working buffer - shift pos. + if (working_buffer.size() && size_t(offset_) >= absolute_position - working_buffer.size() + && size_t(offset_) < absolute_position) + { + pos = working_buffer.end() - (absolute_position - offset_); + return getPosition(); + } + else + { + absolute_position = offset_; + } + } + else + throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + + 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 (size_t i = 0; i < metadata.s3_objects.size(); ++i) + { + current_buf_idx = i; + const auto & [path, size] = metadata.s3_objects[i]; + std::cerr << "MetaData path and size " << path << " " << size << std::endl; + if (size > offset) + { + auto buf = std::make_unique(hdfs_name + path); + std::cerr << "Make offset " << offset << std::endl; + buf->seek(offset, SEEK_SET); + return buf; + } + offset -= size; + + } + std::cerr << "We return nullptr\n"; + return nullptr; + } + + bool nextImpl() override + { + /// Find first available buffer that fits to given offset. + if (!current_buf) + current_buf = initialize(); + + /// 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; + } + + /// If there is no available buffers - nothing to read. + if (current_buf_idx + 1 >= metadata.s3_objects.size()) + return false; + + ++current_buf_idx; + const auto & path = metadata.s3_objects[current_buf_idx].first; + current_buf = std::make_unique(hdfs_name + "/" + path); + current_buf->next(); + working_buffer = current_buf->buffer(); + absolute_position += working_buffer.size(); + + return true; + } + + const String & hdfs_name; + const String & bucket; + Metadata metadata; + size_t buf_size; + + size_t absolute_position = 0; + size_t current_buf_idx = 0; + std::unique_ptr current_buf; + }; + + /// Stores data in HDFS and adds the object key (HDFS path) and object size to metadata file on local FS. + class WriteIndirectBufferFromHDFS final : public WriteBufferFromFileBase + { + public: + WriteIndirectBufferFromHDFS( + const String & hdfs_name_, + const String & hdfs_path_, + Metadata metadata_, + size_t buf_size_) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) + , impl(WriteBufferFromHDFS(hdfs_name_)) + , metadata(std::move(metadata_)) + , s3_path(hdfs_path_) + { + } + + ~WriteIndirectBufferFromHDFS() override + { + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + void finalize() override + { + if (finalized) + return; + + next(); + impl.finalize(); + + metadata.addObject(s3_path, count()); + metadata.save(); + + finalized = true; + } + + void sync() override + { + if (finalized) + metadata.save(true); + } + + std::string getFileName() const override { return metadata.metadata_file_path; } + + private: + void nextImpl() override + { + /// Transfer current working buffer to WriteBufferFromHDFS. + impl.swap(*this); + + /// Write actual data to HDFS. + impl.next(); + + /// Return back working buffer. + impl.swap(*this); + } + + WriteBufferFromHDFS impl; + bool finalized = false; + Metadata metadata; + String s3_path; + }; +} + + +class DiskHDFSDirectoryIterator final : public IDiskDirectoryIterator +{ +public: + DiskHDFSDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {} + + void next() override { ++iter; } + + bool isValid() const override { return iter != Poco::DirectoryIterator(); } + + String path() const override + { + if (iter->isDirectory()) + return folder_path + iter.name() + '/'; + else + return folder_path + iter.name(); + } + + String name() const override { return iter.name(); } + +private: + Poco::DirectoryIterator iter; + String folder_path; +}; + + +using DiskHDFSPtr = std::shared_ptr; + +class DiskHDFSReservation final : public IReservation +{ +public: + DiskHDFSReservation(const DiskHDFSPtr & disk_, UInt64 size_) + : disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_) + { + } + + UInt64 getSize() const override { return size; } + + DiskPtr getDisk() const override { return disk; } + + void update(UInt64 new_size) override + { + std::lock_guard lock(disk->reservation_mutex); + disk->reserved_bytes -= size; + size = new_size; + disk->reserved_bytes += size; + } + + ~DiskHDFSReservation() override; + +private: + DiskHDFSPtr disk; + UInt64 size; + CurrentMetrics::Increment metric_increment; +}; + + +DiskHDFS::DiskHDFS( + String name_, + String hdfs_name_, + String metadata_path_) + : name(std::move(name_)) + , hdfs_name(std::move(hdfs_name_)) + , metadata_path(std::move(metadata_path_)) + , builder(createHDFSBuilder(hdfs_name)) + , fs(createHDFSFS(builder.get())) +{ +} + +ReservationPtr DiskHDFS::reserve(UInt64 bytes) +{ + if (!tryReserve(bytes)) + return {}; + return std::make_unique(std::static_pointer_cast(shared_from_this()), bytes); +} + +bool DiskHDFS::exists(const String & path) const +{ + return Poco::File(metadata_path + path).exists(); +} + +bool DiskHDFS::isFile(const String & path) const +{ + return Poco::File(metadata_path + path).isFile(); +} + +bool DiskHDFS::isDirectory(const String & path) const +{ + return Poco::File(metadata_path + path).isDirectory(); +} + +size_t DiskHDFS::getFileSize(const String & path) const +{ + Metadata metadata(metadata_path, path); + return metadata.total_size; +} + +void DiskHDFS::createDirectory(const String & path) +{ + Poco::File(metadata_path + path).createDirectory(); +} + +void DiskHDFS::createDirectories(const String & path) +{ + Poco::File(metadata_path + path).createDirectories(); +} + +DiskDirectoryIteratorPtr DiskHDFS::iterateDirectory(const String & path) +{ + return std::make_unique(metadata_path + path, path); +} + +void DiskHDFS::clearDirectory(const String & path) +{ + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + if (isFile(it->path())) + remove(it->path()); +} + +void DiskHDFS::moveFile(const String & from_path, const String & to_path) +{ + if (exists(to_path)) + throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS); + Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); +} + +void DiskHDFS::replaceFile(const String & from_path, const String & to_path) +{ + Poco::File from_file(metadata_path + from_path); + Poco::File to_file(metadata_path + to_path); + if (to_file.exists()) + { + Poco::File tmp_file(metadata_path + to_path + ".old"); + to_file.renameTo(tmp_file.path()); + from_file.renameTo(metadata_path + to_path); + remove(to_path + ".old"); + } + else + from_file.renameTo(to_file.path()); +} + +void DiskHDFS::copyFile(const String & from_path, const String & to_path) +{ + if (exists(to_path)) + remove(to_path); + + Metadata from(metadata_path, from_path); + Metadata to(metadata_path, to_path, true); + + for (const auto & [path, size] : from.s3_objects) + { + auto new_path = hdfs_name + getRandomName(); + /// TODO:: hdfs copy semantics + /* + Aws::HDFS::Model::CopyObjectRequest req; + req.SetCopySource(bucket + "/" + path); + req.SetBucket(bucket); + req.SetKey(new_path); + throwIfError(client->CopyObject(req)); + */ + throw Exception("is not implemented yet", 1); + to.addObject(new_path, size); + } + + to.save(); +} + +std::unique_ptr DiskHDFS::readFile(const String & path, size_t buf_size, size_t, size_t, size_t) const +{ + Metadata metadata(metadata_path, path); + + std::cerr << "Read Metadata: objects size " << metadata.s3_objects.size() << " " + << "files: " << metadata.total_size << " " + << "file path: " << metadata.metadata_file_path << " " + << "disk path: " << metadata.disk_path << std::endl; + + LOG_DEBUG( + &Logger::get("DiskHDFS"), + "Read from file by path: " << backQuote(metadata_path + path) << " Existing HDFS objects: " << metadata.s3_objects.size()); + + return std::make_unique(hdfs_name + path, "", metadata, buf_size); +} + +std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t, size_t) +{ + bool exist = exists(path); + /// Path to store new HDFS object. + auto file_name = getRandomName(); + auto HDFS_path = hdfs_name + file_name; + if (!exist || mode == WriteMode::Rewrite) + { + /// If metadata file exists - remove and new. + if (exist) + remove(path); + std::cerr << metadata_path << std::endl; + Metadata metadata(metadata_path, path, true); + /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. + metadata.save(); + + LOG_DEBUG(&Logger::get("DiskHDFS"), "Write to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path); + + return std::make_unique(HDFS_path, file_name, metadata, buf_size); + } + else + { + Metadata metadata(metadata_path, path); + + LOG_DEBUG( + &Logger::get("DiskHDFS"), + "Append to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path + << " Existing HDFS objects: " << metadata.s3_objects.size()); + + return std::make_unique(HDFS_path, file_name, metadata, buf_size); + } +} + +void DiskHDFS::remove(const String & path) +{ + LOG_DEBUG(&Logger::get("DiskHDFS"), "Remove file by path: " << backQuote(metadata_path + path)); + + Poco::File file(metadata_path + path); + if (file.isFile()) + { + Metadata metadata(metadata_path, path); + + /// If there is no references - delete content from HDFS. + if (metadata.ref_count == 0) + { + file.remove(); + for (const auto & [s3_object_path, _] : metadata.s3_objects) + { + auto hdfs_path = "gtest/" + s3_object_path; + int res = hdfsDelete(fs.get(), hdfs_path.c_str(), 0); + if (res == -1) + throw Exception("fuck " + hdfs_path, 1); + } + } + else /// In other case decrement number of references, save metadata and delete file. + { + --metadata.ref_count; + metadata.save(); + file.remove(); + } + } + else + file.remove(); +} + +void DiskHDFS::removeRecursive(const String & path) +{ + checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. + + Poco::File file(metadata_path + path); + if (file.isFile()) + { + remove(path); + } + else + { + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + removeRecursive(it->path()); + file.remove(); + } +} + + +bool DiskHDFS::tryReserve(UInt64 bytes) +{ + std::lock_guard lock(reservation_mutex); + if (bytes == 0) + { + LOG_DEBUG(&Logger::get("DiskHDFS"), "Reserving 0 bytes on HDFS disk " << backQuote(name)); + ++reservation_count; + return true; + } + + auto available_space = getAvailableSpace(); + UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); + if (unreserved_space >= bytes) + { + LOG_DEBUG( + &Logger::get("DiskHDFS"), + "Reserving " << formatReadableSizeWithBinarySuffix(bytes) << " on disk " << backQuote(name) << ", having unreserved " + << formatReadableSizeWithBinarySuffix(unreserved_space) << "."); + ++reservation_count; + reserved_bytes += bytes; + return true; + } + return false; +} + +void DiskHDFS::listFiles(const String & path, std::vector & file_names) +{ + for (auto it = iterateDirectory(path); it->isValid(); it->next()) + file_names.push_back(it->name()); +} + +void DiskHDFS::setLastModified(const String & path, const Poco::Timestamp & timestamp) +{ + Poco::File(metadata_path + path).setLastModified(timestamp); +} + +Poco::Timestamp DiskHDFS::getLastModified(const String & path) +{ + return Poco::File(metadata_path + path).getLastModified(); +} + +void DiskHDFS::createHardLink(const String & src_path, const String & dst_path) +{ + /// Increment number of references. + Metadata src(metadata_path, src_path); + ++src.ref_count; + src.save(); + + /// Create FS hardlink to metadata file. + DB::createHardLink(metadata_path + src_path, metadata_path + dst_path); +} + +void DiskHDFS::createFile(const String & path) +{ + /// Create empty metadata file. + Metadata metadata(metadata_path, path, true); + metadata.save(); +} + +void DiskHDFS::setReadOnly(const String & path) +{ + Poco::File(metadata_path + path).setReadOnly(true); +} + +DiskHDFSReservation::~DiskHDFSReservation() +{ + try + { + std::lock_guard lock(disk->reservation_mutex); + if (disk->reserved_bytes < size) + { + disk->reserved_bytes = 0; + LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'."); + } + else + { + disk->reserved_bytes -= size; + } + + if (disk->reservation_count == 0) + LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'."); + else + --disk->reservation_count; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + +void registerDiskHDFS(DiskFactory & factory) +{ + auto creator = [](const String & name, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const Context & context) -> DiskPtr { + String path = config.getString(config_prefix + ".path", ""); + return std::make_shared(name, path, ""); + }; + factory.registerDiskType("hdfs", creator); +} + +} diff --git a/src/Disks/DiskHDFS.h b/src/Disks/DiskHDFS.h new file mode 100644 index 00000000000..5b35a90286c --- /dev/null +++ b/src/Disks/DiskHDFS.h @@ -0,0 +1,112 @@ +#pragma once + +#include "Disks/DiskFactory.h" + +#include +#include +#include + + +namespace DB +{ +/** + * Storage for persisting data in HDFS and metadata on the local disk. + * Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file) + * that contains HDFS object key with actual data. + */ +class DiskHDFS : public IDisk +{ +public: + friend class DiskHDFSReservation; + + DiskHDFS( + String name_, + String hdfs_name_, + String metadata_path_); + + const String & getName() const override { return name; } + + const String & getPath() const override { return metadata_path; } + + ReservationPtr reserve(UInt64 bytes) override; + + UInt64 getTotalSpace() const override { return std::numeric_limits::max(); } + + UInt64 getAvailableSpace() const override { return std::numeric_limits::max(); } + + UInt64 getUnreservedSpace() const override { return std::numeric_limits::max(); } + + UInt64 getKeepingFreeSpace() const override { return 0; } + + bool exists(const String & path) const override; + + bool isFile(const String & path) const override; + + bool isDirectory(const String & path) const override; + + size_t getFileSize(const String & path) const override; + + void createDirectory(const String & path) override; + + void createDirectories(const String & path) override; + + void clearDirectory(const String & path) override; + + void moveDirectory(const String & from_path, const String & to_path) override { moveFile(from_path, to_path); } + + DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; + + void moveFile(const String & from_path, const String & to_path) override; + + void replaceFile(const String & from_path, const String & to_path) override; + + void copyFile(const String & from_path, const String & to_path) override; + + void listFiles(const String & path, std::vector & file_names) override; + + std::unique_ptr readFile( + const String & path, + size_t buf_size, + size_t estimated_size, + size_t aio_threshold, + size_t mmap_threshold) const override; + + std::unique_ptr writeFile( + const String & path, + size_t buf_size, + WriteMode mode, + size_t estimated_size, + size_t aio_threshold) override; + + void remove(const String & path) override; + + void removeRecursive(const String & path) override; + + void createHardLink(const String & src_path, const String & dst_path) override; + + void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; + + Poco::Timestamp getLastModified(const String & path) override; + + void createFile(const String & path) override; + + void setReadOnly(const String & path) override; + +private: + bool tryReserve(UInt64 bytes); + +private: + const String name; + const String hdfs_name; + const String metadata_path; + + HDFSBuilderPtr builder; + HDFSFSPtr fs; + + UInt64 reserved_bytes = 0; + UInt64 reservation_count = 0; + std::mutex reservation_mutex; + std::mutex copying_mutex; +}; + +} diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 47ca8231001..b837a31668a 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -3,7 +3,7 @@ #include "Disks/DiskFactory.h" #include -#include +#include #include #include #include diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index 525b5e6ce38..2b877429a86 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -144,3 +144,14 @@ TYPED_TEST(DiskTest, iterateDirectory) EXPECT_FALSE(iter->isValid()); } } + +TEST(DiskHdfsTest, testHdfsCreation) +{ + auto disk = DB::DiskHDFS("gtesthdfs", "hdfs://localhost:9010/gtest/", "/home/ershov-ov/metadata/"); + auto out = disk.writeFile("keek", 1024, DB::WriteMode::Rewrite, 1024, 1024); + writeString("test data", *out); + DB::String d; + auto in = disk.readFile("keek", 1024, 1024, 1024, 1024); + readString(d, *in); + EXPECT_EQ("test_data", d); +} \ No newline at end of file diff --git a/src/Disks/tests/gtest_disk.h b/src/Disks/tests/gtest_disk.h index 329ee04d16d..6549f9c67eb 100644 --- a/src/Disks/tests/gtest_disk.h +++ b/src/Disks/tests/gtest_disk.h @@ -1,6 +1,7 @@ #include #include #include +#include template DB::DiskPtr createDisk(); diff --git a/src/Disks/ya.make b/src/Disks/ya.make index a14024e7af3..f8fc9155980 100644 --- a/src/Disks/ya.make +++ b/src/Disks/ya.make @@ -7,6 +7,7 @@ PEERDIR( SRCS( DiskFactory.cpp DiskLocal.cpp + DiskHDFS.cpp DiskMemory.cpp DiskSelector.cpp IDisk.cpp diff --git a/src/IO/ReadBufferFromHDFS.cpp b/src/IO/ReadBufferFromHDFS.cpp index d6dfd12bd82..be048ddbd77 100644 --- a/src/IO/ReadBufferFromHDFS.cpp +++ b/src/IO/ReadBufferFromHDFS.cpp @@ -11,6 +11,8 @@ namespace ErrorCodes { extern const int NETWORK_ERROR; extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int CANNOT_TELL_THROUGH_FILE; } ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; @@ -45,6 +47,24 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl return bytes_read; } + int seek(off_t off) const + { + int seek_status = hdfsSeek(fs.get(), fin, off); + if (seek_status != 0) + throw Exception("Fail to seek HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + return seek_status; + } + + int tell() const + { + int bytes_offset = hdfsTell(fs.get(), fin); + if (bytes_offset == -1) + throw Exception("Fail to tell HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + return bytes_offset; + } + ~ReadBufferFromHDFSImpl() { hdfsCloseFile(fs.get(), fin); @@ -52,7 +72,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl }; ReadBufferFromHDFS::ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size) - : BufferWithOwnMemory(buf_size) + : BufferWithOwnMemory(buf_size) , impl(std::make_unique(hdfs_name_)) { } @@ -69,6 +89,19 @@ bool ReadBufferFromHDFS::nextImpl() return true; } +off_t ReadBufferFromHDFS::seek(off_t off, int whence) +{ + if (whence != SEEK_SET) + throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + impl->seek(off); + return off; +} + +off_t ReadBufferFromHDFS::getPosition() +{ + return impl->tell(); +} + } #endif diff --git a/src/IO/ReadBufferFromHDFS.h b/src/IO/ReadBufferFromHDFS.h index e27159f2e98..e9e0dc5520c 100644 --- a/src/IO/ReadBufferFromHDFS.h +++ b/src/IO/ReadBufferFromHDFS.h @@ -4,6 +4,7 @@ #if USE_HDFS #include +#include "SeekableReadBuffer.h" #include #include #include @@ -13,7 +14,7 @@ namespace DB /** Accepts HDFS path to file and opens it. * Closes file by himself (thus "owns" a file descriptor). */ -class ReadBufferFromHDFS : public BufferWithOwnMemory +class ReadBufferFromHDFS : public BufferWithOwnMemory { struct ReadBufferFromHDFSImpl; std::unique_ptr impl; @@ -23,6 +24,10 @@ public: ~ReadBufferFromHDFS() override; bool nextImpl() override; + + off_t seek(off_t off, int whence) override; + off_t getPosition() override; + }; } #endif From dd210cfeaf1a552af78d86f76ce35b77f3bed6b7 Mon Sep 17 00:00:00 2001 From: Oleg Ershov Date: Fri, 15 May 2020 20:55:18 +0300 Subject: [PATCH 0002/1060] Fix disk issues --- src/Disks/DiskHDFS.cpp | 28 ++++++++++++++++---- src/Disks/tests/gtest_disk.cpp | 47 +++++++++++++++++++++++++++++----- 2 files changed, 64 insertions(+), 11 deletions(-) diff --git a/src/Disks/DiskHDFS.cpp b/src/Disks/DiskHDFS.cpp index 606896810d8..05a2d4f0792 100644 --- a/src/Disks/DiskHDFS.cpp +++ b/src/Disks/DiskHDFS.cpp @@ -97,6 +97,7 @@ namespace UInt32 s3_objects_count; readIntText(s3_objects_count, buf); + std::cerr << "Metadata create, objects count " << s3_objects_count << std::endl; assertChar('\t', buf); readIntText(total_size, buf); assertChar('\n', buf); @@ -214,6 +215,7 @@ namespace std::cerr << "MetaData path and size " << path << " " << size << std::endl; if (size > offset) { + std::cerr << "Make ReadBuffer from " << (hdfs_name + path) << std::endl; auto buf = std::make_unique(hdfs_name + path); std::cerr << "Make offset " << offset << std::endl; buf->seek(offset, SEEK_SET); @@ -514,7 +516,7 @@ std::unique_ptr DiskHDFS::readFile(const String & path, &Logger::get("DiskHDFS"), "Read from file by path: " << backQuote(metadata_path + path) << " Existing HDFS objects: " << metadata.s3_objects.size()); - return std::make_unique(hdfs_name + path, "", metadata, buf_size); + return std::make_unique(hdfs_name, "", metadata, buf_size); } std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t, size_t) @@ -531,7 +533,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path std::cerr << metadata_path << std::endl; Metadata metadata(metadata_path, path, true); /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. - metadata.save(); + // metadata.save(); LOG_DEBUG(&Logger::get("DiskHDFS"), "Write to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path); @@ -565,7 +567,7 @@ void DiskHDFS::remove(const String & path) file.remove(); for (const auto & [s3_object_path, _] : metadata.s3_objects) { - auto hdfs_path = "gtest/" + s3_object_path; + auto hdfs_path = "/gtest/" + s3_object_path; int res = hdfsDelete(fs.get(), hdfs_path.c_str(), 0); if (res == -1) throw Exception("fuck " + hdfs_path, 1); @@ -696,8 +698,24 @@ void registerDiskHDFS(DiskFactory & factory) const Poco::Util::AbstractConfiguration & config, const String & config_prefix, const Context & context) -> DiskPtr { - String path = config.getString(config_prefix + ".path", ""); - return std::make_shared(name, path, ""); + const auto * disk_config = config.createView(config_prefix); + + Poco::File disk{context.getPath() + "disks/" + name}; + disk.createDirectories(); + + DB::String uri{disk_config->getString("endpoint")}; + if (uri.back() != '/') + throw Exception("HDFS path must ends with '/', but '" + uri + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); + + // String metadata_path = context.getPath() + "disks/" + name + "/"; + String metadata_path = "/home/ershov-ov/metadata/"; + + + return std::make_shared( + name, + uri, + metadata_path + ); }; factory.registerDiskType("hdfs", creator); } diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index 2b877429a86..f846ecda966 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -148,10 +148,45 @@ TYPED_TEST(DiskTest, iterateDirectory) TEST(DiskHdfsTest, testHdfsCreation) { auto disk = DB::DiskHDFS("gtesthdfs", "hdfs://localhost:9010/gtest/", "/home/ershov-ov/metadata/"); - auto out = disk.writeFile("keek", 1024, DB::WriteMode::Rewrite, 1024, 1024); - writeString("test data", *out); - DB::String d; - auto in = disk.readFile("keek", 1024, 1024, 1024, 1024); - readString(d, *in); - EXPECT_EQ("test_data", d); + { + auto out = disk.writeFile("keek", 1024, DB::WriteMode::Rewrite, 1024, 1024); + writeString("test data", *out); + } + { + DB::String d; + auto in = disk.readFile("keek", 1024, 1024, 1024, 1024); + readString(d, *in); + EXPECT_EQ("test data", d); + } + + { + std::unique_ptr out = disk.writeFile("test_file", 1024, DB::WriteMode::Rewrite, 1024, 1024); + writeString("test data", *out); + } + + // Test SEEK_SET + { + String buf(4, '0'); + std::unique_ptr in = disk.readFile("test_file", 1024, 1024, 1024, 1024); + + in->seek(5, SEEK_SET); + + in->readStrict(buf.data(), 4); + EXPECT_EQ("data", buf); + } + + // Test SEEK_CUR + { + std::unique_ptr in = disk.readFile("test_file", 1024, 1024, 1024, 1024); + String buf(4, '0'); + + in->readStrict(buf.data(), 4); + EXPECT_EQ("test", buf); + + // Skip whitespace + in->seek(1, SEEK_CUR); + + in->readStrict(buf.data(), 4); + EXPECT_EQ("data", buf); + } } \ No newline at end of file From 8858319281c15ceea894eace3c9e40150369207c Mon Sep 17 00:00:00 2001 From: Oleg Ershov Date: Wed, 20 May 2020 06:14:58 +0300 Subject: [PATCH 0003/1060] global update --- src/Disks/DiskHDFS.cpp | 86 +++++++++++++++------------------- src/Disks/registerDisks.cpp | 4 ++ src/Disks/tests/gtest_disk.cpp | 8 ++-- src/Disks/tests/gtest_disk.h | 3 +- src/IO/ReadBufferFromHDFS.cpp | 51 +++++++++++++------- src/IO/WriteBufferFromHDFS.cpp | 2 +- 6 files changed, 83 insertions(+), 71 deletions(-) diff --git a/src/Disks/DiskHDFS.cpp b/src/Disks/DiskHDFS.cpp index 05a2d4f0792..1e5e2e46270 100644 --- a/src/Disks/DiskHDFS.cpp +++ b/src/Disks/DiskHDFS.cpp @@ -71,13 +71,13 @@ namespace /// Total size of all HDFS objects. size_t total_size; /// HDFS objects paths and their sizes. - std::vector s3_objects; + std::vector hdfs_objects; /// Number of references (hardlinks) to this metadata file. UInt32 ref_count; /// Load metadata by path or create empty if `create` flag is set. explicit Metadata(const String & disk_path_, const String & metadata_file_path_, bool create = false) - : disk_path(disk_path_), metadata_file_path(metadata_file_path_), total_size(0), s3_objects(0), ref_count(0) + : disk_path(disk_path_), metadata_file_path(metadata_file_path_), total_size(0), hdfs_objects(0), ref_count(0) { if (create) return; @@ -95,22 +95,21 @@ namespace assertChar('\n', buf); - UInt32 s3_objects_count; - readIntText(s3_objects_count, buf); - std::cerr << "Metadata create, objects count " << s3_objects_count << std::endl; + UInt32 hdfs_objects_count; + readIntText(hdfs_objects_count, buf); assertChar('\t', buf); readIntText(total_size, buf); assertChar('\n', buf); - s3_objects.resize(s3_objects_count); - for (UInt32 i = 0; i < s3_objects_count; ++i) + hdfs_objects.resize(hdfs_objects_count); + for (UInt32 i = 0; i < hdfs_objects_count; ++i) { - String s3_object_path; - size_t s3_object_size; - readIntText(s3_object_size, buf); + String hdfs_object_path; + size_t hdfs_object_size; + readIntText(hdfs_object_size, buf); assertChar('\t', buf); - readEscapedString(s3_object_path, buf); + readEscapedString(hdfs_object_path, buf); assertChar('\n', buf); - s3_objects[i] = {s3_object_path, s3_object_size}; + hdfs_objects[i] = {hdfs_object_path, hdfs_object_size}; } readIntText(ref_count, buf); @@ -120,7 +119,7 @@ namespace void addObject(const String & path, size_t size) { total_size += size; - s3_objects.emplace_back(path, size); + hdfs_objects.emplace_back(path, size); } /// Fsync metadata file if 'sync' flag is set. @@ -131,15 +130,15 @@ namespace writeIntText(VERSION, buf); writeChar('\n', buf); - writeIntText(s3_objects.size(), buf); + writeIntText(hdfs_objects.size(), buf); writeChar('\t', buf); writeIntText(total_size, buf); writeChar('\n', buf); - for (const auto & [s3_object_path, s3_object_size] : s3_objects) + for (const auto & [hdfs_object_path, hdfs_object_size] : hdfs_objects) { - writeIntText(s3_object_size, buf); + writeIntText(hdfs_object_size, buf); writeChar('\t', buf); - writeEscapedString(s3_object_path, buf); + writeEscapedString(hdfs_object_path, buf); writeChar('\n', buf); } @@ -208,23 +207,21 @@ namespace std::unique_ptr initialize() { size_t offset = absolute_position; - for (size_t i = 0; i < metadata.s3_objects.size(); ++i) + + for (size_t i = 0; i < metadata.hdfs_objects.size(); ++i) { current_buf_idx = i; - const auto & [path, size] = metadata.s3_objects[i]; - std::cerr << "MetaData path and size " << path << " " << size << std::endl; + const auto & [path, size] = metadata.hdfs_objects[i]; + if (size > offset) { - std::cerr << "Make ReadBuffer from " << (hdfs_name + path) << std::endl; - auto buf = std::make_unique(hdfs_name + path); - std::cerr << "Make offset " << offset << std::endl; + auto buf = std::make_unique(hdfs_name + path, buf_size); buf->seek(offset, SEEK_SET); return buf; } offset -= size; } - std::cerr << "We return nullptr\n"; return nullptr; } @@ -243,12 +240,12 @@ namespace } /// If there is no available buffers - nothing to read. - if (current_buf_idx + 1 >= metadata.s3_objects.size()) + if (current_buf_idx + 1 >= metadata.hdfs_objects.size()) return false; ++current_buf_idx; - const auto & path = metadata.s3_objects[current_buf_idx].first; - current_buf = std::make_unique(hdfs_name + "/" + path); + const auto & path = metadata.hdfs_objects[current_buf_idx].first; + current_buf = std::make_unique(hdfs_name + path, buf_size); current_buf->next(); working_buffer = current_buf->buffer(); absolute_position += working_buffer.size(); @@ -276,9 +273,9 @@ namespace Metadata metadata_, size_t buf_size_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , impl(WriteBufferFromHDFS(hdfs_name_)) + , impl(WriteBufferFromHDFS(hdfs_name_, buf_size_)) , metadata(std::move(metadata_)) - , s3_path(hdfs_path_) + , hdfs_path(hdfs_path_) { } @@ -302,7 +299,7 @@ namespace next(); impl.finalize(); - metadata.addObject(s3_path, count()); + metadata.addObject(hdfs_path, count()); metadata.save(); finalized = true; @@ -332,7 +329,7 @@ namespace WriteBufferFromHDFS impl; bool finalized = false; Metadata metadata; - String s3_path; + String hdfs_path; }; } @@ -485,7 +482,7 @@ void DiskHDFS::copyFile(const String & from_path, const String & to_path) Metadata from(metadata_path, from_path); Metadata to(metadata_path, to_path, true); - for (const auto & [path, size] : from.s3_objects) + for (const auto & [path, size] : from.hdfs_objects) { auto new_path = hdfs_name + getRandomName(); /// TODO:: hdfs copy semantics @@ -507,15 +504,10 @@ std::unique_ptr DiskHDFS::readFile(const String & path, { Metadata metadata(metadata_path, path); - std::cerr << "Read Metadata: objects size " << metadata.s3_objects.size() << " " - << "files: " << metadata.total_size << " " - << "file path: " << metadata.metadata_file_path << " " - << "disk path: " << metadata.disk_path << std::endl; - LOG_DEBUG( &Logger::get("DiskHDFS"), - "Read from file by path: " << backQuote(metadata_path + path) << " Existing HDFS objects: " << metadata.s3_objects.size()); - + "Read from file by path: " << backQuote(metadata_path + path) << " Existing HDFS objects: " << metadata.hdfs_objects.size()); + return std::make_unique(hdfs_name, "", metadata, buf_size); } @@ -530,10 +522,9 @@ std::unique_ptr DiskHDFS::writeFile(const String & path /// If metadata file exists - remove and new. if (exist) remove(path); - std::cerr << metadata_path << std::endl; Metadata metadata(metadata_path, path, true); /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. - // metadata.save(); + metadata.save(); LOG_DEBUG(&Logger::get("DiskHDFS"), "Write to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path); @@ -546,7 +537,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path LOG_DEBUG( &Logger::get("DiskHDFS"), "Append to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path - << " Existing HDFS objects: " << metadata.s3_objects.size()); + << " Existing HDFS objects: " << metadata.hdfs_objects.size()); return std::make_unique(HDFS_path, file_name, metadata, buf_size); } @@ -565,9 +556,10 @@ void DiskHDFS::remove(const String & path) if (metadata.ref_count == 0) { file.remove(); - for (const auto & [s3_object_path, _] : metadata.s3_objects) + for (const auto & [hdfs_object_path, _] : metadata.hdfs_objects) { - auto hdfs_path = "/gtest/" + s3_object_path; + const size_t begin_of_path = hdfs_name.find('/', hdfs_name.find("//") + 2); + const std::string hdfs_path = hdfs_name.substr(begin_of_path) + hdfs_object_path; int res = hdfsDelete(fs.get(), hdfs_path.c_str(), 0); if (res == -1) throw Exception("fuck " + hdfs_path, 1); @@ -707,10 +699,8 @@ void registerDiskHDFS(DiskFactory & factory) if (uri.back() != '/') throw Exception("HDFS path must ends with '/', but '" + uri + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); - // String metadata_path = context.getPath() + "disks/" + name + "/"; - String metadata_path = "/home/ershov-ov/metadata/"; - - + String metadata_path = context.getPath() + "disks/" + name + "/"; + return std::make_shared( name, uri, diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 2da39e62b19..963b3bd8f87 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -13,6 +13,8 @@ void registerDiskMemory(DiskFactory & factory); #if USE_AWS_S3 void registerDiskS3(DiskFactory & factory); #endif +void registerDiskHDFS(DiskFactory & factory); + void registerDisks() { @@ -20,6 +22,8 @@ void registerDisks() registerDiskLocal(factory); registerDiskMemory(factory); + registerDiskHDFS(factory); + #if USE_AWS_S3 registerDiskS3(factory); #endif diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index f846ecda966..5875877a14b 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -160,24 +160,24 @@ TEST(DiskHdfsTest, testHdfsCreation) } { - std::unique_ptr out = disk.writeFile("test_file", 1024, DB::WriteMode::Rewrite, 1024, 1024); + std::unique_ptr out = disk.writeFile("test_file", 1, DB::WriteMode::Rewrite, 1024, 1); writeString("test data", *out); } // Test SEEK_SET { String buf(4, '0'); - std::unique_ptr in = disk.readFile("test_file", 1024, 1024, 1024, 1024); + std::unique_ptr in = disk.readFile("test_file", 1, 1024, 1024, 1); in->seek(5, SEEK_SET); in->readStrict(buf.data(), 4); EXPECT_EQ("data", buf); - } + } // Test SEEK_CUR { - std::unique_ptr in = disk.readFile("test_file", 1024, 1024, 1024, 1024); + std::unique_ptr in = disk.readFile("test_file", 1, 1024, 1024, 1); String buf(4, '0'); in->readStrict(buf.data(), 4); diff --git a/src/Disks/tests/gtest_disk.h b/src/Disks/tests/gtest_disk.h index 6549f9c67eb..56b3e573370 100644 --- a/src/Disks/tests/gtest_disk.h +++ b/src/Disks/tests/gtest_disk.h @@ -1,7 +1,8 @@ #include #include -#include #include +#include + template DB::DiskPtr createDisk(); diff --git a/src/IO/ReadBufferFromHDFS.cpp b/src/IO/ReadBufferFromHDFS.cpp index be048ddbd77..f7c5b0fe2b3 100644 --- a/src/IO/ReadBufferFromHDFS.cpp +++ b/src/IO/ReadBufferFromHDFS.cpp @@ -23,6 +23,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl hdfsFile fin; HDFSBuilderPtr builder; HDFSFSPtr fs; + off_t offset = 0; + bool initialized = false; explicit ReadBufferFromHDFSImpl(const std::string & hdfs_name_) : hdfs_uri(hdfs_name_) @@ -36,10 +38,28 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl if (fin == nullptr) throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), ErrorCodes::CANNOT_OPEN_FILE); + } - int read(char * start, size_t size) const + void initialize() { + if (offset) + { + int seek_status = hdfsSeek(fs.get(), fin, offset); + if (seek_status != 0) + throw Exception("Fail to seek HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + } + } + + int read(char * start, size_t size) + { + if (!initialized) + { + initialize(); + initialized = true; + } + int bytes_read = hdfsRead(fs.get(), fin, start, size); if (bytes_read < 0) throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), @@ -47,22 +67,22 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl return bytes_read; } - int seek(off_t off) const + int seek(off_t off, int whence) { - int seek_status = hdfsSeek(fs.get(), fin, off); - if (seek_status != 0) - throw Exception("Fail to seek HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - return seek_status; + 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); + + offset = off; + return offset; } int tell() const { - int bytes_offset = hdfsTell(fs.get(), fin); - if (bytes_offset == -1) - throw Exception("Fail to tell HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - return bytes_offset; + return offset; } ~ReadBufferFromHDFSImpl() @@ -91,15 +111,12 @@ bool ReadBufferFromHDFS::nextImpl() off_t ReadBufferFromHDFS::seek(off_t off, int whence) { - if (whence != SEEK_SET) - throw Exception("Only SEEK_SET mode is allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - impl->seek(off); - return off; + return impl->seek(off, whence); } off_t ReadBufferFromHDFS::getPosition() { - return impl->tell(); + return impl->tell() + count(); } } diff --git a/src/IO/WriteBufferFromHDFS.cpp b/src/IO/WriteBufferFromHDFS.cpp index 2176c77b914..1fe3b19f865 100644 --- a/src/IO/WriteBufferFromHDFS.cpp +++ b/src/IO/WriteBufferFromHDFS.cpp @@ -49,7 +49,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl } ~WriteBufferFromHDFSImpl() - { + { hdfsCloseFile(fs.get(), fout); } From fce1ca255df049ecde15e2de57010327798e53d9 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Wed, 20 Jan 2021 19:17:34 +0900 Subject: [PATCH 0004/1060] Implement sequenceNextNode --- .../AggregateFunctionNull.h | 11 +- .../AggregateFunctionSequenceNextNode.cpp | 102 ++++++ .../AggregateFunctionSequenceNextNode.h | 298 ++++++++++++++++++ src/AggregateFunctions/IAggregateFunction.h | 1 + .../registerAggregateFunctions.cpp | 2 + src/Columns/ColumnNullable.cpp | 1 - .../01656_sequence_next_node.reference | 120 +++++++ .../0_stateless/01656_sequence_next_node.sql | 51 +++ 8 files changed, 583 insertions(+), 3 deletions(-) create mode 100644 src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp create mode 100644 src/AggregateFunctions/AggregateFunctionSequenceNextNode.h create mode 100644 tests/queries/0_stateless/01656_sequence_next_node.reference create mode 100644 tests/queries/0_stateless/01656_sequence_next_node.sql diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 5c94e68cb26..3bfcacf7d7b 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -157,8 +157,15 @@ public: ColumnNullable & to_concrete = assert_cast(to); if (getFlag(place)) { - nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); + if (unlikely(nested_function->doesInsertResultNeedNullableColumn())) + { + nested_function->insertResultInto(nestedPlace(place), to_concrete, arena); + } + else + { + nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); + } } else { diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp new file mode 100644 index 00000000000..66f24ec8cbf --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -0,0 +1,102 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include "registerAggregateFunctions.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int BAD_ARGUMENTS; +} + +namespace +{ + +template +inline AggregateFunctionPtr createAggregateFunctionSequenceNextNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) +{ + if (descending_order) + return std::make_shared>(data_type, argument_types); + else + return std::make_shared>(data_type, argument_types); +} + +AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) +{ + bool descending_order = false; + + if (parameters.size() == 1) + { + auto type = parameters[0].getType(); + if (type != Field::Types::Int64 && type != Field::Types::UInt64) + throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); + + descending_order = parameters[0].get(); + } + else + throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (argument_types.size() < 3) + throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > 2 + 64) + throw Exception("Aggregate function " + name + " requires at most 66(timestamp, value_column, 64 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (const auto i : ext::range(2, argument_types.size())) + { + const auto * cond_arg = argument_types[i].get(); + if (!isUInt8(cond_arg)) + throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + " of aggregate function " + + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) + throw Exception{"Illegal type " + argument_types.front().get()->getName() + + " of second argument of aggregate function " + name + ", must be String", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + + DataTypePtr data_type; + if (typeid_cast(argument_types[1].get())) + data_type = argument_types[1]; + else + data_type = std::make_shared(argument_types[1]); + + WhichDataType timestamp_type(argument_types[0].get()); + if (timestamp_type.idx == TypeIndex::UInt8) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.idx == TypeIndex::UInt16) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.idx == TypeIndex::UInt32) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.idx == TypeIndex::UInt64) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.isDate()) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + if (timestamp_type.isDateTime()) + return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + + throw Exception{"Illegal type " + argument_types.front().get()->getName() + + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; +} + +} + +void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) +{ + AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; + + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); +} + +} diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h new file mode 100644 index 00000000000..a455e16e267 --- /dev/null +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -0,0 +1,298 @@ +#pragma once + +#include +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include + +#include + + +namespace DB +{ + +template +struct NodeBase +{ + UInt64 size; // size of payload + + DataTypeDateTime::FieldType event_time; + UInt64 events_bitmap; + + /// Returns pointer to actual payload + char * data() { return reinterpret_cast(this) + sizeof(Node); } + + const char * data() const { return reinterpret_cast(this) + sizeof(Node); } + + /// Clones existing node (does not modify next field) + Node * clone(Arena * arena) const + { + return reinterpret_cast( + const_cast(arena->alignedInsert(reinterpret_cast(this), sizeof(Node) + size, alignof(Node)))); + } + + /// Write node to buffer + void write(WriteBuffer & buf) const + { + writeVarUInt(size, buf); + buf.write(data(), size); + } + + /// Reads and allocates node from ReadBuffer's data (doesn't set next) + static Node * read(ReadBuffer & buf, Arena * arena) + { + UInt64 size; + readVarUInt(size, buf); + + Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); + node->size = size; + buf.read(node->data(), size); + return node; + } +}; + +struct NodeString : public NodeBase +{ + using Node = NodeString; + + /// Create node from string + static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) + { + StringRef string = assert_cast(column).getDataAt(row_num); + + Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + string.size, alignof(Node))); + node->size = string.size; + memcpy(node->data(), string.data, string.size); + + return node; + } + + void insertInto(IColumn & column) + { + assert_cast(column).insertData(data(), size); + } +}; + +template +struct SequenceNextNodeGeneralData +{ + // Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena + using Allocator = MixedAlignedArenaAllocator; + using Array = PODArray; + + Array value; + bool sorted = false; + + struct Comparator final + { + bool operator()(const Node * lhs, const Node * rhs) const + { + if (Descending) + return lhs->event_time == rhs->event_time ? + lhs->events_bitmap < rhs->events_bitmap: lhs->event_time > rhs->event_time; + else + return lhs->event_time == rhs->event_time ? + lhs->events_bitmap < rhs->events_bitmap : lhs->event_time < rhs->event_time; + } + }; + + void sort() + { + if (!sorted) + { + std::stable_sort(std::begin(value), std::end(value), Comparator{}); + sorted = true; + } + } +}; + +/// Implementation of groupArray for String or any ComplexObject via Array +template +class SequenceNextNodeImpl final + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> +{ + using Data = SequenceNextNodeGeneralData; + static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } + static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } + + DataTypePtr & data_type; + UInt8 events_size; + UInt64 max_elems; + +public: + SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + {data_type_}, {}) + , data_type(this->argument_types[0]) + , events_size(arguments.size() - 2) + , max_elems(max_elems_) + { + } + + String getName() const override { return "sequenceNextNode"; } + + DataTypePtr getReturnType() const override { return data_type; } + + void insert(Data & a, const Node * v, Arena * arena) const + { + ++a.total_values; + a.value.push_back(v->clone(arena), arena); + } + + void create(AggregateDataPtr place) const override + { + [[maybe_unused]] auto a = new (place) Data; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + Node * node = Node::allocate(*columns[1], row_num, arena); + + const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; + + UInt64 events_bitmap = 0; + for (UInt8 i = 0; i < events_size; ++i) + if (assert_cast *>(columns[2 + i])->getData()[row_num]) + events_bitmap += (1 << i); + + node->event_time = timestamp; + node->events_bitmap = events_bitmap; + + data(place).value.push_back(node, arena); + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + if (data(rhs).value.empty()) /// rhs state is empty + return; + + UInt64 new_elems; + if (data(place).value.size() >= max_elems) + return; + + new_elems = std::min(data(rhs).value.size(), static_cast(max_elems) - data(place).value.size()); + + auto & a = data(place).value; + const auto size = a.size(); + + auto & b = data(rhs).value; + for (UInt64 i = 0; i < new_elems; ++i) + a.push_back(b[i]->clone(arena), arena); + + using Comparator = typename SequenceNextNodeGeneralData::Comparator; + + /// either sort whole container or do so partially merging ranges afterwards + if (!data(place).sorted && !data(rhs).sorted) + std::stable_sort(std::begin(a), std::end(a), Comparator{}); + else + { + const auto begin = std::begin(a); + const auto middle = std::next(begin, size); + const auto end = std::end(a); + + if (!data(place).sorted) + std::stable_sort(begin, middle, Comparator{}); + + if (!data(rhs).sorted) + std::stable_sort(middle, end, Comparator{}); + + std::inplace_merge(begin, middle, end, Comparator{}); + } + + data(place).sorted = true; + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + writeVarUInt(data(place).value.size(), buf); + + auto & value = data(place).value; + for (auto & node : value) + node->write(buf); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + { + UInt64 elems; + readVarUInt(elems, buf); + + if (unlikely(elems == 0)) + return; + + auto & value = data(place).value; + + value.resize(elems, arena); + for (UInt64 i = 0; i < elems; ++i) + value[i] = Node::read(buf, arena); + } + + inline UInt64 getSkipCount(const Data & data, const UInt64 i, const UInt64 j) const + { + UInt64 k = 0; + for (; k < events_size - j; ++k) + if (data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j - k))) + return k; + return k; + } + + UInt64 getNextNodeIndex(Data & data) const + { + if (data.value.size() <= events_size) + return 0; + + data.sort(); + + UInt64 i = events_size - 1; + while (i < data.value.size()) + { + UInt64 j = 0; + for (; j < events_size; ++j) + if (!(data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j)))) + break; + + if (j == events_size) + return i + 1; + + i += getSkipCount(data, i, j); + } + + return 0; + } + + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + { + auto & value = data(place).value; + + UInt64 event_idx = getNextNodeIndex(this->data(place)); + if (event_idx != 0 && event_idx < value.size()) + { + ColumnNullable & to_concrete = assert_cast(to); + value[event_idx]->insertInto(to_concrete.getNestedColumn()); + to_concrete.getNullMapData().push_back(0); + } + else + to.insertDefault(); + } + + bool doesInsertResultNeedNullableColumn() const override { return true; } + + bool allocatesMemoryInArena() const override { return true; } +}; + +} diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index a9fe26688d7..d9570fa5f8b 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -112,6 +112,7 @@ public: /// in `runningAccumulate`, or when calculating an aggregate function as a /// window function. virtual void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const = 0; + virtual bool doesInsertResultNeedNullableColumn() const { return false; } /// Used for machine learning methods. Predict result from trained model. /// Will insert result into `to` column for rows in range [offset, offset + limit). diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index d8e4eb7ba98..28b758aee2c 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -44,6 +44,7 @@ void registerAggregateFunctionRankCorrelation(AggregateFunctionFactory &); void registerAggregateFunctionMannWhitney(AggregateFunctionFactory &); void registerAggregateFunctionWelchTTest(AggregateFunctionFactory &); void registerAggregateFunctionStudentTTest(AggregateFunctionFactory &); +void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory &); class AggregateFunctionCombinatorFactory; void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &); @@ -101,6 +102,7 @@ void registerAggregateFunctions() registerAggregateFunctionMannWhitney(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); + registerAggregateFunctionSequenceNextNode(factory); } { diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp index 35ce005073a..a9be62e515f 100644 --- a/src/Columns/ColumnNullable.cpp +++ b/src/Columns/ColumnNullable.cpp @@ -34,7 +34,6 @@ ColumnNullable::ColumnNullable(MutableColumnPtr && nested_column_, MutableColumn throw Exception{"ColumnNullable cannot have constant null map", ErrorCodes::ILLEGAL_COLUMN}; } - void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const { const auto & arr = getNullMapData(); diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference new file mode 100644 index 00000000000..540e5bdfb5a --- /dev/null +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -0,0 +1,120 @@ +(0, A) 1 B +(0, A) 2 B +(0, A) 3 B +(0, A) 4 A +(0, A) 5 B +(0, A) 6 B +(0, B) 1 C +(0, B) 2 D +(0, B) 3 \N +(0, B) 4 C +(0, B) 5 A +(0, B) 6 A +(0, C) 1 D +(0, C) 2 \N +(0, C) 3 \N +(0, C) 4 \N +(0, C) 5 \N +(0, C) 6 \N +(0, D) 1 \N +(0, D) 2 C +(0, D) 3 \N +(0, D) 4 \N +(0, D) 5 \N +(0, D) 6 \N +(0, E) 1 \N +(0, E) 2 \N +(0, E) 3 \N +(0, E) 4 \N +(0, E) 5 \N +(0, E) 6 \N +(1, A) 1 \N +(1, A) 2 \N +(1, A) 3 \N +(1, A) 4 A +(1, A) 5 B +(1, A) 6 B +(1, B) 1 A +(1, B) 2 A +(1, B) 3 A +(1, B) 4 A +(1, B) 5 A +(1, B) 6 A +(1, C) 1 B +(1, C) 2 D +(1, C) 3 \N +(1, C) 4 B +(1, C) 5 A +(1, C) 6 B +(1, D) 1 C +(1, D) 2 B +(1, D) 3 \N +(1, D) 4 \N +(1, D) 5 \N +(1, D) 6 \N +(1, E) 1 \N +(1, E) 2 \N +(1, E) 3 \N +(1, E) 4 \N +(1, E) 5 \N +(1, E) 6 \N +(0, A->B) 1 C +(0, A->B) 2 D +(0, A->B) 3 \N +(0, A->B) 4 C +(0, A->B) 5 A +(0, A->B) 6 A +(0, A->C) 1 \N +(0, A->C) 2 \N +(0, A->C) 3 \N +(0, A->C) 4 \N +(0, A->C) 5 \N +(0, A->C) 6 \N +(0, B->A) 1 \N +(0, B->A) 2 \N +(0, B->A) 3 \N +(0, B->A) 4 \N +(0, B->A) 5 C +(0, B->A) 6 B +(1, A->B) 1 \N +(1, A->B) 2 \N +(1, A->B) 3 \N +(1, A->B) 4 \N +(1, A->B) 5 A +(1, A->B) 6 A +(1, A->C) 1 \N +(1, A->C) 2 \N +(1, A->C) 3 \N +(1, A->C) 4 \N +(1, A->C) 5 \N +(1, A->C) 6 \N +(1, B->A) 1 \N +(1, B->A) 2 \N +(1, B->A) 3 \N +(1, B->A) 4 A +(1, B->A) 5 \N +(1, B->A) 6 B +(0, A->A->B) 1 \N +(0, A->A->B) 2 \N +(0, A->A->B) 3 \N +(0, A->A->B) 4 C +(0, A->A->B) 5 \N +(0, A->A->B) 6 \N +(0, B->A->A) 1 \N +(0, B->A->A) 2 \N +(0, B->A->A) 3 \N +(0, B->A->A) 4 \N +(0, B->A->A) 5 \N +(0, B->A->A) 6 \N +(1, A->A->B) 1 \N +(1, A->A->B) 2 \N +(1, A->A->B) 3 \N +(1, A->A->B) 4 \N +(1, A->A->B) 5 \N +(1, A->A->B) 6 \N +(1, B->A->A) 1 \N +(1, B->A->A) 2 \N +(1, B->A->A) 3 \N +(1, B->A->A) 4 A +(1, B->A->A) 5 \N +(1, B->A->A) 6 \N diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql new file mode 100644 index 00000000000..b11a5c7bc0e --- /dev/null +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -0,0 +1,51 @@ +DROP TABLE IF EXISTS test_sequenceNextNode; + +CREATE TABLE iF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',1,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',1,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',1,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',1,'D'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',2,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',2,'D'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',2,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',3,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',3,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',4,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',4,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',4,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',5,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',5,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',5,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',5,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',6,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',6,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); + +SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_sequenceNextNode; From e93caefd62c68dc34dce623b192f26483b642a05 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 12:00:37 +0900 Subject: [PATCH 0005/1060] Refactoring --- .../AggregateFunctionNull.h | 42 ++++++++--- .../AggregateFunctionSequenceNextNode.cpp | 4 +- .../AggregateFunctionSequenceNextNode.h | 71 ++++++++++--------- src/AggregateFunctions/IAggregateFunction.h | 1 - 4 files changed, 71 insertions(+), 47 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 3bfcacf7d7b..4dc3c580fd7 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -157,15 +157,8 @@ public: ColumnNullable & to_concrete = assert_cast(to); if (getFlag(place)) { - if (unlikely(nested_function->doesInsertResultNeedNullableColumn())) - { - nested_function->insertResultInto(nestedPlace(place), to_concrete, arena); - } - else - { - nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); - } + nested_function->insertResultInto(nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); } else { @@ -235,7 +228,7 @@ public: }; -template +template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> @@ -283,6 +276,35 @@ public: this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); } + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override + { + if constexpr (result_is_nullable) + { + ColumnNullable & to_concrete = assert_cast(to); + if (this->getFlag(place)) + { + if constexpr (insertion_requires_nullable_column) + { + this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete, arena); + } + else + { + this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); + } + } + else + { + to_concrete.insertDefault(); + } + } + else + { + this->nested_function->insertResultInto(this->nestedPlace(place), to, arena); + } + } + + private: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 66f24ec8cbf..b185859e00e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -49,8 +49,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & if (argument_types.size() < 3) throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > 2 + 64) - throw Exception("Aggregate function " + name + " requires at most 66(timestamp, value_column, 64 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > 2 + 32) + throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 32 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) { diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index a455e16e267..9bdd54e8b4b 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -11,7 +11,6 @@ #include #include -#include #include #include #include @@ -20,6 +19,7 @@ #include #include +#include #include @@ -33,28 +33,24 @@ struct NodeBase UInt64 size; // size of payload DataTypeDateTime::FieldType event_time; - UInt64 events_bitmap; + UInt32 events_bitset; // UInt32 for combiniant comparesons between bitsets (< operator on bitsets). - /// Returns pointer to actual payload char * data() { return reinterpret_cast(this) + sizeof(Node); } const char * data() const { return reinterpret_cast(this) + sizeof(Node); } - /// Clones existing node (does not modify next field) Node * clone(Arena * arena) const { return reinterpret_cast( const_cast(arena->alignedInsert(reinterpret_cast(this), sizeof(Node) + size, alignof(Node)))); } - /// Write node to buffer void write(WriteBuffer & buf) const { writeVarUInt(size, buf); buf.write(data(), size); } - /// Reads and allocates node from ReadBuffer's data (doesn't set next) static Node * read(ReadBuffer & buf, Arena * arena) { UInt64 size; @@ -71,7 +67,6 @@ struct NodeString : public NodeBase { using Node = NodeString; - /// Create node from string static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) { StringRef string = assert_cast(column).getDataAt(row_num); @@ -92,7 +87,6 @@ struct NodeString : public NodeBase template struct SequenceNextNodeGeneralData { - // Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena using Allocator = MixedAlignedArenaAllocator; using Array = PODArray; @@ -103,12 +97,12 @@ struct SequenceNextNodeGeneralData { bool operator()(const Node * lhs, const Node * rhs) const { - if (Descending) + if constexpr (Descending) return lhs->event_time == rhs->event_time ? - lhs->events_bitmap < rhs->events_bitmap: lhs->event_time > rhs->event_time; + lhs->events_bitset < rhs->events_bitset: lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? - lhs->events_bitmap < rhs->events_bitmap : lhs->event_time < rhs->event_time; + lhs->events_bitset < rhs->events_bitset : lhs->event_time < rhs->event_time; } }; @@ -122,7 +116,6 @@ struct SequenceNextNodeGeneralData } }; -/// Implementation of groupArray for String or any ComplexObject via Array template class SequenceNextNodeImpl final : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> @@ -149,6 +142,18 @@ public: DataTypePtr getReturnType() const override { return data_type; } + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties & /*properties*/) const override + { + // This aggregate function sets insertion_requires_nullable_column on. + // Even though some values are mapped to aggregating key, it could return nulls for the below case. + // aggregated events: [A -> B -> C] + // events to find: [C -> D] + // [C -> D] is not matched to 'A -> B -> C' so that it returns null. + return std::make_shared>(nested_function, arguments, params); + } + void insert(Data & a, const Node * v, Arena * arena) const { ++a.total_values; @@ -166,44 +171,42 @@ public: const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; - UInt64 events_bitmap = 0; + UInt32 events_bitset = 0; for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) - events_bitmap += (1 << i); + events_bitset += (1 << i); node->event_time = timestamp; - node->events_bitmap = events_bitmap; + node->events_bitset = events_bitset; data(place).value.push_back(node, arena); } void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override { - if (data(rhs).value.empty()) /// rhs state is empty + if (data(rhs).value.empty()) return; - UInt64 new_elems; if (data(place).value.size() >= max_elems) return; - new_elems = std::min(data(rhs).value.size(), static_cast(max_elems) - data(place).value.size()); - auto & a = data(place).value; - const auto size = a.size(); - auto & b = data(rhs).value; + const auto a_size = a.size(); + + const UInt64 new_elems = std::min(data(rhs).value.size(), static_cast(max_elems) - data(place).value.size()); for (UInt64 i = 0; i < new_elems; ++i) a.push_back(b[i]->clone(arena), arena); + /// either sort whole container or do so partially merging ranges afterwards using Comparator = typename SequenceNextNodeGeneralData::Comparator; - /// either sort whole container or do so partially merging ranges afterwards if (!data(place).sorted && !data(rhs).sorted) std::stable_sort(std::begin(a), std::end(a), Comparator{}); else { const auto begin = std::begin(a); - const auto middle = std::next(begin, size); + const auto middle = std::next(begin, a_size); const auto end = std::end(a); if (!data(place).sorted) @@ -242,34 +245,36 @@ public: value[i] = Node::read(buf, arena); } - inline UInt64 getSkipCount(const Data & data, const UInt64 i, const UInt64 j) const + inline UInt32 calculateJump(const Data & data, const UInt32 i, const UInt32 j) const { - UInt64 k = 0; + UInt32 k = 0; for (; k < events_size - j; ++k) - if (data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j - k))) + if (data.value[i - j]->events_bitset & (1 << (events_size - 1 - j - k))) return k; return k; } - UInt64 getNextNodeIndex(Data & data) const + // This method returns an index of next node that matched the events. + // It is one as referring Boyer-Moore-Algorithm. + UInt32 getNextNodeIndex(Data & data) const { if (data.value.size() <= events_size) return 0; data.sort(); - UInt64 i = events_size - 1; + UInt32 i = events_size - 1; while (i < data.value.size()) { - UInt64 j = 0; + UInt32 j = 0; for (; j < events_size; ++j) - if (!(data.value[i - j]->events_bitmap & (1 << (events_size - 1 - j)))) + if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) break; if (j == events_size) return i + 1; - i += getSkipCount(data, i, j); + i += calculateJump(data, i, j); } return 0; @@ -279,7 +284,7 @@ public: { auto & value = data(place).value; - UInt64 event_idx = getNextNodeIndex(this->data(place)); + UInt32 event_idx = getNextNodeIndex(this->data(place)); if (event_idx != 0 && event_idx < value.size()) { ColumnNullable & to_concrete = assert_cast(to); @@ -290,8 +295,6 @@ public: to.insertDefault(); } - bool doesInsertResultNeedNullableColumn() const override { return true; } - bool allocatesMemoryInArena() const override { return true; } }; diff --git a/src/AggregateFunctions/IAggregateFunction.h b/src/AggregateFunctions/IAggregateFunction.h index d9570fa5f8b..a9fe26688d7 100644 --- a/src/AggregateFunctions/IAggregateFunction.h +++ b/src/AggregateFunctions/IAggregateFunction.h @@ -112,7 +112,6 @@ public: /// in `runningAccumulate`, or when calculating an aggregate function as a /// window function. virtual void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const = 0; - virtual bool doesInsertResultNeedNullableColumn() const { return false; } /// Used for machine learning methods. Predict result from trained model. /// Will insert result into `to` column for rows in range [offset, offset + limit). From 994e01e0abae2556cb76615c529e2450ac89abb4 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 16:11:32 +0900 Subject: [PATCH 0006/1060] Serialization --- .../AggregateFunctionSequenceNextNode.h | 15 +++++++++------ src/AggregateFunctions/ya.make | 1 + 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 9bdd54e8b4b..0a6f686e9bc 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -223,25 +223,28 @@ public: void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override { - writeVarUInt(data(place).value.size(), buf); + writeBinary(data(place).sorted, buf); auto & value = data(place).value; + writeVarUInt(value.size(), buf); for (auto & node : value) node->write(buf); } void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override { - UInt64 elems; - readVarUInt(elems, buf); + readBinary(data(place).sorted, buf); - if (unlikely(elems == 0)) + UInt64 size; + readVarUInt(size, buf); + + if (unlikely(size == 0)) return; auto & value = data(place).value; - value.resize(elems, arena); - for (UInt64 i = 0; i < elems; ++i) + value.resize(size, arena); + for (UInt64 i = 0; i < size; ++i) value[i] = Node::read(buf, arena); } diff --git a/src/AggregateFunctions/ya.make b/src/AggregateFunctions/ya.make index f2105688feb..5af62a68396 100644 --- a/src/AggregateFunctions/ya.make +++ b/src/AggregateFunctions/ya.make @@ -42,6 +42,7 @@ SRCS( AggregateFunctionResample.cpp AggregateFunctionRetention.cpp AggregateFunctionSequenceMatch.cpp + AggregateFunctionSequenceNextNode.cpp AggregateFunctionSimpleLinearRegression.cpp AggregateFunctionSimpleState.cpp AggregateFunctionState.cpp From 7ca8811216bae8ed3b5ccff11d84d3fc7f2bbb33 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 17:14:56 +0900 Subject: [PATCH 0007/1060] Fix --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 0a6f686e9bc..44a2018980a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -49,6 +49,9 @@ struct NodeBase { writeVarUInt(size, buf); buf.write(data(), size); + + writeBinary(event_time, buf); + writeBinary(events_bitset, buf); } static Node * read(ReadBuffer & buf, Arena * arena) @@ -59,6 +62,10 @@ struct NodeBase Node * node = reinterpret_cast(arena->alignedAlloc(sizeof(Node) + size, alignof(Node))); node->size = size; buf.read(node->data(), size); + + readBinary(node->event_time, buf); + readBinary(node->events_bitset, buf); + return node; } }; From 05c3767d05c42440610e9a387b95b3be1eae5d81 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 28 Jan 2021 18:55:33 +0900 Subject: [PATCH 0008/1060] Fix --- .../AggregateFunctionSequenceNextNode.cpp | 4 +- .../AggregateFunctionSequenceNextNode.h | 2 +- .../01656_sequence_next_node.reference | 132 ++++++++++++++++++ .../0_stateless/01656_sequence_next_node.sql | 119 +++++++++++++--- 4 files changed, 233 insertions(+), 24 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index b185859e00e..47279aa882d 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -94,9 +94,7 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { - AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 44a2018980a..3fbb481c19a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -158,7 +158,7 @@ public: // aggregated events: [A -> B -> C] // events to find: [C -> D] // [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 540e5bdfb5a..ecb39686bfa 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -118,3 +118,135 @@ (1, B->A->A) 4 A (1, B->A->A) 5 \N (1, B->A->A) 6 \N +(0, A) id >= 10 10 B +(0, A) id >= 10 10 C +(0, A) id >= 10 10 D +(0, A) id >= 10 10 B +(0, A) id >= 10 10 B +(0, A) id >= 10 10 A +(0, A) 1 B +(0, A) 2 B +(0, A) 3 B +(0, A) 4 A +(0, A) 5 B +(0, A) 6 B +(0, B) 1 C +(0, B) 2 D +(0, B) 3 \N +(0, B) 4 C +(0, B) 5 A +(0, B) 6 A +(0, C) 1 D +(0, C) 2 \N +(0, C) 3 \N +(0, C) 4 \N +(0, C) 5 \N +(0, C) 6 \N +(0, D) 1 \N +(0, D) 2 C +(0, D) 3 \N +(0, D) 4 \N +(0, D) 5 \N +(0, D) 6 \N +(0, E) 1 \N +(0, E) 2 \N +(0, E) 3 \N +(0, E) 4 \N +(0, E) 5 \N +(0, E) 6 \N +(1, A) 1 \N +(1, A) 2 \N +(1, A) 3 \N +(1, A) 4 A +(1, A) 5 B +(1, A) 6 B +(1, B) 1 A +(1, B) 2 A +(1, B) 3 A +(1, B) 4 A +(1, B) 5 A +(1, B) 6 A +(1, C) 1 B +(1, C) 2 D +(1, C) 3 \N +(1, C) 4 B +(1, C) 5 A +(1, C) 6 B +(1, D) 1 C +(1, D) 2 B +(1, D) 3 \N +(1, D) 4 \N +(1, D) 5 \N +(1, D) 6 \N +(1, E) 1 \N +(1, E) 2 \N +(1, E) 3 \N +(1, E) 4 \N +(1, E) 5 \N +(1, E) 6 \N +(0, A->B) 1 C +(0, A->B) 2 D +(0, A->B) 3 \N +(0, A->B) 4 C +(0, A->B) 5 A +(0, A->B) 6 A +(0, A->C) 1 \N +(0, A->C) 2 \N +(0, A->C) 3 \N +(0, A->C) 4 \N +(0, A->C) 5 \N +(0, A->C) 6 \N +(0, B->A) 1 \N +(0, B->A) 2 \N +(0, B->A) 3 \N +(0, B->A) 4 \N +(0, B->A) 5 C +(0, B->A) 6 B +(1, A->B) 1 \N +(1, A->B) 2 \N +(1, A->B) 3 \N +(1, A->B) 4 \N +(1, A->B) 5 A +(1, A->B) 6 A +(1, A->C) 1 \N +(1, A->C) 2 \N +(1, A->C) 3 \N +(1, A->C) 4 \N +(1, A->C) 5 \N +(1, A->C) 6 \N +(1, B->A) 1 \N +(1, B->A) 2 \N +(1, B->A) 3 \N +(1, B->A) 4 A +(1, B->A) 5 \N +(1, B->A) 6 B +(0, A->A->B) 1 \N +(0, A->A->B) 2 \N +(0, A->A->B) 3 \N +(0, A->A->B) 4 C +(0, A->A->B) 5 \N +(0, A->A->B) 6 \N +(0, B->A->A) 1 \N +(0, B->A->A) 2 \N +(0, B->A->A) 3 \N +(0, B->A->A) 4 \N +(0, B->A->A) 5 \N +(0, B->A->A) 6 \N +(1, A->A->B) 1 \N +(1, A->A->B) 2 \N +(1, A->A->B) 3 \N +(1, A->A->B) 4 \N +(1, A->A->B) 5 \N +(1, A->A->B) 6 \N +(1, B->A->A) 1 \N +(1, B->A->A) 2 \N +(1, B->A->A) 3 \N +(1, B->A->A) 4 A +(1, B->A->A) 5 \N +(1, B->A->A) 6 \N +(0, A) id >= 10 10 B +(0, A) id >= 10 10 C +(0, A) id >= 10 10 D +(0, A) id >= 10 10 B +(0, A) id >= 10 10 B +(0, A) id >= 10 10 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index b11a5c7bc0e..e5a89da7341 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -1,3 +1,70 @@ +DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; + +CREATE TABLE iF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',1,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',1,'D'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',2,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',2,'D'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',2,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',3,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',3,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',4,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',4,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',4,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',5,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',5,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',5,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',5,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',6,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',6,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); + +SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); + +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; + +-- The same testcases for a non-null type. + DROP TABLE IF EXISTS test_sequenceNextNode; CREATE TABLE iF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; @@ -27,25 +94,37 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test GROUP BY id ORDER BY id; -SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test GROUP BY id ORDER BY id; +SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); + +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode; From e092eeee7a3085026e699e7d27b6779b71a8fd0b Mon Sep 17 00:00:00 2001 From: achimbab Date: Fri, 29 Jan 2021 08:51:57 +0900 Subject: [PATCH 0009/1060] Documentation for sequenceNextNode --- .../parametric-functions.md | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 3b02e145ff4..b371cbcb9fa 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -495,3 +495,56 @@ Solution: Write in the GROUP BY query SearchPhrase HAVING uniqUpTo(4)(UserID) >= ## sumMapFiltered(keys_to_keep)(keys, values) {#summapfilteredkeys-to-keepkeys-values} Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summap.md#agg_functions-summap) except that an array of keys is passed as a parameter. This can be especially useful when working with a high cardinality of keys. + +## sequenceNextNode {#sequenceNextNode} + +Returns a value of next event that matched an event chain. + +**Syntax** + +``` sql +sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, ... event32) +``` + +**Parameters** +- `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. +- `timestamp` — Name of the column containing the timestamp. Data types supported: Date, DateTime and other unsigned integer types. +- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: String and Nullable(String) +- `cond` — Conditions or data describing the chain of events. UInt8. + +**Returned value** + +Nullable(String). + +- event_column[next_index] - if the pattern is matched and next value exists. +- null - if the pattern isn’t matched or next value doesn't exist. + +Type: `Nullable(String)`. + +**Example** + +It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E. + +The query statement searching the event following B->C : + +``` sql +CREATE TABLE test_flow ( + dt DateTime, + id int, + action String) +ENGINE = MergeTree() +PARTITION BY toYYYYMMDD(dt) +ORDER BY id; + +INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); + +SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') as next_flow FROM test_flow GROUP BY id; +``` + +Result: + +``` text +┌─id─┬─next_flow─┐ +│ 1 │ E │ +└────┴───────────┘ +``` From 138c5ab64476fc7c071214515a7973bf61f2add8 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 29 Jan 2021 10:33:42 +0900 Subject: [PATCH 0010/1060] For some functional stateless tests (ubsan) --- src/AggregateFunctions/AggregateFunctionNull.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 4dc3c580fd7..adf2aed1e82 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -231,12 +231,12 @@ public: template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> + AggregateFunctionNullVariadic> { public: AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), + AggregateFunctionNullVariadic>(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) From 753c32ee13706d34c10cb11e455d0d864dd1ef96 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 1 Feb 2021 14:01:18 +0900 Subject: [PATCH 0011/1060] Use 0x80000000 as unmatched event --- .../AggregateFunctionSequenceNextNode.cpp | 4 +-- .../AggregateFunctionSequenceNextNode.h | 3 +- .../01656_sequence_next_node.reference | 10 +++--- .../0_stateless/01656_sequence_next_node.sql | 35 +++++++++++++++---- 4 files changed, 38 insertions(+), 14 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 47279aa882d..adfa1592c86 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -49,8 +49,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & if (argument_types.size() < 3) throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > 2 + 32) - throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 32 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > 2 + 31) + throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 31 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) { diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 3fbb481c19a..85413513d50 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -106,7 +106,7 @@ struct SequenceNextNodeGeneralData { if constexpr (Descending) return lhs->event_time == rhs->event_time ? - lhs->events_bitset < rhs->events_bitset: lhs->event_time > rhs->event_time; + lhs->events_bitset < rhs->events_bitset : lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? lhs->events_bitset < rhs->events_bitset : lhs->event_time < rhs->event_time; @@ -182,6 +182,7 @@ public: for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) events_bitset += (1 << i); + if (events_bitset == 0) events_bitset = 0x80000000; // Any events are not matched. node->event_time = timestamp; node->events_bitset = events_bitset; diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index ecb39686bfa..b0689ff136c 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -245,8 +245,8 @@ (1, B->A->A) 5 \N (1, B->A->A) 6 \N (0, A) id >= 10 10 B -(0, A) id >= 10 10 C -(0, A) id >= 10 10 D -(0, A) id >= 10 10 B -(0, A) id >= 10 10 B -(0, A) id >= 10 10 A +(0, A->B) id >= 10 10 C +(0, B->C) id >= 10 10 D +(0, C->) id >= 10 10 B +(0, D->C) id >= 10 10 B +(0, C->B) id >= 10 10 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index e5a89da7341..55d98e93342 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -61,6 +61,15 @@ SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS n SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); + DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; -- The same testcases for a non-null type. @@ -120,11 +129,25 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(0, A) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, A->B) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, B->C) id = 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, D->C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(0, C->B) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); + +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); DROP TABLE IF EXISTS test_sequenceNextNode; From 03ad81ee726cb1bb2bc90c93499c1e78812b0127 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 1 Feb 2021 15:26:16 +0900 Subject: [PATCH 0012/1060] Correct testcases --- .../01656_sequence_next_node.reference | 28 +++++++++++++++---- .../0_stateless/01656_sequence_next_node.sql | 16 +++++------ 2 files changed, 30 insertions(+), 14 deletions(-) diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index b0689ff136c..3e8da4bbd48 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -124,6 +124,14 @@ (0, A) id >= 10 10 B (0, A) id >= 10 10 B (0, A) id >= 10 10 A +(0, A) id = 11 0 +(0, C) id = 11 0 +(0, B->C) id = 11 0 +(0, A->B->C) id = 11 0 +(0, A) id = 11 0 +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 0 (0, A) 1 B (0, A) 2 B (0, A) 3 B @@ -244,9 +252,17 @@ (1, B->A->A) 4 A (1, B->A->A) 5 \N (1, B->A->A) 6 \N -(0, A) id >= 10 10 B -(0, A->B) id >= 10 10 C -(0, B->C) id >= 10 10 D -(0, C->) id >= 10 10 B -(0, D->C) id >= 10 10 B -(0, C->B) id >= 10 10 A +(0, A) id = 10 10 B +(0, A->B) id = 10 10 C +(0, B->C) id = 10 10 D +(0, C) id = 10 10 B +(0, D->C) id = 10 10 B +(0, C->B) id = 10 10 A +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, B->C) id = 11 1 +(0, A->B->C) id = 11 1 +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, C->B) id = 11 1 +(0, C->B->A) id = 11 1 diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 55d98e93342..31c224fd2a4 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -61,14 +61,14 @@ SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS n SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; From 19dd09ea8ea0471a25de5b86003b70dc9c2c84bf Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Tue, 2 Feb 2021 17:50:08 +0900 Subject: [PATCH 0013/1060] Fix paramters for nulls --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp | 3 ++- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index adfa1592c86..57ba87c922f 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -94,7 +94,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode }); + AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 85413513d50..888149c77da 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -158,7 +158,7 @@ public: // aggregated events: [A -> B -> C] // events to find: [C -> D] // [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const From 2cc69893f26920aa41c5da497035b46e6ce67360 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 4 Feb 2021 16:15:04 +0900 Subject: [PATCH 0014/1060] Add sequenceFirstNode --- .../AggregateFunctionSequenceNextNode.cpp | 43 ++++-- .../AggregateFunctionSequenceNextNode.h | 135 ++++++++++++++++++ .../01656_sequence_next_node.reference | 48 +++++++ .../0_stateless/01656_sequence_next_node.sql | 10 ++ 4 files changed, 221 insertions(+), 15 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 57ba87c922f..af90c80de61 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -23,15 +23,27 @@ namespace { template -inline AggregateFunctionPtr createAggregateFunctionSequenceNextNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) { - if (descending_order) - return std::make_shared>(data_type, argument_types); + if (argument_types.size() == 2) + { + // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. + if (descending_order) + return std::make_shared>(data_type); + else + return std::make_shared>(data_type); + } else - return std::make_shared>(data_type, argument_types); + { + if (descending_order) + return std::make_shared>(data_type, argument_types); + else + return std::make_shared>(data_type, argument_types); + } } -AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) +template +AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) { bool descending_order = false; @@ -47,9 +59,9 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (argument_types.size() < 3) - throw Exception("Aggregate function " + name + " requires at least three arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > 2 + 31) + if (argument_types.size() < 2) + throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > MaxArgs) throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 31 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) @@ -73,17 +85,17 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.idx == TypeIndex::UInt16) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.idx == TypeIndex::UInt32) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.idx == TypeIndex::UInt64) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.isDate()) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); if (timestamp_type.isDateTime()) - return createAggregateFunctionSequenceNextNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); throw Exception{"Illegal type " + argument_types.front().get()->getName() + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", @@ -95,7 +107,8 @@ AggregateFunctionPtr createAggregateFunctionSequenceNextNode(const std::string & void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNextNode, properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode<2 + 31>, properties }); + factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNode<2>, properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 888149c77da..ffcc02b805a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -309,4 +309,139 @@ public: bool allocatesMemoryInArena() const override { return true; } }; +template +class SequenceFirstNodeImpl final + : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> +{ + using Data = SequenceNextNodeGeneralData; + static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } + static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } + + DataTypePtr & data_type; + +public: + SequenceFirstNodeImpl(const DataTypePtr & data_type_) + : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( + {data_type_}, {}) + , data_type(this->argument_types[0]) + { + } + + String getName() const override { return "sequenceFirstNode"; } + + DataTypePtr getReturnType() const override { return data_type; } + + AggregateFunctionPtr getOwnNullAdapter( + const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, + const AggregateFunctionProperties &) const override + { + return std::make_shared>(nested_function, arguments, params); + } + + void insert(Data & a, const Node * v, Arena * arena) const + { + ++a.total_values; + a.value.push_back(v->clone(arena), arena); + } + + void create(AggregateDataPtr place) const override + { + [[maybe_unused]] auto a = new (place) Data; + } + + bool compare(const T lhs_timestamp, const T rhs_timestamp) const + { + return Descending ? lhs_timestamp < rhs_timestamp : lhs_timestamp > rhs_timestamp; + } + + void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + { + bool is_first = true; + auto & value = data(place).value; + const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; + + if (value.size() != 0) + { + if (compare(value[0]->event_time, timestamp)) + value.pop_back(); + else + is_first = false; + } + + + if (is_first) + { + Node * node = Node::allocate(*columns[1], row_num, arena); + node->event_time = timestamp; + node->events_bitset = 0x80000000; + + data(place).value.push_back(node, arena); + } + } + + void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + { + auto & a = data(place).value; + auto & b = data(rhs).value; + + if (b.empty()) + return; + + if (a.empty()) + { + a.push_back(b[0]->clone(arena), arena); + return; + } + + if (compare(a[0]->event_time, b[0]->event_time)) + { + data(place).value.pop_back(); + a.push_back(b[0]->clone(arena), arena); + } + } + + void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + { + writeBinary(data(place).sorted, buf); + + auto & value = data(place).value; + writeVarUInt(value.size(), buf); + for (auto & node : value) + node->write(buf); + } + + void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + { + readBinary(data(place).sorted, buf); + + UInt64 size; + readVarUInt(size, buf); + + if (unlikely(size == 0)) + return; + + auto & value = data(place).value; + + value.resize(size, arena); + for (UInt64 i = 0; i < size; ++i) + value[i] = Node::read(buf, arena); + } + + void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + { + auto & value = data(place).value; + + if (value.size() > 0) + { + ColumnNullable & to_concrete = assert_cast(to); + value[0]->insertInto(to_concrete.getNestedColumn()); + to_concrete.getNullMapData().push_back(0); + } + else + to.insertDefault(); + } + + bool allocatesMemoryInArena() const override { return true; } +}; + } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 3e8da4bbd48..50755232cb9 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -132,6 +132,30 @@ (0, C) id = 11 0 (0, C->B) id = 11 0 (0, C->B->A) id = 11 0 +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C (0, A) 1 B (0, A) 2 B (0, A) 3 B @@ -266,3 +290,27 @@ (0, C) id = 11 1 (0, C->B) id = 11 1 (0, C->B->A) id = 11 1 +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(0) id < 10 1 A +(0) id < 10 2 A +(0) id < 10 3 A +(0) id < 10 4 A +(0) id < 10 5 A +(0) id < 10 6 A +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C +(1) id < 10 1 D +(1) id < 10 2 C +(1) id < 10 3 B +(1) id < 10 4 C +(1) id < 10 5 C +(1) id < 10 6 C diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 31c224fd2a4..9af59d5c8e2 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -70,6 +70,11 @@ SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; + DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; -- The same testcases for a non-null type. @@ -150,4 +155,9 @@ SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; + DROP TABLE IF EXISTS test_sequenceNextNode; From ac149592c687646992df1cb0ddc19f0ab169e55a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 11 Feb 2021 19:05:04 +0300 Subject: [PATCH 0015/1060] wip --- .../common.py | 87 ++++++++++--------- .../test_cassandra.py | 16 ++-- .../test_clickhouse_remote.py | 28 +++--- 3 files changed, 62 insertions(+), 69 deletions(-) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py index ef6d133893a..a3d0e8a019b 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/common.py @@ -1,4 +1,5 @@ import os +import shutil from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout @@ -89,34 +90,54 @@ VALUES = { SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs/dictionaries') +DICT_CONFIG_PATH = os.path.join(SCRIPT_DIR, 'configs', 'dictionaries') -def get_dict(source, layout, fields, suffix_name=''): - global DICT_CONFIG_PATH - structure = DictionaryStructure(layout, fields) - dict_name = source.name + "_" + layout.name + '_' + suffix_name - dict_path = os.path.join(DICT_CONFIG_PATH, dict_name + '.xml') - dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) - dictionary.generate_config() - return dictionary +class BaseLayoutTester: + def __init__(self, test_name): + self.test_name = test_name + self.layouts = [] -class SimpleLayoutTester: - def __init__(self): - self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] - self.values = VALUES["simple"] - self.data = [Row(self.fields, vals) for vals in self.values] - self.layout_to_dictionary = dict() + def get_dict_directory(self): + return os.path.join(DICT_CONFIG_PATH, self.test_name) + + def cleanup(self): + shutil.rmtree(self.get_dict_directory(), ignore_errors=True) + os.makedirs(self.get_dict_directory()) + + def list_dictionaries(self): + dictionaries = [] + directory = self.get_dict_directory() + for fname in os.listdir(directory): + dictionaries.append(os.path.join(directory, fname)) + return dictionaries def create_dictionaries(self, source_): - for layout in LAYOUTS_SIMPLE: + for layout in self.layouts: if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) + self.layout_to_dictionary[layout] = self.get_dict(source_, Layout(layout), self.fields) def prepare(self, cluster_): for _, dictionary in list(self.layout_to_dictionary.items()): dictionary.prepare_source(cluster_) dictionary.load_data(self.data) + def get_dict(self, source, layout, fields, suffix_name=''): + structure = DictionaryStructure(layout, fields) + dict_name = source.name + "_" + layout.name + '_' + suffix_name + dict_path = os.path.join(self.get_dict_directory(), dict_name + '.xml') + dictionary = Dictionary(dict_name, structure, source, dict_path, "table_" + dict_name, fields) + dictionary.generate_config() + return dictionary + +class SimpleLayoutTester(BaseLayoutTester): + def __init__(self, test_name): + self.fields = KEY_FIELDS["simple"] + START_FIELDS["simple"] + MIDDLE_FIELDS + END_FIELDS["simple"] + self.values = VALUES["simple"] + self.data = [Row(self.fields, vals) for vals in self.values] + self.layout_to_dictionary = dict() + self.test_name = test_name + self.layouts = LAYOUTS_SIMPLE + def execute(self, layout_name, node): if layout_name not in self.layout_to_dictionary: raise RuntimeError("Source doesn't support layout: {}".format(layout_name)) @@ -157,22 +178,14 @@ class SimpleLayoutTester: assert node.query(query) == str(answer) + '\n' -class ComplexLayoutTester: - def __init__(self): +class ComplexLayoutTester(BaseLayoutTester): + def __init__(self, test_name): self.fields = KEY_FIELDS["complex"] + START_FIELDS["complex"] + MIDDLE_FIELDS + END_FIELDS["complex"] self.values = VALUES["complex"] self.data = [Row(self.fields, vals) for vals in self.values] self.layout_to_dictionary = dict() - - def create_dictionaries(self, source_): - for layout in LAYOUTS_COMPLEX: - if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) - - def prepare(self, cluster_): - for _, dictionary in list(self.layout_to_dictionary.items()): - dictionary.prepare_source(cluster_) - dictionary.load_data(self.data) + self.test_name = test_name + self.layouts = LAYOUTS_COMPLEX def execute(self, layout_name, node): if layout_name not in self.layout_to_dictionary: @@ -200,22 +213,14 @@ class ComplexLayoutTester: assert node.query(query) == str(answer) + '\n' -class RangedLayoutTester: - def __init__(self): +class RangedLayoutTester(BaseLayoutTester): + def __init__(self, test_name): self.fields = KEY_FIELDS["ranged"] + START_FIELDS["ranged"] + MIDDLE_FIELDS + END_FIELDS["ranged"] self.values = VALUES["ranged"] self.data = [Row(self.fields, vals) for vals in self.values] self.layout_to_dictionary = dict() - - def create_dictionaries(self, source_): - for layout in LAYOUTS_RANGED: - if source_.compatible_with_layout(Layout(layout)): - self.layout_to_dictionary[layout] = get_dict(source_, Layout(layout), self.fields) - - def prepare(self, cluster_): - for _, dictionary in list(self.layout_to_dictionary.items()): - dictionary.prepare_source(cluster_) - dictionary.load_data(self.data) + self.test_name = test_name + self.layouts = LAYOUTS_RANGED def execute(self, layout_name, node): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 81f9db1964b..8154cc55453 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -15,7 +15,7 @@ node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "cassandra" def setup_module(module): global cluster @@ -24,10 +24,9 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) simple_tester = SimpleLayoutTester() + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) complex_tester = ComplexLayoutTester() @@ -37,24 +36,19 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py index 374e620e1c3..3ed335a1987 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_remote.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceClickHouse -SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse1", "9000", "default", "") +SOURCE = SourceClickHouse("RemoteClickHouse", "localhost", "9000", "clickhouse_remote", "9000", "default", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "remote" def setup_module(module): global cluster @@ -24,37 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_remote', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('remote_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + simple_tester.cleanup() @pytest.fixture(scope="module") From 581e080b3532807dfd74186feaa5d462eebfec7f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 12 Feb 2021 18:37:29 +0300 Subject: [PATCH 0016/1060] wip --- .../test_cassandra.py | 8 ++--- .../test_clickhouse_local.py | 30 ++++++++----------- .../test_executable_cache.py | 29 +++++++----------- .../test_executable_hashed.py | 28 +++++++---------- .../test_file.py | 27 +++++++---------- .../test_http.py | 29 +++++++----------- .../test_https.py | 29 +++++++----------- .../test_mongo.py | 27 +++++++---------- .../test_mongo_uri.py | 29 ++++++++---------- .../test_mysql.py | 26 +++++++--------- 10 files changed, 102 insertions(+), 160 deletions(-) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 8154cc55453..8be25207d51 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -25,14 +25,14 @@ def setup_module(module): global ranged_tester - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created @@ -44,7 +44,7 @@ def setup_module(module): dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) + node = cluster.add_instance('cass_node', main_configs=main_configs, dictionaries=dictionaries, with_cassandra=True) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py index 3d56746be6e..051b4ff3086 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_clickhouse_local.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceClickHouse -SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "node", "9000", "default", "") +SOURCE = SourceClickHouse("LocalClickHouse", "localhost", "9000", "local_node", "9000", "default", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "local" def setup_module(module): global cluster @@ -24,35 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('local_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) + simple_tester.cleanup() @pytest.fixture(scope="module") diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py index 1d741d5271c..5d694bc09a2 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_cache.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceExecutableCache -SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceExecutableCache("ExecutableCache", "localhost", "9000", "cache_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "cache" def setup_module(module): global cluster @@ -24,36 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('cache_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py index dfcc35c54f8..8c0e6f8b878 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_executable_hashed.py @@ -8,13 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceExecutableHashed -SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceExecutableHashed("ExecutableHashed", "localhost", "9000", "hashed_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "hashed" def setup_module(module): @@ -24,36 +25,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('hashed_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py index aa81cca466b..97a06fadc5e 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_file.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceFile -SOURCE = SourceFile("File", "localhost", "9000", "node", "9000", "", "") +SOURCE = SourceFile("File", "localhost", "9000", "file_node", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "file" def setup_module(module): global cluster @@ -24,36 +24,29 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('file_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py index 7c8b5a41b01..c8c73011f61 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_http.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceHTTP -SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse1", "9000", "", "") +SOURCE = SourceHTTP("SourceHTTP", "localhost", "9000", "clickhouse_h", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "http" def setup_module(module): global cluster @@ -24,38 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_h', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('http_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py index 44950f013b3..42f33e3da3c 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_https.py @@ -8,14 +8,14 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceHTTPS -SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse1", "9000", "", "") +SOURCE = SourceHTTPS("SourceHTTPS", "localhost", "9000", "clickhouse_hs", "9000", "", "") cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "https" def setup_module(module): global cluster @@ -24,38 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - cluster.add_instance('clickhouse1', main_configs=main_configs) + cluster.add_instance('clickhouse_hs', main_configs=main_configs) - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries) + node = cluster.add_instance('https_node', main_configs=main_configs, dictionaries=dictionaries) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py index 7d808845854..deaaf044bce 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo.py @@ -8,14 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMongo -SOURCE = SourceMongo("MongoDB", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None - +test_name = "mongo" def setup_module(module): global cluster @@ -24,36 +23,30 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) + SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 5c09627d0b9..9561ee5acc0 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -8,13 +8,13 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMongoURI -SOURCE = SourceMongoURI("MongoDB_URI", "localhost", "27018", "mongo1", "27017", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "mongo_uri" def setup_module(module): @@ -24,36 +24,31 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) + cluster = ClickHouseCluster(__file__, name=test_name) - simple_tester = SimpleLayoutTester() + SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") + + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) - - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) + dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) + node = cluster.add_instance('uri_node', main_configs=main_configs, dictionaries=dictionaries, with_mongo=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 7a6b0b7ce8d..9bf3e888f31 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -15,6 +15,7 @@ node = None simple_tester = None complex_tester = None ranged_tester = None +test_name = "mysql" def setup_module(module): @@ -24,37 +25,30 @@ def setup_module(module): global complex_tester global ranged_tester - for f in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, f)) - - simple_tester = SimpleLayoutTester() + simple_tester = SimpleLayoutTester(test_name) + simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) - complex_tester = ComplexLayoutTester() + complex_tester = ComplexLayoutTester(test_name) complex_tester.create_dictionaries(SOURCE) - ranged_tester = RangedLayoutTester() + ranged_tester = RangedLayoutTester(test_name) ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__) + cluster = ClickHouseCluster(__file__, name=test_name) - dictionaries = [] main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) - for fname in os.listdir(DICT_CONFIG_PATH): - dictionaries.append(os.path.join(DICT_CONFIG_PATH, fname)) - - node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) + dictionaries = simple_tester.list_dictionaries() + + node = cluster.add_instance('mysql_node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) def teardown_module(module): - global DICT_CONFIG_PATH - for fname in os.listdir(DICT_CONFIG_PATH): - os.remove(os.path.join(DICT_CONFIG_PATH, fname)) - + simple_tester.cleanup() @pytest.fixture(scope="module") def started_cluster(): From 8aca6e9cd00111b0dc74a282e5c374c5e3dfa5f2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 12 Feb 2021 18:50:52 +0300 Subject: [PATCH 0017/1060] update runner --- docker/test/integration/runner/Dockerfile | 8 ++++---- .../integration/runner/compose/docker_compose_mongo.yml | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 502dc3736b2..9e850fe5281 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -1,5 +1,5 @@ # docker build -t yandex/clickhouse-integration-tests-runner . -FROM ubuntu:18.04 +FROM ubuntu:20.04 RUN apt-get update \ && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \ @@ -14,7 +14,6 @@ RUN apt-get update \ wget \ git \ iproute2 \ - module-init-tools \ cgroupfs-mount \ python3-pip \ tzdata \ @@ -65,17 +64,18 @@ RUN python3 -m pip install \ dict2xml \ dicttoxml \ docker \ - docker-compose==1.22.0 \ + docker-compose==1.28.2 \ grpcio \ grpcio-tools \ kafka-python \ kazoo \ minio \ protobuf \ - psycopg2-binary==2.7.5 \ + psycopg2-binary==2.8.6 \ pymongo \ pytest \ pytest-timeout \ + pytest-xdist \ redis \ tzlocal \ urllib3 \ diff --git a/docker/test/integration/runner/compose/docker_compose_mongo.yml b/docker/test/integration/runner/compose/docker_compose_mongo.yml index 8c54544ed88..e131fad21d6 100644 --- a/docker/test/integration/runner/compose/docker_compose_mongo.yml +++ b/docker/test/integration/runner/compose/docker_compose_mongo.yml @@ -7,5 +7,5 @@ services: MONGO_INITDB_ROOT_USERNAME: root MONGO_INITDB_ROOT_PASSWORD: clickhouse ports: - - 27018:27017 + - ${MONGO_EXTERNAL_PORT}:${MONGO_INTERNAL_PORT} command: --profile=2 --verbose From 20864076edd6619dbe6b2d526c9353557e53771e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 12 Feb 2021 18:51:21 +0300 Subject: [PATCH 0018/1060] wip --- tests/integration/helpers/cluster.py | 127 ++++++++++-------- .../test_mongo_uri.py | 2 +- 2 files changed, 72 insertions(+), 57 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 14aa2f252c5..43a1a0bd673 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -32,17 +32,23 @@ from .hdfs_api import HDFSApi HELPERS_DIR = p.dirname(__file__) CLICKHOUSE_ROOT_DIR = p.join(p.dirname(__file__), "../../..") LOCAL_DOCKER_COMPOSE_DIR = p.join(CLICKHOUSE_ROOT_DIR, "docker/test/integration/runner/compose/") -DEFAULT_ENV_NAME = 'env_file' +DEFAULT_ENV_NAME = '.env' SANITIZER_SIGN = "==================" -def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): - full_path = os.path.join(path, fname) - with open(full_path, 'w') as f: +def _create_env_file(path, variables): + with open(path, 'w') as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") - return full_path + return path + +def env_to_compose_args(env): + args = [] + for key, value in env.items(): + args += ["-e", "{}={}".format(key, value)] + return args + def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) @@ -50,8 +56,18 @@ def run_and_check(args, env=None, shell=False): # check_call(...) from subprocess does not print stderr, so we do it manually print('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) print('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) + print('Env:\n{}\n'.format(env)) raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) +# Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 +def get_open_port(): + import socket + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + s.bind(("",0)) + s.listen(1) + port = s.getsockname()[1] + s.close() + return port def subprocess_check_call(args): # Uncomment for debugging @@ -165,6 +181,10 @@ class ClickHouseCluster: self.schema_registry_host = "schema-registry" self.schema_registry_port = 8081 + # available when with_mongo == True + self.mongo_host = "mongo1" + self.mongo_port = get_open_port() + self.zookeeper_use_tmpfs = True self.docker_client = None @@ -246,6 +266,8 @@ class ClickHouseCluster: docker_compose_yml_dir = get_docker_compose_path() + assert instance.env_file is not None + self.instances[name] = instance if ipv4_address is not None or ipv6_address is not None: self.with_net_trics = True @@ -261,14 +283,14 @@ class ClickHouseCluster: self.with_zookeeper = True self.zookeeper_use_tmpfs = zookeeper_use_tmpfs self.base_cmd.extend(['--file', zookeeper_docker_compose_path]) - self.base_zookeeper_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_zookeeper_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', zookeeper_docker_compose_path] cmds.append(self.base_zookeeper_cmd) if with_mysql and not self.with_mysql: self.with_mysql = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) @@ -276,7 +298,7 @@ class ClickHouseCluster: if with_postgres and not self.with_postgres: self.with_postgres = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) @@ -285,56 +307,61 @@ class ClickHouseCluster: if not self.with_mysql: self.with_mysql = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] cmds.append(self.base_mysql_cmd) if not self.with_postgres: self.with_postgres = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] cmds.append(self.base_postgres_cmd) if with_kafka and not self.with_kafka: self.with_kafka = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) - self.base_kafka_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] cmds.append(self.base_kafka_cmd) if with_kerberized_kafka and not self.with_kerberized_kafka: + env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' self.with_kerberized_kafka = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_kerberized_kafka_cmd = ['docker-compose','--project-name', self.project_name, + self.base_kerberized_kafka_cmd = ['docker-compose', '--env-file', instance.env_file,'--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] cmds.append(self.base_kerberized_kafka_cmd) if with_rabbitmq and not self.with_rabbitmq: self.with_rabbitmq = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) - self.base_rabbitmq_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] cmds.append(self.base_rabbitmq_cmd) if with_hdfs and not self.with_hdfs: self.with_hdfs = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) - self.base_hdfs_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] cmds.append(self.base_hdfs_cmd) if with_kerberized_hdfs and not self.with_kerberized_hdfs: self.with_kerberized_hdfs = True + env_variables['KERBERIZED_HDFS_DIR'] = instance.path + '/' self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')]) - self.base_kerberized_hdfs_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_kerberized_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')] cmds.append(self.base_kerberized_hdfs_cmd) if with_mongo and not self.with_mongo: self.with_mongo = True + env_variables['MONGO_HOST'] = self.mongo_host + env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) + env_variables['MONGO_INTERNAL_PORT'] = "27017" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) - self.base_mongo_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] cmds.append(self.base_mongo_cmd) @@ -345,21 +372,30 @@ class ClickHouseCluster: if with_redis and not self.with_redis: self.with_redis = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) - self.base_redis_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] if with_minio and not self.with_minio: self.with_minio = True self.minio_certs_dir = minio_certs_dir + if self.minio_certs_dir: + env_variables['MINIO_CERTS_DIR'] = p.join(self.base_dir, self.minio_certs_dir) + # Minio client (urllib3) uses SSL_CERT_FILE for certificate validation. + env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') + else: + # Attach empty certificates directory to ensure non-secure mode. + minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') + os.mkdir(minio_certs_dir) + env_variables['MINIO_CERTS_DIR'] = minio_certs_dir self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) - self.base_minio_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] cmds.append(self.base_minio_cmd) if with_cassandra and not self.with_cassandra: self.with_cassandra = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) - self.base_cassandra_cmd = ['docker-compose', '--project-name', self.project_name, + self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] print("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( @@ -532,7 +568,7 @@ class ClickHouseCluster: def wait_mongo_to_start(self, timeout=30): connection_str = 'mongodb://{user}:{password}@{host}:{port}'.format( - host='localhost', port='27018', user='root', password='clickhouse') + host='localhost', port=self.mongo_port, user='root', password='clickhouse') connection = pymongo.MongoClient(connection_str) start = time.time() while time.time() - start < timeout: @@ -662,9 +698,7 @@ class ClickHouseCluster: if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: print('Setup kerberized kafka') - env = os.environ.copy() - env['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes'], env=env) + run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) @@ -678,9 +712,7 @@ class ClickHouseCluster: if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: print('Setup kerberized HDFS') - env = os.environ.copy() - env['KERBERIZED_HDFS_DIR'] = instance.path + '/' - run_and_check(self.base_kerberized_hdfs_cmd + common_opts, env=env) + run_and_check(self.base_kerberized_hdfs_cmd + common_opts) self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(timeout=300) @@ -691,38 +723,16 @@ class ClickHouseCluster: if self.with_redis and self.base_redis_cmd: print('Setup Redis') - subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate']) + subprocess_check_call(self.base_redis_cmd + common_opts) time.sleep(10) if self.with_minio and self.base_minio_cmd: - env = os.environ.copy() - prev_ca_certs = os.environ.get('SSL_CERT_FILE') - if self.minio_certs_dir: - minio_certs_dir = p.join(self.base_dir, self.minio_certs_dir) - env['MINIO_CERTS_DIR'] = minio_certs_dir - # Minio client (urllib3) uses SSL_CERT_FILE for certificate validation. - os.environ['SSL_CERT_FILE'] = p.join(minio_certs_dir, 'public.crt') - else: - # Attach empty certificates directory to ensure non-secure mode. - minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') - os.mkdir(minio_certs_dir) - env['MINIO_CERTS_DIR'] = minio_certs_dir - minio_start_cmd = self.base_minio_cmd + common_opts logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd))) - run_and_check(minio_start_cmd, env=env) - - try: - logging.info("Trying to connect to Minio...") - self.wait_minio_to_start(secure=self.minio_certs_dir is not None) - finally: - # Safely return previous value of SSL_CERT_FILE environment variable. - if self.minio_certs_dir: - if prev_ca_certs: - os.environ['SSL_CERT_FILE'] = prev_ca_certs - else: - os.environ.pop('SSL_CERT_FILE') + run_and_check(minio_start_cmd) + logging.info("Trying to connect to Minio...") + self.wait_minio_to_start(secure=self.minio_certs_dir is not None) if self.with_cassandra and self.base_cassandra_cmd: subprocess_check_call(self.base_cassandra_cmd + ['up', '-d', '--force-recreate']) @@ -843,7 +853,9 @@ class ClickHouseCluster: subprocess_check_call(self.base_zookeeper_cmd + ["start", n]) -CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/config.xml --log-file=/var/log/clickhouse-server/clickhouse-server.log --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" +CLICKHOUSE_START_COMMAND = "clickhouse server --config-file=/etc/clickhouse-server/config.xml" \ + " --log-file=/var/log/clickhouse-server/clickhouse-server.log " \ + " --errorlog-file=/var/log/clickhouse-server/clickhouse-server.err.log" CLICKHOUSE_STAY_ALIVE_COMMAND = 'bash -c "{} --daemon; tail -f /dev/null"'.format(CLICKHOUSE_START_COMMAND) @@ -933,6 +945,7 @@ class ClickHouseInstance: self.path = p.join(self.cluster.instances_dir, name) self.docker_compose_path = p.join(self.path, 'docker-compose.yml') self.env_variables = env_variables or {} + self.env_file = None if with_odbc_drivers: self.odbc_ini_path = self.path + "/odbc.ini:/etc/odbc.ini" self.with_mysql = True @@ -956,6 +969,8 @@ class ClickHouseInstance: self.ipv4_address = ipv4_address self.ipv6_address = ipv6_address self.with_installed_binary = with_installed_binary + self.env_file = os.path.join(os.path.dirname(self.docker_compose_path), DEFAULT_ENV_NAME) + def is_built_with_thread_sanitizer(self): build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'") @@ -1333,9 +1348,9 @@ class ClickHouseInstance: if self.with_minio: depends_on.append("minio1") - env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables) + _create_env_file(os.path.join(self.env_file), self.env_variables) - print("Env {} stored in {}".format(self.env_variables, env_file)) + print("Env {} stored in {}".format(self.env_variables, self.env_file)) odbc_ini_path = "" if self.odbc_ini_path: @@ -1383,7 +1398,7 @@ class ClickHouseInstance: logs_dir=logs_dir, depends_on=str(depends_on), user=os.getuid(), - env_file=env_file, + env_file=self.env_file, odbc_ini_path=odbc_ini_path, keytab_path=self.keytab_path, krb5_conf=self.krb5_conf, diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py index 9561ee5acc0..fdf4826cb63 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mongo_uri.py @@ -26,7 +26,7 @@ def setup_module(module): cluster = ClickHouseCluster(__file__, name=test_name) - SOURCE = SourceMongo("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") + SOURCE = SourceMongoURI("MongoDB", "localhost", cluster.mongo_port, cluster.mongo_host, "27017", "root", "clickhouse") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() From 06e1188d9e68c2fd8d79f767a49a31e0a3763321 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 15 Feb 2021 12:35:45 +0300 Subject: [PATCH 0019/1060] wip --- .../compose/docker_compose_cassandra.yml | 2 +- .../runner/compose/docker_compose_mysql.yml | 4 +- tests/integration/helpers/cluster.py | 44 ++++++++++++------- tests/integration/helpers/external_sources.py | 1 + .../test_cassandra.py | 8 ++-- .../test_mysql.py | 11 ++--- .../integration/test_dictionaries_ddl/test.py | 14 +++--- .../test_dictionaries_mysql/test.py | 7 +-- .../test_disabled_mysql_server/test.py | 11 ++--- .../materialize_with_ddl.py | 6 +-- .../test_materialize_mysql_database/test.py | 2 +- .../test_mysql_database_engine/test.py | 41 ++++++++--------- .../integration/test_odbc_interaction/test.py | 8 ++-- tests/integration/test_storage_mysql/test.py | 22 +++++----- 14 files changed, 101 insertions(+), 80 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_cassandra.yml b/docker/test/integration/runner/compose/docker_compose_cassandra.yml index 6567a352027..7b2d297d089 100644 --- a/docker/test/integration/runner/compose/docker_compose_cassandra.yml +++ b/docker/test/integration/runner/compose/docker_compose_cassandra.yml @@ -4,4 +4,4 @@ services: image: cassandra restart: always ports: - - 9043:9042 + - ${CASSANDRA_EXTERNAL_PORT}:${CASSANDRA_INTERNAL_PORT} diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 90daf8a4238..a9e540be99a 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -1,10 +1,10 @@ version: '2.3' services: - mysql1: + mysql57: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse ports: - - 3308:3306 + - ${MYSQL_EXTERNAL_PORT}:${MYSQL_INTERNAL_PORT} command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 43a1a0bd673..f6e277f3711 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -185,6 +185,14 @@ class ClickHouseCluster: self.mongo_host = "mongo1" self.mongo_port = get_open_port() + # available when with_cassandra == True + self.cassandra_host = "cassandra1" + self.cassandra_port = get_open_port() + + # available when with_mysql == True + self.mysql_host = "mysql57" + self.mysql_port = get_open_port() + self.zookeeper_use_tmpfs = True self.docker_client = None @@ -197,6 +205,17 @@ class ClickHouseCluster: cmd += " client" return cmd + def setup_mysql_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mysql = True + env_variables['MYSQL_HOST'] = self.mysql_host + env_variables['MYSQL_EXTERNAL_PORT'] = str(self.mysql_port) + env_variables['MYSQL_INTERNAL_PORT'] = "3306" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) + self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] + + return self.base_mysql_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -288,12 +307,7 @@ class ClickHouseCluster: cmds.append(self.base_zookeeper_cmd) if with_mysql and not self.with_mysql: - self.with_mysql = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] - - cmds.append(self.base_mysql_cmd) + cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) if with_postgres and not self.with_postgres: self.with_postgres = True @@ -305,11 +319,7 @@ class ClickHouseCluster: if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True if not self.with_mysql: - self.with_mysql = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) - self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] - cmds.append(self.base_mysql_cmd) + cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) if not self.with_postgres: self.with_postgres = True @@ -385,7 +395,7 @@ class ClickHouseCluster: else: # Attach empty certificates directory to ensure non-secure mode. minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') - os.mkdir(minio_certs_dir) + os.makedirs(minio_certs_dir, exist_ok=True) env_variables['MINIO_CERTS_DIR'] = minio_certs_dir self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, @@ -394,6 +404,8 @@ class ClickHouseCluster: if with_cassandra and not self.with_cassandra: self.with_cassandra = True + env_variables['CASSANDRA_EXTERNAL_PORT'] = str(self.cassandra_port) + env_variables['CASSANDRA_INTERNAL_PORT'] = "9042" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] @@ -488,7 +500,7 @@ class ClickHouseCluster: start = time.time() while time.time() - start < timeout: try: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) conn.close() print("Mysql Started") return @@ -622,7 +634,7 @@ class ClickHouseCluster: time.sleep(1) def wait_cassandra_to_start(self, timeout=30): - cass_client = cassandra.cluster.Cluster(["localhost"], port="9043") + cass_client = cassandra.cluster.Cluster(["localhost"], self.cassandra_port) start = time.time() while time.time() - start < timeout: try: @@ -1213,7 +1225,7 @@ class ClickHouseInstance: "Database": "clickhouse", "Uid": "root", "Pwd": "clickhouse", - "Server": "mysql1", + "Server": self.cluster.mysql_host, }, "PostgreSQL": { "DSN": "postgresql_odbc", @@ -1325,7 +1337,7 @@ class ClickHouseInstance: depends_on = [] if self.with_mysql: - depends_on.append("mysql1") + depends_on.append("mysql57") if self.with_kafka: depends_on.append("kafka1") diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index 47de9dd0caf..fe4c5269a56 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -100,6 +100,7 @@ class SourceMySQL(ExternalSource): def prepare(self, structure, table_name, cluster): self.create_mysql_conn() self.execute_mysql_query("create database if not exists test default character set 'utf8'") + self.execute_mysql_query("drop table if exists test.{}".format(table_name)) fields_strs = [] for field in structure.keys + structure.ordinary_fields + structure.range_fields: fields_strs.append(field.name + ' ' + self.TYPE_MAPPING[field.field_type]) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 8be25207d51..1271619f1f7 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -8,8 +8,7 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceCassandra -SOURCE = SourceCassandra("Cassandra", "localhost", "9043", "cassandra1", "9042", "", "") - +SOURCE = None cluster = None node = None simple_tester = None @@ -24,6 +23,9 @@ def setup_module(module): global complex_tester global ranged_tester + cluster = ClickHouseCluster(__file__, name=test_name) + + SOURCE = SourceCassandra("Cassandra", "localhost", cluster.cassandra_port, cluster.cassandra_host, "9042", "", "") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() @@ -36,8 +38,6 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) - main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 9bf3e888f31..119709bf229 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -8,8 +8,7 @@ from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceMySQL -SOURCE = SourceMySQL("MySQL", "localhost", "3308", "mysql1", "3306", "root", "clickhouse") - +SOURCE = None cluster = None node = None simple_tester = None @@ -25,6 +24,10 @@ def setup_module(module): global complex_tester global ranged_tester + cluster = ClickHouseCluster(__file__, name=test_name) + + SOURCE = SourceMySQL("MySQL", "localhost", cluster.mysql_port, cluster.mysql_host, "3306", "root", "clickhouse") + simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() simple_tester.create_dictionaries(SOURCE) @@ -36,15 +39,13 @@ def setup_module(module): ranged_tester.create_dictionaries(SOURCE) # Since that all .xml configs were created - cluster = ClickHouseCluster(__file__, name=test_name) - main_configs = [] main_configs.append(os.path.join('configs', 'disable_ssl_verification.xml')) main_configs.append(os.path.join('configs', 'log_conf.xml')) dictionaries = simple_tester.list_dictionaries() - node = cluster.add_instance('mysql_node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) + node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_mysql=True) def teardown_module(module): diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 3ea64383fbf..59ccc59d0af 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -4,6 +4,7 @@ import time import pymysql import pytest +import logging from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster @@ -23,6 +24,7 @@ node4 = cluster.add_instance('node4', user_configs=['configs/user_admin.xml', 'c def create_mysql_conn(user, password, hostname, port): + logging.debug("Created MySQL connection user:{}, password:{}, host:{}, port{}".format(user, password, hostname, port)) return pymysql.connect( user=user, password=password, @@ -31,6 +33,7 @@ def create_mysql_conn(user, password, hostname, port): def execute_mysql_query(connection, query): + logging.debug("Execute MySQL query:{}".format(query)) with warnings.catch_warnings(): warnings.simplefilter("ignore") with connection.cursor() as cursor: @@ -63,8 +66,9 @@ def started_cluster(): (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) - execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") + mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) + execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS clickhouse") + execute_mysql_query(mysql_conn, "CREATE DATABASE clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.{} (key_field1 int, key_field2 bigint, value1 text, value2 float, PRIMARY KEY (key_field1, key_field2))".format( name)) @@ -87,7 +91,7 @@ def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): DB 'clickhouse' TABLE '{}' REPLICA(PRIORITY 1 HOST '127.0.0.1' PORT 3333) - REPLICA(PRIORITY 2 HOST 'mysql1' PORT 3306) + REPLICA(PRIORITY 2 HOST 'mysql57' PORT 3306) )) {} LIFETIME(MIN 1 MAX 3) @@ -241,7 +245,7 @@ def test_file_dictionary_restrictions(started_cluster): def test_dictionary_with_where(started_cluster): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", 3308) + mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))") @@ -258,7 +262,7 @@ def test_dictionary_with_where(started_cluster): PASSWORD 'clickhouse' DB 'clickhouse' TABLE 'special_table' - REPLICA(PRIORITY 1 HOST 'mysql1' PORT 3306) + REPLICA(PRIORITY 1 HOST 'mysql57' PORT 3306) WHERE 'value1 = \\'qweqwe\\' OR value1 = \\'\\\\u3232\\'' )) LAYOUT(FLAT()) diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 16e432c6425..70af73a1592 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -37,7 +37,7 @@ def started_cluster(): instance.query("CREATE DATABASE IF NOT EXISTS test") # Create database in ClickChouse using MySQL protocol (will be used for data insertion) - instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql1:3306', 'test', 'root', 'clickhouse')") + instance.query("CREATE DATABASE clickhouse_mysql ENGINE = MySQL('mysql57:3306', 'test', 'root', 'clickhouse')") yield cluster @@ -66,7 +66,8 @@ def test_load_mysql_dictionaries(started_cluster): def create_mysql_db(mysql_connection, name): with mysql_connection.cursor() as cursor: - cursor.execute("CREATE DATABASE IF NOT EXISTS {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def prepare_mysql_table(table_name, index): @@ -88,7 +89,7 @@ def prepare_mysql_table(table_name, index): def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=cluster.mysql_port) return conn diff --git a/tests/integration/test_disabled_mysql_server/test.py b/tests/integration/test_disabled_mysql_server/test.py index a2cbcb17534..2fc84ee74a5 100644 --- a/tests/integration/test_disabled_mysql_server/test.py +++ b/tests/integration/test_disabled_mysql_server/test.py @@ -22,7 +22,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): self.user = user self.port = port self.hostname = hostname @@ -46,15 +46,16 @@ class MySQLNodeInstance: def test_disabled_mysql_server(started_cluster): with contextlib.closing(MySQLNodeInstance()) as mysql_node: - mysql_node.query("CREATE DATABASE test_db;") - mysql_node.query("CREATE TABLE test_db.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_db_disabled;") + mysql_node.query("CREATE DATABASE test_db_disabled;") + mysql_node.query("CREATE TABLE test_db_disabled.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") with PartitionManager() as pm: - clickhouse_node.query("CREATE DATABASE test_db ENGINE = MySQL('mysql1:3306', 'test_db', 'root', 'clickhouse')") + clickhouse_node.query("CREATE DATABASE test_db_disabled ENGINE = MySQL('mysql57:3306', 'test_db_disabled', 'root', 'clickhouse')") pm._add_rule({'source': clickhouse_node.ip_address, 'destination_port': 3306, 'action': 'DROP'}) clickhouse_node.query("SELECT * FROM system.parts") clickhouse_node.query("SELECT * FROM system.mutations") clickhouse_node.query("SELECT * FROM system.graphite_retentions") - clickhouse_node.query("DROP DATABASE test_db") + clickhouse_node.query("DROP DATABASE test_db_disabled") diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index c9be2387fc7..0dcb4c146c7 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -246,7 +246,7 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql1" else "(id)")) + "0" if service_name == "mysql57" else "(id)")) # create mapping clickhouse_node.query( @@ -264,9 +264,9 @@ def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( - "0" if service_name == "mysql1" else "(id)")) + "0" if service_name == "mysql57" else "(id)")) - default_expression = "DEFAULT\t0" if service_name == "mysql1" else "DEFAULT\tid" + default_expression = "DEFAULT\t0" if service_name == "mysql57" else "DEFAULT\tid" check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index e55772d9e1d..13c702f998b 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -28,7 +28,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=3308, docker_compose=None, project_name=cluster.project_name): + def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=cluster.mysql_port, docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 4d10e2ea6f5..f5fb9be337d 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -21,7 +21,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=3308): + def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): self.user = user self.port = port self.hostname = hostname @@ -55,11 +55,12 @@ class MySQLNodeInstance: def test_mysql_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") assert 'test_database' in clickhouse_node.query('SHOW DATABASES') mysql_node.query( @@ -88,13 +89,13 @@ def test_mysql_ddl_for_mysql_database(started_cluster): def test_clickhouse_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;') clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', 'test_database', 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', 'test_database', 'root', 'clickhouse')") assert 'test_table' in clickhouse_node.query('SHOW TABLES FROM test_database') clickhouse_node.query("DROP TABLE test_database.test_table") @@ -113,12 +114,12 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster): def test_clickhouse_dml_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `i``d` int(11) NOT NULL, PRIMARY KEY (`i``d`)) ENGINE=InnoDB;') clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test_database, 'root', 'clickhouse')") + "CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test_database, 'root', 'clickhouse')") assert clickhouse_node.query("SELECT count() FROM `test_database`.`test_table`").rstrip() == '0' clickhouse_node.query("INSERT INTO `test_database`.`test_table`(`i``d`) select number from numbers(10000)") @@ -131,7 +132,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster): def test_clickhouse_join_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test.t1_mysql_local (" "pays VARCHAR(55) DEFAULT 'FRA' NOT NULL," @@ -143,9 +144,9 @@ def test_clickhouse_join_for_mysql_database(started_cluster): "opco VARCHAR(5) DEFAULT ''" ")") clickhouse_node.query( - "CREATE TABLE default.t1_remote_mysql AS mysql('mysql1:3306','test','t1_mysql_local','root','clickhouse')") + "CREATE TABLE default.t1_remote_mysql AS mysql('mysql57:3306','test','t1_mysql_local','root','clickhouse')") clickhouse_node.query( - "CREATE TABLE default.t2_remote_mysql AS mysql('mysql1:3306','test','t2_mysql_local','root','clickhouse')") + "CREATE TABLE default.t2_remote_mysql AS mysql('mysql57:3306','test','t2_mysql_local','root','clickhouse')") assert clickhouse_node.query("SELECT s.pays " "FROM default.t1_remote_mysql AS s " "LEFT JOIN default.t1_remote_mysql AS s_ref " @@ -154,19 +155,19 @@ def test_clickhouse_join_for_mysql_database(started_cluster): def test_bad_arguments_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( - "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql1:3306', test_bad_arguments, root, 'clickhouse')") + "CREATE DATABASE test_database_bad_arguments ENGINE = MySQL('mysql57:3306', test_bad_arguments, root, 'clickhouse')") assert 'Database engine MySQL requested literal argument.' in str(exception.value) mysql_node.query("DROP DATABASE test_bad_arguments") def test_data_types_support_level_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse')", + clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") @@ -177,7 +178,7 @@ def test_data_types_support_level_for_mysql_database_engine(started_cluster): assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database FORMAT TSV") clickhouse_node.query( - "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql1:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", + "CREATE DATABASE test_database_1 ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse') SETTINGS mysql_datatypes_support_level = 'decimal,datetime64'", settings={"mysql_datatypes_support_level": "decimal"}) assert "SETTINGS mysql_datatypes_support_level = \\'decimal,datetime64\\'" in clickhouse_node.query("SHOW CREATE DATABASE test_database_1 FORMAT TSV") @@ -306,7 +307,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m else: return [do_execute(q) for q in query] - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=3308)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: execute_query(mysql_node, [ "DROP DATABASE IF EXISTS ${mysql_db}", "CREATE DATABASE ${mysql_db} DEFAULT CHARACTER SET 'utf8'", @@ -323,7 +324,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # MySQL TABLE ENGINE execute_query(clickhouse_node, [ "DROP TABLE IF EXISTS ${ch_mysql_table};", - "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", + "CREATE TABLE ${ch_mysql_table} (value ${expected_ch_type}) ENGINE = MySQL('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", ], settings=clickhouse_query_settings) # Validate type @@ -343,7 +344,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # MySQL DATABASE ENGINE execute_query(clickhouse_node, [ "DROP DATABASE IF EXISTS ${ch_mysql_db}", - "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql1:3306', '${mysql_db}', 'root', 'clickhouse')" + "CREATE DATABASE ${ch_mysql_db} ENGINE = MySQL('mysql57:3306', '${mysql_db}', 'root', 'clickhouse')" ], settings=clickhouse_query_settings) # Validate type @@ -364,7 +365,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m # Validate type assert \ execute_query(clickhouse_node, - "SELECT toTypeName(value) FROM mysql('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", + "SELECT toTypeName(value) FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse') LIMIT 1", settings=clickhouse_query_settings) \ == \ expected_ch_type @@ -374,5 +375,5 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m execute_query(mysql_node, "SELECT value FROM ${mysql_db}.${table_name}") \ == \ execute_query(clickhouse_node, - "SELECT value FROM mysql('mysql1:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", + "SELECT value FROM mysql('mysql57:3306', '${mysql_db}', '${table_name}', 'root', 'clickhouse')", settings=clickhouse_query_settings) diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 084fc407f39..470d816d2d1 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -27,14 +27,14 @@ create_table_sql_template = """ def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) return conn def create_mysql_db(conn, name): with conn.cursor() as cursor: - cursor.execute( - "CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def create_mysql_table(conn, table_name): @@ -119,7 +119,7 @@ def test_mysql_simple_select_works(started_cluster): settings={"external_table_functions_use_nulls": "0"}) == '0\n511\n' node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index 7b23e20e200..a285b263688 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -39,7 +39,7 @@ def test_many_connections(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query("INSERT INTO {} (id, name) SELECT number, concat('name_', toString(number)) from numbers(10) ".format(table_name)) @@ -58,7 +58,7 @@ def test_insert_select(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -74,7 +74,7 @@ def test_replace_select(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -93,7 +93,7 @@ def test_insert_on_duplicate_select(started_cluster): create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 0, 'update money = money + values(money)'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -111,7 +111,7 @@ def test_where(started_cluster): conn = get_mysql_conn() create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); '''.format(table_name, table_name)) node1.query( "INSERT INTO {}(id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000) ".format( @@ -130,7 +130,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): conn = get_mysql_conn() create_mysql_table(conn, 'table_function') - table_function = "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') + table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' node1.query( "INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format( @@ -152,7 +152,7 @@ def test_binary_type(started_cluster): conn = get_mysql_conn() with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") - table_function = "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') + table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') node1.query("INSERT INTO {} VALUES (42, 'clickhouse')".format('TABLE FUNCTION ' + table_function)) assert node1.query("SELECT * FROM {}".format(table_function)) == '42\tclickhouse\\0\\0\\0\\0\\0\\0\n' @@ -161,7 +161,7 @@ def test_enum_type(started_cluster): conn = get_mysql_conn() create_mysql_table(conn, table_name) node1.query(''' -CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); +CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); '''.format(table_name, table_name)) node1.query("INSERT INTO {} (id, name, age, money, source) VALUES (1, 'name', 0, 0, 'URL')".format(table_name)) assert node1.query("SELECT source FROM {} LIMIT 1".format(table_name)).rstrip() == 'URL' @@ -169,14 +169,14 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) return conn def create_mysql_db(conn, name): with conn.cursor() as cursor: - cursor.execute( - "CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) + cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) def create_mysql_table(conn, tableName): From 0b62ef4aa9eb3d49ff7681c6946d8de7a7aa58c1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 10:10:01 +0300 Subject: [PATCH 0020/1060] more fixes --- tests/integration/helpers/cluster.py | 67 ++++++- .../materialize_with_ddl.py | 51 +++--- .../test_materialize_mysql_database/test.py | 169 +++++++----------- .../integration/test_odbc_interaction/test.py | 27 ++- 4 files changed, 176 insertions(+), 138 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f6e277f3711..1b07122469a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -134,6 +134,8 @@ class ClickHouseCluster: self.project_name = re.sub(r'[^a-z0-9]', '', project_name.lower()) self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name)) self.docker_logs_path = p.join(self.instances_dir, 'docker.log') + self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) + self.env_variables = {} custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') self.docker_api_version = os.environ.get("DOCKER_API_VERSION") @@ -142,6 +144,7 @@ class ClickHouseCluster: self.base_cmd = ['docker-compose'] if custom_dockerd_host: self.base_cmd += ['--host', custom_dockerd_host] + self.base_cmd += ['--env-file', self.env_file] self.base_cmd += ['--project-name', self.project_name] self.base_zookeeper_cmd = None @@ -154,6 +157,7 @@ class ClickHouseCluster: self.instances = {} self.with_zookeeper = False self.with_mysql = False + self.with_mysql8 = False self.with_postgres = False self.with_kafka = False self.with_kerberized_kafka = False @@ -193,6 +197,10 @@ class ClickHouseCluster: self.mysql_host = "mysql57" self.mysql_port = get_open_port() + # available when with_mysql8 == True + self.mysql8_host = "mysql80" + self.mysql8_port = get_open_port() + self.zookeeper_use_tmpfs = True self.docker_client = None @@ -216,9 +224,20 @@ class ClickHouseCluster: return self.base_mysql_cmd + def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mysql8 = True + env_variables['MYSQL8_HOST'] = self.mysql8_host + env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port) + env_variables['MYSQL8_INTERNAL_PORT'] = "3306" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')]) + self.base_mysql8_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')] + + return self.base_mysql8_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, - with_zookeeper=False, with_mysql=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, + with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_kerberized_hdfs=False, with_mongo=False, with_redis=False, with_minio=False, with_cassandra=False, @@ -261,6 +280,7 @@ class ClickHouseCluster: with_zookeeper=with_zookeeper, zookeeper_config_path=self.zookeeper_config_path, with_mysql=with_mysql, + with_mysql8=with_mysql8, with_kafka=with_kafka, with_kerberized_kafka=with_kerberized_kafka, with_rabbitmq=with_rabbitmq, @@ -285,8 +305,6 @@ class ClickHouseCluster: docker_compose_yml_dir = get_docker_compose_path() - assert instance.env_file is not None - self.instances[name] = instance if ipv4_address is not None or ipv6_address is not None: self.with_net_trics = True @@ -309,6 +327,9 @@ class ClickHouseCluster: if with_mysql and not self.with_mysql: cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_mysql8 and not self.with_mysql8: + cmds.append(self.setup_mysql8_cmd(instance, env_variables, docker_compose_yml_dir)) + if with_postgres and not self.with_postgres: self.with_postgres = True self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) @@ -445,6 +466,9 @@ class ClickHouseCluster: node.wait_for_start(start_deadline) return node + def restart_service(self, service_name): + run_and_check(self.base_cmd + ["restart", service_name]) + def get_instance_ip(self, instance_name): print("get_instance_ip instance_name={}".format(instance_name)) docker_id = self.get_instance_docker_id(instance_name) @@ -498,6 +522,7 @@ class ClickHouseCluster: def wait_mysql_to_start(self, timeout=60): start = time.time() + errors = [] while time.time() - start < timeout: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) @@ -505,12 +530,28 @@ class ClickHouseCluster: print("Mysql Started") return except Exception as ex: - print("Can't connect to MySQL " + str(ex)) + errors += [str(ex)] time.sleep(0.5) subprocess_call(['docker-compose', 'ps', '--services', '--all']) + logging.error("Can't connect to MySQL:{}".format(errors)) raise Exception("Cannot wait MySQL container") + def wait_mysql8_to_start(self, timeout=60): + start = time.time() + while time.time() - start < timeout: + try: + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql8_port) + conn.close() + print("Mysql 8 Started") + return + except Exception as ex: + print("Can't connect to MySQL 8 " + str(ex)) + time.sleep(0.5) + + subprocess_call(['docker-compose', 'ps', '--services', '--all']) + raise Exception("Cannot wait MySQL 8 container") + def wait_postgres_to_start(self, timeout=60): start = time.time() while time.time() - start < timeout: @@ -671,7 +712,7 @@ class ClickHouseCluster: self.docker_client = docker.from_env(version=self.docker_api_version) - common_opts = ['up', '-d', '--force-recreate'] + common_opts = ['up', '-d', '--force-recreate', '--remove-orphans'] if self.with_zookeeper and self.base_zookeeper_cmd: print('Setup ZooKeeper') @@ -697,6 +738,11 @@ class ClickHouseCluster: subprocess_check_call(self.base_mysql_cmd + common_opts) self.wait_mysql_to_start(120) + if self.with_mysql8 and self.base_mysql8_cmd: + print('Setup MySQL 8') + subprocess_check_call(self.base_mysql8_cmd + common_opts) + self.wait_mysql8_to_start(120) + if self.with_postgres and self.base_postgres_cmd: print('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) @@ -750,6 +796,8 @@ class ClickHouseCluster: subprocess_check_call(self.base_cassandra_cmd + ['up', '-d', '--force-recreate']) self.wait_cassandra_to_start() + _create_env_file(os.path.join(self.env_file), self.env_variables) + clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) subprocess.check_output(clickhouse_start_cmd) @@ -796,7 +844,7 @@ class ClickHouseCluster: subprocess_check_call(self.base_cmd + ['kill']) try: - subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) + subprocess_check_call(self.base_cmd + ['down', '--volumes']) except Exception as e: print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) @@ -917,7 +965,7 @@ class ClickHouseInstance: def __init__( self, cluster, base_path, name, base_config_dir, custom_main_configs, custom_user_configs, custom_dictionaries, - macros, with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs, + macros, with_zookeeper, zookeeper_config_path, with_mysql, with_mysql8, with_kafka, with_kerberized_kafka, with_rabbitmq, with_kerberized_hdfs, with_mongo, with_redis, with_minio, with_cassandra, server_bin_path, odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables=None, @@ -945,6 +993,7 @@ class ClickHouseInstance: self.odbc_bridge_bin_path = odbc_bridge_bin_path self.with_mysql = with_mysql + self.with_mysql8 = with_mysql8 self.with_kafka = with_kafka self.with_kerberized_kafka = with_kerberized_kafka self.with_rabbitmq = with_rabbitmq @@ -1339,6 +1388,9 @@ class ClickHouseInstance: if self.with_mysql: depends_on.append("mysql57") + if self.with_mysql8: + depends_on.append("mysql80") + if self.with_kafka: depends_on.append("kafka1") depends_on.append("schema-registry") @@ -1360,6 +1412,7 @@ class ClickHouseInstance: if self.with_minio: depends_on.append("minio1") + self.cluster.env_variables.update(self.env_variables) _create_env_file(os.path.join(self.env_file), self.env_variables) print("Env {} stored in {}".format(self.env_variables, self.env_file)) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 0dcb4c146c7..1c9655b457f 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -31,12 +31,12 @@ def check_query(clickhouse_node, query, result_set, retry_count=60, interval_sec def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_dml") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_dml") + mysql_node.query("CREATE DATABASE test_database_dml DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (" + mysql_node.query("CREATE TABLE test_database_dml.test_table_1 (" "`key` INT NOT NULL PRIMARY KEY, " "unsigned_tiny_int TINYINT UNSIGNED, tiny_int TINYINT, " "unsigned_small_int SMALLINT UNSIGNED, small_int SMALLINT, " @@ -53,68 +53,68 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam # it already has some data mysql_node.query(""" - INSERT INTO test_database.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', + INSERT INTO test_database_dml.test_table_1 VALUES(1, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', '2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', true); """) clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_dml ENGINE = MaterializeMySQL('{}:3306', 'test_database_dml', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") + assert "test_database_dml" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n") mysql_node.query(""" - INSERT INTO test_database.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', + INSERT INTO test_database_dml.test_table_1 VALUES(2, 1, -1, 2, -2, 3, -3, 4, -4, 5, -5, 6, -6, 3.2, -3.2, 3.4, -3.4, 'varchar', 'char', 'binary', '2020-01-01', '2020-01-01 00:00:00', '2020-01-01 00:00:00', false); """) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "1\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t2020-01-01 00:00:00\t0\n") - mysql_node.query("UPDATE test_database.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") + mysql_node.query("UPDATE test_database_dml.test_table_1 SET unsigned_tiny_int = 2 WHERE `key` = 1") check_query(clickhouse_node, """ SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int, small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ - _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV + _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV """, "1\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t1\n2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t0\n") # update primary key - mysql_node.query("UPDATE test_database.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") + mysql_node.query("UPDATE test_database_dml.test_table_1 SET `key` = 3 WHERE `unsigned_tiny_int` = 2") check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, " " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " - " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + " _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "2\t1\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\t" "varchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t0\n3\t2\t-1\t2\t-2\t3\t-3\t" "4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t2020-01-01 00:00:00\t1\n") - mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `key` = 2') + mysql_node.query('DELETE FROM test_database_dml.test_table_1 WHERE `key` = 2') check_query(clickhouse_node, "SELECT key, unsigned_tiny_int, tiny_int, unsigned_small_int," " small_int, unsigned_medium_int, medium_int, unsigned_int, _int, unsigned_integer, _integer, " " unsigned_bigint, _bigint, unsigned_float, _float, unsigned_double, _double, _varchar, _char, binary_col, " " _date, _datetime, /* exclude it, because ON UPDATE CURRENT_TIMESTAMP _timestamp, */ " - " _bool FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + " _bool FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "3\t2\t-1\t2\t-2\t3\t-3\t4\t-4\t5\t-5\t6\t-6\t3.2\t-3.2\t3.4\t-3.4\tvarchar\tchar\tbinary\\0\\0\t2020-01-01\t" "2020-01-01 00:00:00\t1\n") - mysql_node.query('DELETE FROM test_database.test_table_1 WHERE `unsigned_tiny_int` = 2') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", "") + mysql_node.query('DELETE FROM test_database_dml.test_table_1 WHERE `unsigned_tiny_int` = 2') + check_query(clickhouse_node, "SELECT * FROM test_database_dml.test_table_1 ORDER BY key FORMAT TSV", "") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_dml") + mysql_node.query("DROP DATABASE test_database_dml") def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): @@ -525,7 +525,7 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABАASE IF EXISTS priv_err_db") mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") @@ -670,6 +670,7 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") mysql_node.query("CREATE DATABASE kill_mysql_while_insert") mysql_node.query("CREATE TABLE kill_mysql_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse')".format(service_name)) @@ -684,17 +685,14 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): t = threading.Thread(target=insert, args=(10000,)) t.start() - run_and_check( - ['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'stop']) + clickhouse_node.cluster.restart_service(service_name) finally: with pytest.raises(QueryRuntimeException) as execption: time.sleep(5) clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test") assert "Master maybe lost." in str(execption.value) - run_and_check( - ['docker-compose', '-p', mysql_node.project_name, '-f', mysql_node.docker_compose, 'start']) - mysql_node.wait_mysql_to_start(120) + mysql_node.alloc_connection() clickhouse_node.query("DETACH DATABASE kill_mysql_while_insert") clickhouse_node.query("ATTACH DATABASE kill_mysql_while_insert") @@ -709,6 +707,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): def clickhouse_killed_while_insert(clickhouse_node, mysql_node, service_name): + mysql_node.query("DROP DATABASE IF EXISTS kill_clickhouse_while_insert") mysql_node.query("CREATE DATABASE kill_clickhouse_while_insert") mysql_node.query("CREATE TABLE kill_clickhouse_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") clickhouse_node.query("CREATE DATABASE kill_clickhouse_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_clickhouse_while_insert', 'root', 'clickhouse')".format(service_name)) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index 13c702f998b..04c9868a282 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -7,15 +7,18 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check import docker +import logging from . import materialize_with_ddl DOCKER_COMPOSE_PATH = get_docker_compose_path() cluster = ClickHouseCluster(__file__) +mysql_node = None +mysql8_node = None -node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=False, stay_alive=True) -node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql=False, stay_alive=True) +node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=True, stay_alive=True) +node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql8=True, stay_alive=True) @pytest.fixture(scope="module") @@ -27,26 +30,29 @@ def started_cluster(): cluster.shutdown() -class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', ip_address='127.0.0.1', port=cluster.mysql_port, docker_compose=None, project_name=cluster.project_name): +class MySQLConnection: + def __init__(self, port, user='root', password='clickhouse', ip_address='127.0.0.1', docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address self.password = password self.mysql_connection = None # lazy init - self.docker_compose = docker_compose - self.project_name = project_name - def alloc_connection(self): - if self.mysql_connection is None: - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, - port=self.port, autocommit=True) - else: - if self.mysql_connection.ping(): - self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, - port=self.port, autocommit=True) - return self.mysql_connection + errors = [] + for _ in range(5): + try: + if self.mysql_connection is None: + self.mysql_connection = pymysql.connect(user=self.user, password=self.password, host=self.ip_address, + port=self.port, autocommit=True) + else: + self.mysql_connection.ping(reconnect=True) + logging.debug("MySQL Connection establised: {}:{}".format(self.ip_address, self.port)) + return self.mysql_connection + except Exception as e: + errors += [str(e)] + time.sleep(1) + raise Exception("Connection not establised, {}".format(errors)) def query(self, execution_query): with self.alloc_connection().cursor() as cursor: @@ -75,174 +81,133 @@ class MySQLNodeInstance: if self.mysql_connection is not None: self.mysql_connection.close() - def wait_mysql_to_start(self, timeout=60): - start = time.time() - while time.time() - start < timeout: - try: - self.alloc_connection() - print("Mysql Started") - return - except Exception as ex: - print("Can't connect to MySQL " + str(ex)) - time.sleep(0.5) - - run_and_check(['docker-compose', 'ps', '--services', 'all']) - raise Exception("Cannot wait MySQL container") - @pytest.fixture(scope="module") def started_mysql_5_7(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose) - - try: - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) - mysql_node.wait_mysql_to_start(120) - yield mysql_node - finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) - + mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', '127.0.0.1') + yield mysql_node @pytest.fixture(scope="module") def started_mysql_8_0(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose) - - try: - run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d']) - mysql_node.wait_mysql_to_start(120) - yield mysql_node - finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) - + mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', '127.0.0.1') + yield mysql8_node @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") # mysql 5.7 cannot support alter rename column - # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + # materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") - materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") - materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, - "mysql8_0") + materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_add_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_drop_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") + materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_select_without_columns_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_select_without_columns_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_insert_with_modify_binlog_checksum_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_insert_with_modify_binlog_checksum_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): - materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql1") - materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql57") + materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql80") @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_tables_table(started_cluster, started_mysql_8_0, clickhouse_node): - materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql80") diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 470d816d2d1..3ef59e7bd23 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -3,6 +3,8 @@ import time import psycopg2 import pymysql.cursors import pytest +import logging + from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT @@ -15,6 +17,11 @@ node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, 'configs/dictionaries/sqlite3_odbc_cached_dictionary.xml', 'configs/dictionaries/postgres_odbc_hashed_dictionary.xml'], stay_alive=True) + +drop_table_sql_template = """ + DROP TABLE IF EXISTS `clickhouse`.`{}` + """ + create_table_sql_template = """ CREATE TABLE `clickhouse`.`{}` ( `id` int(11) NOT NULL, @@ -27,9 +34,22 @@ create_table_sql_template = """ def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) - return conn - + errors = [] + conn = None + for _ in range(5): + try: + if conn is None: + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + else: + conn.ping(reconnect=True) + logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) + def create_mysql_db(conn, name): with conn.cursor() as cursor: @@ -39,6 +59,7 @@ def create_mysql_db(conn, name): def create_mysql_table(conn, table_name): with conn.cursor() as cursor: + cursor.execute(drop_table_sql_template.format(table_name)) cursor.execute(create_table_sql_template.format(table_name)) From 178d3fdba07e9c8dda213a503bb9787d10fff9a1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 10:36:15 +0300 Subject: [PATCH 0021/1060] typo --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 1c9655b457f..096ffc2a8f4 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -525,7 +525,7 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABАASE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABАSE IF EXISTS priv_err_db") mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") From e4b3511d28420875e289c1284606a56a8a738b8e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 10:37:35 +0300 Subject: [PATCH 0022/1060] no more separate compose files for materialize mysql --- ...ompose_mysql_5_7_for_materialize_mysql.yml | 10 --------- ...mysql.yml => docker_compose_mysql_8_0.yml} | 6 +++--- .../configs/dictionaries/mysql_dict1.xml | 4 ++-- .../configs/dictionaries/mysql_dict2.xml | 6 +++--- .../test_dictionaries_mysql/test.py | 21 +++++++++++++++---- 5 files changed, 25 insertions(+), 22 deletions(-) delete mode 100644 docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml rename docker/test/integration/runner/compose/{docker_compose_mysql_8_0_for_materialize_mysql.yml => docker_compose_mysql_8_0.yml} (76%) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml deleted file mode 100644 index e7d762203ee..00000000000 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ /dev/null @@ -1,10 +0,0 @@ -version: '2.3' -services: - mysql1: - image: mysql:5.7 - restart: 'no' - environment: - MYSQL_ROOT_PASSWORD: clickhouse - ports: - - 3308:3306 - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml similarity index 76% rename from docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml rename to docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml index 918a2b5f80f..7b551371ab2 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -1,10 +1,10 @@ version: '2.3' services: - mysql8_0: + mysql80: image: mysql:8.0 - restart: 'no' + restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse ports: - - 33308:3306 + - ${MYSQL8_EXTERNAL_PORT}:${MYSQL8_INTERNAL_PORT} command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml index 514c73f3be2..2242088ebc4 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict1.xml @@ -5,7 +5,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -41,7 +41,7 @@ test - mysql1 + mysql57 3306 root clickhouse diff --git a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml index 91506481cc9..278fad49d03 100644 --- a/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml +++ b/tests/integration/test_dictionaries_mysql/configs/dictionaries/mysql_dict2.xml @@ -5,7 +5,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -42,7 +42,7 @@ test - mysql1 + mysql57 3306 root clickhouse @@ -78,7 +78,7 @@ test - mysql1 + mysql57 3306 root clickhouse diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 70af73a1592..90cfe53dd68 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -2,6 +2,8 @@ import pymysql.cursors import pytest from helpers.cluster import ClickHouseCluster +import time +import logging CONFIG_FILES = ['configs/dictionaries/mysql_dict1.xml', 'configs/dictionaries/mysql_dict2.xml', 'configs/remote_servers.xml'] @@ -87,11 +89,22 @@ def prepare_mysql_table(table_name, index): # Create CH Dictionary tables based on MySQL tables query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index))) - def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.10', port=cluster.mysql_port) - return conn - + errors = [] + conn = None + for _ in range(5): + try: + if conn is None: + conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + else: + conn.ping(reconnect=True) + logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) def create_mysql_table(conn, table_name): with conn.cursor() as cursor: From 3d1bcb1272c0279978b752af2bc827a3d67d03ac Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 11:04:52 +0300 Subject: [PATCH 0023/1060] fix --- .../test_materialize_mysql_database/materialize_with_ddl.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 096ffc2a8f4..1e4d51e325c 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -525,7 +525,7 @@ def insert_with_modify_binlog_checksum(clickhouse_node, mysql_node, service_name def err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABАSE IF EXISTS priv_err_db") + clickhouse_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("DROP DATABASE IF EXISTS priv_err_db") mysql_node.query("CREATE DATABASE priv_err_db DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE priv_err_db.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") From 8603855f5fd70f282566f9113fd27447e0a876a9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Feb 2021 17:16:15 +0300 Subject: [PATCH 0024/1060] fix --- .../compose/docker_compose_rabbitmq.yml | 4 +- .../runner/compose/docker_compose_redis.yml | 2 +- tests/integration/helpers/cluster.py | 44 ++++++++++--- .../test_dictionaries_redis/test.py | 7 +- .../integration/test_storage_rabbitmq/test.py | 64 +++++++++---------- 5 files changed, 74 insertions(+), 47 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml index 1e9c3777505..d16a19f7ac2 100644 --- a/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml +++ b/docker/test/integration/runner/compose/docker_compose_rabbitmq.yml @@ -5,8 +5,8 @@ services: image: rabbitmq:3-management hostname: rabbitmq1 ports: - - "5672:5672" - - "15672:15672" + - ${RABBITMQ_EXTERNAL_PORT}:${RABBITMQ_INTERNAL_PORT} + - ${RABBITMQ_EXTERNAL_HTTP_PORT}:${RABBITMQ_INTERNAL_HTTP_PORT} environment: RABBITMQ_DEFAULT_USER: "root" RABBITMQ_DEFAULT_PASS: "clickhouse" diff --git a/docker/test/integration/runner/compose/docker_compose_redis.yml b/docker/test/integration/runner/compose/docker_compose_redis.yml index 72df99ec59b..b14d0dce98b 100644 --- a/docker/test/integration/runner/compose/docker_compose_redis.yml +++ b/docker/test/integration/runner/compose/docker_compose_redis.yml @@ -4,5 +4,5 @@ services: image: redis restart: always ports: - - 6380:6379 + - ${REDIS_EXTERNAL_PORT}:${REDIS_INTERNAL_PORT} command: redis-server --requirepass "clickhouse" --databases 32 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1b07122469a..560650a09d7 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -193,6 +193,15 @@ class ClickHouseCluster: self.cassandra_host = "cassandra1" self.cassandra_port = get_open_port() + # available when with_rabbitmq == True + self.rabbitmq_host = "rabbitmq1" + self.rabbitmq_port = get_open_port() + self.rabbitmq_http_port = get_open_port() + + # available when with_redis == True + self.redis_host = "redis1" + self.redis_port = get_open_port() + # available when with_mysql == True self.mysql_host = "mysql57" self.mysql_port = get_open_port() @@ -235,6 +244,30 @@ class ClickHouseCluster: return self.base_mysql8_cmd + def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_redis = True + env_variables['REDIS_HOST'] = self.redis_host + env_variables['REDIS_EXTERNAL_PORT'] = str(self.redis_port) + env_variables['REDIS_INTERNAL_PORT'] = "6379" + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] + return self.base_redis_cmd + + def setup_rabbitmq_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_rabbitmq = True + env_variables['RABBITMQ_HOST'] = self.rabbitmq_host + env_variables['RABBITMQ_EXTERNAL_PORT'] = str(self.rabbitmq_port) + env_variables['RABBITMQ_INTERNAL_PORT'] = "5672" + env_variables['RABBITMQ_EXTERNAL_HTTP_PORT'] = str(self.rabbitmq_http_port) + env_variables['RABBITMQ_INTERNAL_HTTP_PORT'] = "15672" + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) + self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] + return self.base_rabbitmq_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -365,11 +398,7 @@ class ClickHouseCluster: cmds.append(self.base_kerberized_kafka_cmd) if with_rabbitmq and not self.with_rabbitmq: - self.with_rabbitmq = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')]) - self.base_rabbitmq_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] - cmds.append(self.base_rabbitmq_cmd) + cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) if with_hdfs and not self.with_hdfs: self.with_hdfs = True @@ -401,10 +430,7 @@ class ClickHouseCluster: cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_net.yml')]) if with_redis and not self.with_redis: - self.with_redis = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')]) - self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_redis.yml')] + cmds.append(self.setup_redis_cmd(instance, env_variables, docker_compose_yml_dir)) if with_minio and not self.with_minio: self.with_minio = True diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index d7d7e0ee3ad..7fcbd0d0909 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -93,15 +93,17 @@ def setup_module(module): global node global dict_configs_path + cluster = ClickHouseCluster(__file__) + for f in os.listdir(dict_configs_path): os.remove(os.path.join(dict_configs_path, f)) for i, field in enumerate(FIELDS): DICTIONARIES.append([]) sources = [] - sources.append(SourceRedis("RedisSimple", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2, + sources.append(SourceRedis("RedisSimple", "localhost", cluster.redis_port, cluster.redis_host, "6379", "", "clickhouse", i * 2, storage_type="simple")) - sources.append(SourceRedis("RedisHash", "localhost", "6380", "redis1", "6379", "", "clickhouse", i * 2 + 1, + sources.append(SourceRedis("RedisHash", "localhost", cluster.redis_port, cluster.redis_host, "6379", "", "clickhouse", i * 2 + 1, storage_type="hash_map")) for source in sources: for layout in LAYOUTS: @@ -117,7 +119,6 @@ def setup_module(module): for fname in os.listdir(dict_configs_path): dictionaries.append(os.path.join(dict_configs_path, fname)) - cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node', main_configs=main_configs, dictionaries=dictionaries, with_redis=True) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 911f6d144f9..c78a5f8d600 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -128,14 +128,14 @@ def test_rabbitmq_select(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'select', rabbitmq_format = 'JSONEachRow', rabbitmq_row_delimiter = '\\n'; - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -164,11 +164,11 @@ def test_rabbitmq_select_empty(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'empty', rabbitmq_format = 'TSV', rabbitmq_row_delimiter = '\\n'; - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) assert int(instance.query('SELECT count() FROM test.rabbitmq')) == 0 @@ -178,13 +178,13 @@ def test_rabbitmq_json_without_delimiter(rabbitmq_cluster): instance.query(''' CREATE TABLE test.rabbitmq (key UInt64, value UInt64) ENGINE = RabbitMQ - SETTINGS rabbitmq_host_port = 'rabbitmq1:5672', + SETTINGS rabbitmq_host_port = '{}:5672', rabbitmq_exchange_name = 'json', rabbitmq_format = 'JSONEachRow' - ''') + '''.format(rabbitmq_cluster.rabbitmq_host)) credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -227,7 +227,7 @@ def test_rabbitmq_csv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -262,7 +262,7 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -304,7 +304,7 @@ def test_rabbitmq_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -347,7 +347,7 @@ def test_rabbitmq_materialized_view_with_subquery(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -397,7 +397,7 @@ def test_rabbitmq_many_materialized_views(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -445,7 +445,7 @@ def test_rabbitmq_protobuf(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -498,7 +498,7 @@ def test_rabbitmq_big_message(rabbitmq_cluster): messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(rabbitmq_messages)] credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -562,7 +562,7 @@ def test_rabbitmq_sharding_between_queues_publish(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -639,7 +639,7 @@ def test_rabbitmq_mv_combo(rabbitmq_cluster): messages_num = 10000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -697,7 +697,7 @@ def test_rabbitmq_insert(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -750,7 +750,7 @@ def test_rabbitmq_insert_headers_exchange(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) consumer_connection = pika.BlockingConnection(parameters) consumer = consumer_connection.channel() @@ -979,7 +979,7 @@ def test_rabbitmq_direct_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1052,7 +1052,7 @@ def test_rabbitmq_fanout_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1138,7 +1138,7 @@ def test_rabbitmq_topic_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1216,7 +1216,7 @@ def test_rabbitmq_hash_exchange(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1295,7 +1295,7 @@ def test_rabbitmq_multiple_bindings(rabbitmq_cluster): messages_num = 500 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): # init connection here because otherwise python rabbitmq client might fail @@ -1392,7 +1392,7 @@ def test_rabbitmq_headers_exchange(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1446,7 +1446,7 @@ def test_rabbitmq_virtual_columns(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1513,7 +1513,7 @@ def test_rabbitmq_virtual_columns_with_materialized_view(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1591,7 +1591,7 @@ def test_rabbitmq_many_consumers_to_each_queue(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) def produce(): connection = pika.BlockingConnection(parameters) @@ -1671,7 +1671,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_1(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1731,7 +1731,7 @@ def test_rabbitmq_restore_failed_connection_without_losses_2(rabbitmq_cluster): messages_num = 150000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1803,7 +1803,7 @@ def test_rabbitmq_commit_on_block_write(rabbitmq_cluster): ''') credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() @@ -1889,7 +1889,7 @@ def test_rabbitmq_no_connection_at_startup(rabbitmq_cluster): messages_num = 1000 credentials = pika.PlainCredentials('root', 'clickhouse') - parameters = pika.ConnectionParameters('localhost', 5672, '/', credentials) + parameters = pika.ConnectionParameters('localhost', rabbitmq_cluster.rabbitmq_port, '/', credentials) connection = pika.BlockingConnection(parameters) channel = connection.channel() for i in range(messages_num): From 677528326fa7401122a8d47b837f1174e3aa7b89 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 17 Feb 2021 18:40:01 +0300 Subject: [PATCH 0025/1060] postgres --- .../compose/docker_compose_postgres.yml | 9 +- ...esql.yml => docker_compose_postgresql.yml} | 0 ...docker_compose_postgresql_java_client.yml} | 0 tests/integration/helpers/client.py | 12 +-- tests/integration/helpers/cluster.py | 85 ++++++++++--------- .../test_dictionaries_postgresql/test.py | 20 ++--- .../postgres_odbc_hashed_dictionary.xml | 2 +- .../integration/test_odbc_interaction/test.py | 31 ++++--- .../test_postgresql_database_engine/test.py | 4 +- .../test_postgresql_protocol/test.py | 4 +- .../test_storage_postgresql/test.py | 4 +- 11 files changed, 98 insertions(+), 73 deletions(-) rename docker/test/integration/runner/compose/{docker_compose_postgesql.yml => docker_compose_postgresql.yml} (100%) rename docker/test/integration/runner/compose/{docker_compose_postgesql_java_client.yml => docker_compose_postgresql_java_client.yml} (100%) diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index fff4fb1fa42..cb0017ccd33 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -6,8 +6,15 @@ services: environment: POSTGRES_PASSWORD: mysecretpassword ports: - - 5432:5432 + - ${POSTGRES_EXTERNAL_PORT}:${POSTGRES_INTERNAL_PORT} + healthcheck: + test: ["CMD-SHELL", "pg_isready -U postgres"] + interval: 10s + timeout: 5s + retries: 5 networks: default: aliases: - postgre-sql.local + environment: + POSTGRES_HOST_AUTH_METHOD: "trust" \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql.yml b/docker/test/integration/runner/compose/docker_compose_postgresql.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgesql.yml rename to docker/test/integration/runner/compose/docker_compose_postgresql.yml diff --git a/docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml b/docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml similarity index 100% rename from docker/test/integration/runner/compose/docker_compose_postgesql_java_client.yml rename to docker/test/integration/runner/compose/docker_compose_postgresql_java_client.yml diff --git a/tests/integration/helpers/client.py b/tests/integration/helpers/client.py index 04b6082e95f..95f8a58dbf1 100644 --- a/tests/integration/helpers/client.py +++ b/tests/integration/helpers/client.py @@ -101,8 +101,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read().decode() - stderr = self.stderr_file.read().decode() + stdout = self.stdout_file.read().decode('utf-8', errors='replace') + stderr = self.stderr_file.read().decode('utf-8', errors='replace') if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') @@ -118,8 +118,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read().decode() - stderr = self.stderr_file.read().decode() + stdout = self.stdout_file.read().decode('utf-8', errors='replace') + stderr = self.stderr_file.read().decode('utf-8', errors='replace') if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') @@ -134,8 +134,8 @@ class CommandRequest: self.stdout_file.seek(0) self.stderr_file.seek(0) - stdout = self.stdout_file.read().decode() - stderr = self.stderr_file.read().decode() + stdout = self.stdout_file.read().decode('utf-8', errors='replace') + stderr = self.stderr_file.read().decode('utf-8', errors='replace') if self.timer is not None and not self.process_finished_before_timeout and not self.ignore_error: raise QueryTimeoutExceedException('Client timed out!') diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 560650a09d7..d582c0b7401 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -38,6 +38,7 @@ SANITIZER_SIGN = "==================" def _create_env_file(path, variables): + logging.debug("Env {} stored in {}".format(variables, path)) with open(path, 'w') as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") @@ -71,7 +72,7 @@ def get_open_port(): def subprocess_check_call(args): # Uncomment for debugging - # print('run:', ' ' . join(args)) + print('run:', ' '.join(args)) run_and_check(args) @@ -136,6 +137,7 @@ class ClickHouseCluster: self.docker_logs_path = p.join(self.instances_dir, 'docker.log') self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} + self.up_called = False custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') self.docker_api_version = os.environ.get("DOCKER_API_VERSION") @@ -202,6 +204,10 @@ class ClickHouseCluster: self.redis_host = "redis1" self.redis_port = get_open_port() + # available when with_postgres == True + self.postgres_host = "postgres1" + self.postgres_port = get_open_port() + # available when with_mysql == True self.mysql_host = "mysql57" self.mysql_port = get_open_port() @@ -268,6 +274,16 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_rabbitmq.yml')] return self.base_rabbitmq_cmd + def setup_postgres_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) + env_variables['POSTGRES_HOST'] = self.postgres_host + env_variables['POSTGRES_EXTERNAL_PORT'] = str(self.postgres_port) + env_variables['POSTGRES_INTERNAL_PORT'] = "5432" + self.with_postgres = True + self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] + return self.base_postgres_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -364,11 +380,7 @@ class ClickHouseCluster: cmds.append(self.setup_mysql8_cmd(instance, env_variables, docker_compose_yml_dir)) if with_postgres and not self.with_postgres: - self.with_postgres = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] - cmds.append(self.base_postgres_cmd) + cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True @@ -376,11 +388,7 @@ class ClickHouseCluster: cmds.append(self.setup_mysql_cmd(instance, env_variables, docker_compose_yml_dir)) if not self.with_postgres: - self.with_postgres = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] - cmds.append(self.base_postgres_cmd) + cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kafka and not self.with_kafka: self.with_kafka = True @@ -582,8 +590,7 @@ class ClickHouseCluster: start = time.time() while time.time() - start < timeout: try: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" - conn = psycopg2.connect(conn_string) + conn = psycopg2.connect(host='127.0.0.1', port=self.postgres_port, user='postgres', password='mysecretpassword') conn.close() print("Postgres Started") return @@ -738,7 +745,7 @@ class ClickHouseCluster: self.docker_client = docker.from_env(version=self.docker_api_version) - common_opts = ['up', '-d', '--force-recreate', '--remove-orphans'] + common_opts = ['up', '-d', '--force-recreate'] if self.with_zookeeper and self.base_zookeeper_cmd: print('Setup ZooKeeper') @@ -772,7 +779,7 @@ class ClickHouseCluster: if self.with_postgres and self.base_postgres_cmd: print('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) - self.wait_postgres_to_start(120) + self.wait_postgres_to_start(30) if self.with_kafka and self.base_kafka_cmd: print('Setup Kafka') @@ -826,6 +833,7 @@ class ClickHouseCluster: clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) + self.up_called = True subprocess.check_output(clickhouse_start_cmd) print("ClickHouse instance created") @@ -850,29 +858,30 @@ class ClickHouseCluster: def shutdown(self, kill=True): sanitizer_assert_instance = None - with open(self.docker_logs_path, "w+") as f: - try: - subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL - except Exception as e: - print("Unable to get logs from docker.") - f.seek(0) - for line in f: - if SANITIZER_SIGN in line: - sanitizer_assert_instance = line.split('|')[0].strip() - break + if self.up_called: + with open(self.docker_logs_path, "w+") as f: + try: + subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + except Exception as e: + print("Unable to get logs from docker.") + f.seek(0) + for line in f: + if SANITIZER_SIGN in line: + sanitizer_assert_instance = line.split('|')[0].strip() + break - if kill: - try: - subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) - except Exception as e: - print("Kill command failed during shutdown. {}".format(repr(e))) - print("Trying to kill forcefully") - subprocess_check_call(self.base_cmd + ['kill']) + if kill: + try: + subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) + except Exception as e: + print("Kill command failed during shutdown. {}".format(repr(e))) + print("Trying to kill forcefully") + subprocess_check_call(self.base_cmd + ['kill']) - try: - subprocess_check_call(self.base_cmd + ['down', '--volumes']) - except Exception as e: - print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) + try: + subprocess_check_call(self.base_cmd + ['down', '--volumes']) + except Exception as e: + print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) self.is_up = False @@ -1308,7 +1317,7 @@ class ClickHouseInstance: "UserName": "postgres", "Password": "mysecretpassword", "Port": "5432", - "Servername": "postgres1", + "Servername": self.cluster.postgres_host, "Protocol": "9.3", "ReadOnly": "No", "RowVersioning": "No", @@ -1441,8 +1450,6 @@ class ClickHouseInstance: self.cluster.env_variables.update(self.env_variables) _create_env_file(os.path.join(self.env_file), self.env_variables) - print("Env {} stored in {}".format(self.env_variables, self.env_file)) - odbc_ini_path = "" if self.odbc_ini_path: self._create_odbc_config_file() diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index b83c00409af..1be6ea6d295 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -18,11 +18,11 @@ click_dict_table_template = """ ) ENGINE = Dictionary({}) """ -def get_postgres_conn(database=False): +def get_postgres_conn(database, port): if database == True: - conn_string = "host='localhost' dbname='clickhouse' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' dbname='clickhouse' user='postgres' password='mysecretpassword'".format(port) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' user='postgres' password='mysecretpassword'".format(port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -36,8 +36,8 @@ def create_postgres_table(conn, table_name): cursor = conn.cursor() cursor.execute(postgres_dict_table_template.format(table_name)) -def create_and_fill_postgres_table(table_name): - conn = get_postgres_conn(True) +def create_and_fill_postgres_table(table_name, port): + conn = get_postgres_conn(True, port) create_postgres_table(conn, table_name) # Fill postgres table using clickhouse postgres table function and check table_func = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) @@ -54,7 +54,7 @@ def create_dict(table_name, index=0): def started_cluster(): try: cluster.start() - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(False, cluster.postgres_port) node1.query("CREATE DATABASE IF NOT EXISTS test") print("postgres connected") create_postgres_db(postgres_conn, 'clickhouse') @@ -65,10 +65,10 @@ def started_cluster(): def test_load_dictionaries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(True, started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(table_name) + create_and_fill_postgres_table(table_name, started_cluster.postgres_port) create_dict(table_name) dict_name = 'dict0' @@ -80,10 +80,10 @@ def test_load_dictionaries(started_cluster): def test_invalidate_query(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(True, started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(table_name) + create_and_fill_postgres_table(table_name, started_cluster.postgres_port) # invalidate query: SELECT value FROM test0 WHERE id = 0 dict_name = 'dict0' diff --git a/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml index 19eed6ebd6a..6aad3ad9917 100644 --- a/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml +++ b/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -4,7 +4,7 @@ clickhouse.test_table
- DSN=postgresql_odbc; + DSN=postgresql_odbc postgres
diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 3ef59e7bd23..7510ff02011 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -4,13 +4,14 @@ import psycopg2 import pymysql.cursors import pytest import logging +import os.path from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT cluster = ClickHouseCluster(__file__) -node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, +node1 = cluster.add_instance('node1', with_odbc_drivers=True, with_mysql=True, with_postgres=True, main_configs=['configs/openssl.xml', 'configs/odbc_logging.xml', 'configs/enable_dictionaries.xml', 'configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml', @@ -36,7 +37,7 @@ create_table_sql_template = """ def get_mysql_conn(): errors = [] conn = None - for _ in range(5): + for _ in range(15): try: if conn is None: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) @@ -64,11 +65,20 @@ def create_mysql_table(conn, table_name): def get_postgres_conn(): - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" - conn = psycopg2.connect(conn_string) - conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) - conn.autocommit = True - return conn + conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + errors = [] + for _ in range(15): + try: + conn = psycopg2.connect(conn_string) + logging.debug("Postgre Connection establised: {}".format(conn_string)) + conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) + conn.autocommit = True + return conn + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Postgre connection not establised DSN={}, {}".format(conn_string, errors)) def create_postgres_db(conn, name): @@ -319,12 +329,13 @@ def test_postgres_insert(started_cluster): "create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')") node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')") assert node1.query("select * from pg_insert") == '1\thello\n2\tworld\n' - node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') format CSV 3,test") + node1.query("insert into table function odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table') format CSV 3,test") node1.query( - "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)") + "insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')" \ + " select number, 's' || toString(number) from numbers (4, 7)") assert node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n" assert node1.query( - "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table')))") == "55\t10\n" + "select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc', 'clickhouse', 'test_table')))") == "55\t10\n" def test_bridge_dies_with_parent(started_cluster): diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index b3f7c0fa9af..2cb148db6ef 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -16,9 +16,9 @@ postgres_table_template = """ def get_postgres_conn(database=False): if database == True: - conn_string = "host='localhost' dbname='test_database' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port={} dbname='test_database' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index c63896eb196..633c02ef408 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -38,7 +38,7 @@ def server_address(): @pytest.fixture(scope='module') def psql_client(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_psql_1') @@ -62,7 +62,7 @@ def psql_server(psql_client): @pytest.fixture(scope='module') def java_container(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgesql_java_client.yml') + docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) yield docker.from_env().containers.get(cluster.project_name + '_java_1') diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 4f567c19f2b..e46d6f98aae 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -11,9 +11,9 @@ node1 = cluster.add_instance('node1', main_configs=[], with_postgres=True) def get_postgres_conn(database=False): if database == True: - conn_string = "host='localhost' dbname='clickhouse' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' dbname='clickhouse' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) else: - conn_string = "host='localhost' user='postgres' password='mysecretpassword'" + conn_string = "host='localhost' port='{}' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True From 44fddb2c295f83093f429d91d95e6f9a2b5d786e Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 18 Feb 2021 20:36:29 +0900 Subject: [PATCH 0026/1060] Add some comments --- .../parametric-functions.md | 4 ++-- .../AggregateFunctionSequenceNextNode.h | 21 ++++++++++++++++--- 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 1e84bf5e084..56fb5207648 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -503,14 +503,14 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, ... event32) +sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, event3, ...) ``` **Parameters** - `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. - `timestamp` — Name of the column containing the timestamp. Data types supported: Date, DateTime and other unsigned integer types. - `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: String and Nullable(String) -- `cond` — Conditions or data describing the chain of events. UInt8. +- `cond` — Conditions describing the chain of events. UInt8. **Returned value** diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index ffcc02b805a..be672974ac6 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,13 +27,15 @@ namespace DB { + +/// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template struct NodeBase { UInt64 size; // size of payload DataTypeDateTime::FieldType event_time; - UInt32 events_bitset; // UInt32 for combiniant comparesons between bitsets (< operator on bitsets). + UInt32 events_bitset; // Bitsets of UInt32 are easy to compare. (< operator on bitsets) char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -70,6 +72,7 @@ struct NodeBase } }; +/// It stores String, timestamp, bitset of matched events. struct NodeString : public NodeBase { using Node = NodeString; @@ -91,6 +94,7 @@ struct NodeString : public NodeBase } }; +/// TODO : Expends SequenceNextNodeGeneralData to support other types template struct SequenceNextNodeGeneralData { @@ -123,6 +127,7 @@ struct SequenceNextNodeGeneralData } }; +/// Implementation of sequenceNextNode template class SequenceNextNodeImpl final : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> @@ -178,6 +183,13 @@ public: const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; + /// The events_bitset variable stores matched events in the form of bitset. + /// It uses UInt32 instead of std::bitset because bitsets of UInt32 are easy to compare. (< operator on bitsets) + /// Each Nth-bit indicates that the Nth-event are matched. + /// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005. + /// 0x00000000 + /// + 1 (bit of event1) + /// + 4 (bit of event3) UInt32 events_bitset = 0; for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) @@ -265,8 +277,8 @@ public: return k; } - // This method returns an index of next node that matched the events. - // It is one as referring Boyer-Moore-Algorithm. + /// This method returns an index of next node that matched the events. + /// It is one as referring Boyer-Moore-Algorithm. UInt32 getNextNodeIndex(Data & data) const { if (data.value.size() <= events_size) @@ -278,10 +290,12 @@ public: while (i < data.value.size()) { UInt32 j = 0; + /// It checks whether the chain of events are matched or not. for (; j < events_size; ++j) if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) break; + /// If the chain of events are matched returns the index of result value. if (j == events_size) return i + 1; @@ -309,6 +323,7 @@ public: bool allocatesMemoryInArena() const override { return true; } }; +/// Implementation of sequenceFirstNode template class SequenceFirstNodeImpl final : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> From 3409aea6f6ba7cab2c203ce042d3ee9af50d7ae6 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 18 Feb 2021 21:18:44 +0900 Subject: [PATCH 0027/1060] Fix registerAggregateFunctions.cpp --- src/AggregateFunctions/registerAggregateFunctions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/registerAggregateFunctions.cpp b/src/AggregateFunctions/registerAggregateFunctions.cpp index 28b758aee2c..9bc951629c6 100644 --- a/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -100,9 +100,9 @@ void registerAggregateFunctions() registerAggregateFunctionAggThrow(factory); registerAggregateFunctionRankCorrelation(factory); registerAggregateFunctionMannWhitney(factory); + registerAggregateFunctionSequenceNextNode(factory); registerAggregateFunctionWelchTTest(factory); registerAggregateFunctionStudentTTest(factory); - registerAggregateFunctionSequenceNextNode(factory); } { From 872e36c207f5ed96f97883cb974038474160c84d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 18 Feb 2021 15:57:45 +0300 Subject: [PATCH 0028/1060] wip --- .../runner/compose/docker_compose_kafka.yml | 9 +- .../docker_compose_kerberized_kafka.yml | 6 +- tests/integration/helpers/cluster.py | 55 +++-- .../test_aggregate_function_state_avg.py | 2 +- .../test_short_strings_aggregation.py | 2 +- .../test_format_avro_confluent/test.py | 2 +- tests/integration/test_storage_kafka/test.py | 232 ++++++++++-------- .../test_storage_kerberized_kafka/test.py | 7 +- 8 files changed, 174 insertions(+), 141 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kafka.yml index 219d977ffd9..46f3383e25a 100644 --- a/docker/test/integration/runner/compose/docker_compose_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kafka.yml @@ -15,10 +15,11 @@ services: image: confluentinc/cp-kafka:5.2.0 hostname: kafka1 ports: - - "9092:9092" + - ${KAFKA_EXTERNAL_PORT}:${KAFKA_EXTERNAL_PORT} environment: - KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kafka1:19092 - KAFKA_LISTENERS: INSIDE://:9092,OUTSIDE://:19092 + KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:${KAFKA_EXTERNAL_PORT},OUTSIDE://kafka1:19092 + KAFKA_ADVERTISED_HOST_NAME: kafka1 + KAFKA_LISTENERS: INSIDE://0.0.0.0:${KAFKA_EXTERNAL_PORT},OUTSIDE://0.0.0.0:19092 KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: INSIDE:PLAINTEXT,OUTSIDE:PLAINTEXT KAFKA_INTER_BROKER_LISTENER_NAME: INSIDE KAFKA_BROKER_ID: 1 @@ -34,7 +35,7 @@ services: image: confluentinc/cp-schema-registry:5.2.0 hostname: schema-registry ports: - - "8081:8081" + - ${SCHEMA_REGISTRY_EXTERNAL_PORT}:${SCHEMA_REGISTRY_INTERNAL_PORT} environment: SCHEMA_REGISTRY_HOST_NAME: schema-registry SCHEMA_REGISTRY_KAFKASTORE_SECURITY_PROTOCOL: PLAINTEXT diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 6e1e11344bb..1c1ab837592 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -23,13 +23,13 @@ services: # restart: always hostname: kerberized_kafka1 ports: - - "9092:9092" - - "9093:9093" + - ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT} environment: KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 - KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:9093 + KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT} # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 + KAFKA_ADVERTISED_HOST_NAME: kerberized_kafka1 KAFKA_SASL_MECHANISM_INTER_BROKER_PROTOCOL: GSSAPI KAFKA_SASL_ENABLED_MECHANISMS: GSSAPI KAFKA_SASL_KERBEROS_SERVICE_NAME: kafka diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index d582c0b7401..0673deaa892 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -183,9 +183,19 @@ class ClickHouseCluster: self.minio_redirect_port = 8080 # available when with_kafka == True + self.kafka_host = "kafka1" + self.kafka_port = get_open_port() + self.kafka_docker_id = None self.schema_registry_client = None self.schema_registry_host = "schema-registry" - self.schema_registry_port = 8081 + self.schema_registry_port = get_open_port() + self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + + # available when with_kerberozed_kafka == True + self.kerberized_kafka_host = "kerberized_kafka1" + self.kerberized_kafka_port = get_open_port() + self.kerberized_kafka_docker_id = None + self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kafka_host) # available when with_mongo == True self.mongo_host = "mongo1" @@ -250,6 +260,27 @@ class ClickHouseCluster: return self.base_mysql8_cmd + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_redis = True + env_variables['KAFKA_HOST'] = self.kafka_host + env_variables['KAFKA_EXTERNAL_PORT'] = str(self.kafka_port) + env_variables['SCHEMA_REGISTRY_EXTERNAL_PORT'] = str(self.schema_registry_port) + env_variables['SCHEMA_REGISTRY_INTERNAL_PORT'] = "8081" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] + return self.base_redis_cmd + + def setup_kerberized_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_redis = True + env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' + env_variables['KERBERIZED_KAFKA_HOST'] = self.kerberized_kafka_host + env_variables['KERBERIZED_KAFKA_EXTERNAL_PORT'] = str(self.kerberized_kafka_port) + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) + self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] + return self.base_redis_cmd + def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True env_variables['REDIS_HOST'] = self.redis_host @@ -391,19 +422,10 @@ class ClickHouseCluster: cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kafka and not self.with_kafka: - self.with_kafka = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) - self.base_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] - cmds.append(self.base_kafka_cmd) + cmds.append(self.setup_kafka_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kerberized_kafka and not self.with_kerberized_kafka: - env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' - self.with_kerberized_kafka = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_kerberized_kafka_cmd = ['docker-compose', '--env-file', instance.env_file,'--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] - cmds.append(self.base_kerberized_kafka_cmd) + cmds.append(self.setup_kerberized_kafka_cmd(instance, env_variables, docker_compose_yml_dir)) if with_rabbitmq and not self.with_rabbitmq: cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -695,14 +717,13 @@ class ClickHouseCluster: raise Exception("Can't wait Minio to start") def wait_schema_registry_to_start(self, timeout=10): - sr_client = CachedSchemaRegistryClient('http://localhost:8081') + sr_client = CachedSchemaRegistryClient('http://localhost:{}'.format(cluster.schema_registry_port)) start = time.time() while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) - self.schema_registry_client = sr_client print("Connected to SchemaRegistry") - return + return sr_client except Exception as ex: print(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) @@ -784,13 +805,11 @@ class ClickHouseCluster: if self.with_kafka and self.base_kafka_cmd: print('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.kafka_docker_id = self.get_instance_docker_id('kafka1') - self.wait_schema_registry_to_start(120) + self.schema_registry_client = self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: print('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.kerberized_kafka_docker_id = self.get_instance_docker_id('kerberized_kafka1') if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') diff --git a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py index 3b35c112887..5ed97e7a9a5 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_function_state_avg.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="aggregate_state") node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True) diff --git a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py index 91a0a87b6e2..463fadc36e8 100644 --- a/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py +++ b/tests/integration/test_backward_compatibility/test_short_strings_aggregation.py @@ -2,7 +2,7 @@ import pytest from helpers.cluster import ClickHouseCluster -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="short_strings") node1 = cluster.add_instance('node1', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', stay_alive=True, with_installed_binary=True) node2 = cluster.add_instance('node2', with_zookeeper=False, image='yandex/clickhouse-server', tag='19.16.9.37', diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index cc006801735..ec2ed875075 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -65,7 +65,7 @@ def test_select(cluster): instance = cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( cluster.schema_registry_host, - cluster.schema_registry_port + 8081 ) run_query(instance, "create table avro_data(value Int64) engine = Memory()") diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 5f2726832cc..e1b5d1068a0 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -5,6 +5,7 @@ import socket import subprocess import threading import time +import logging import avro.schema from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient @@ -41,7 +42,7 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], with_kafka=True, - with_zookeeper=True, +# with_zookeeper=True, macros={"kafka_broker":"kafka1", "kafka_topic_old":"old", "kafka_group_name_old":"old", @@ -50,7 +51,7 @@ instance = cluster.add_instance('instance', "kafka_client_id":"instance", "kafka_format_json_each_row":"JSONEachRow"}, clickhouse_path_dir='clickhouse_path') -kafka_id = '' +kafka_id = cluster.kafka_docker_id # Helpers @@ -62,8 +63,8 @@ def check_kafka_is_available(): kafka_id, '/usr/bin/kafka-broker-api-versions', '--bootstrap-server', - 'INSIDE://localhost:9092'), - stdout=subprocess.PIPE) + 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), + stdout=subprocess.PIPE, stderr=subprocess.PIPE) p.communicate() return p.returncode == 0 @@ -76,21 +77,35 @@ def wait_kafka_is_available(max_retries=50): else: retries += 1 if retries > max_retries: - raise "Kafka is not available" - print("Waiting for Kafka to start up") + raise Exception("Kafka is not available") + logging.debug("Waiting for Kafka to start up") time.sleep(1) +def get_kafka_producer(port, serializer): + errors = [] + for _ in range(15): + try: + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) + logging.debug("Kafka Connection establised: localhost:{}".format(port)) + return producer + except Exception as e: + errors += [str(e)] + time.sleep(1) + + raise Exception("Connection not establised, {}".format(errors)) + def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) +def kafka_produce(kafka_cluster, topic, messages, timestamp=None): + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() -def kafka_consume(topic): - consumer = KafkaConsumer(bootstrap_servers="localhost:9092", auto_offset_reset="earliest") +def kafka_consume(kafka_cluster, topic): + consumer = KafkaConsumer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest") consumer.subscribe(topics=(topic)) for toppar, messages in list(consumer.poll(5000).items()): if toppar.topic == topic: @@ -100,7 +115,7 @@ def kafka_consume(topic): consumer.close() -def kafka_produce_protobuf_messages(topic, start_index, num_messages): +def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() @@ -108,14 +123,14 @@ def kafka_produce_protobuf_messages(topic, start_index, num_messages): msg.value = str(i) serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print(("Produced {} messages for topic {}".format(num_messages, topic))) + logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) -def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messages): +def kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, topic, start_index, num_messages): data = '' - producer = KafkaProducer(bootstrap_servers="localhost:9092") + producer = KafkaProducer(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() msg.key = i @@ -123,9 +138,9 @@ def kafka_produce_protobuf_messages_no_delimeters(topic, start_index, num_messag serialized_msg = msg.SerializeToString() producer.send(topic=topic, value=serialized_msg) producer.flush() - print("Produced {} messages for topic {}".format(num_messages, topic)) + logging.debug("Produced {} messages for topic {}".format(num_messages, topic)) -def kafka_produce_protobuf_social(topic, start_index, num_messages): +def kafka_produce_protobuf_social(kafka_cluster,topic, start_index, num_messages): data = b'' for i in range(start_index, start_index + num_messages): msg = social_pb2.User() @@ -133,10 +148,10 @@ def kafka_produce_protobuf_social(topic, start_index, num_messages): msg.timestamp=1000000+i serialized_msg = msg.SerializeToString() data = data + _VarintBytes(len(serialized_msg)) + serialized_msg - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), value_serializer=producer_serializer) producer.send(topic=topic, value=data) producer.flush() - print(("Produced {} messages for topic {}".format(num_messages, topic))) + logging.debug(("Produced {} messages for topic {}".format(num_messages, topic))) def avro_confluent_message(schema_registry_client, value): @@ -159,7 +174,7 @@ def avro_confluent_message(schema_registry_client, value): @pytest.mark.timeout(180) def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', + kafka_produce(kafka_cluster, 'kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) instance.query(''' @@ -180,11 +195,13 @@ def test_kafka_json_as_string(kafka_cluster): ''' assert TSV(result) == TSV(expected) assert instance.contains_in_log( - "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") + "Parsing of message (topic: kafka_json_as_string, partition: 0, offset: [0-9]*) return no rows") @pytest.mark.timeout(300) def test_kafka_formats(kafka_cluster): + schema_registry_client = CachedSchemaRegistryClient('http://localhost:{}'.format(kafka_cluster.schema_registry_port)) + # data was dumped from clickhouse itself in a following manner # clickhouse-client --format=Native --query='SELECT toInt64(number) as id, toUInt16( intDiv( id, 65536 ) ) as blockNo, reinterpretAsString(19777) as val1, toFloat32(0.5) as val2, toUInt8(1) as val3 from numbers(100) ORDER BY id' | xxd -ps | tr -d '\n' | sed 's/\(..\)/\\x\1/g' @@ -453,19 +470,19 @@ def test_kafka_formats(kafka_cluster): # }, 'AvroConfluent': { 'data_sample': [ - avro_confluent_message(cluster.schema_registry_client, + avro_confluent_message(schema_registry_client, {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), - b''.join([avro_confluent_message(cluster.schema_registry_client, + b''.join([avro_confluent_message(schema_registry_client, {'id': id, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}) for id in range(1, 16)]), - avro_confluent_message(cluster.schema_registry_client, + avro_confluent_message(schema_registry_client, {'id': 0, 'blockNo': 0, 'val1': str('AM'), 'val2': 0.5, "val3": 1}), ], 'extra_settings': ", format_avro_schema_registry_url='http://{}:{}'".format( - cluster.schema_registry_host, - cluster.schema_registry_port + kafka_cluster.schema_registry_host, + 8081 ), 'supports_empty_value': True, } @@ -514,14 +531,14 @@ def test_kafka_formats(kafka_cluster): } for format_name, format_opts in list(all_formats.items()): - print(('Set up {}'.format(format_name))) + logging.debug(('Set up {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) data_sample = format_opts['data_sample'] data_prefix = [] # prepend empty value when supported if format_opts.get('supports_empty_value', False): data_prefix = data_prefix + [''] - kafka_produce(topic_name, data_prefix + data_sample) + kafka_produce(kafka_cluster, topic_name, data_prefix + data_sample) instance.query(''' DROP TABLE IF EXISTS test.kafka_{format_name}; @@ -548,7 +565,7 @@ def test_kafka_formats(kafka_cluster): time.sleep(12) for format_name, format_opts in list(all_formats.items()): - print(('Checking {}'.format(format_name))) + logging.debug(('Checking {}'.format(format_name))) topic_name = 'format_tests_{}'.format(format_name) # shift offsets by 1 if format supports empty value offsets = [1, 2, 3] if format_opts.get('supports_empty_value', False) else [0, 1, 2] @@ -587,8 +604,8 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' # https://stackoverflow.com/a/57692111/1555175 -def describe_consumer_group(name): - client = BrokerConnection('localhost', 9092, socket.AF_INET) +def describe_consumer_group(kafka_cluster, name): + client = BrokerConnection('localhost', kafka_cluster.kafka_port, socket.AF_INET) client.connect_blocking() list_members_in_groups = DescribeGroupsRequest_v1(groups=[name]) @@ -619,10 +636,7 @@ def describe_consumer_group(name): @pytest.fixture(scope="module") def kafka_cluster(): try: - global kafka_id cluster.start() - kafka_id = instance.cluster.kafka_docker_id - print(("kafka_id is {}".format(kafka_id))) yield cluster finally: @@ -633,7 +647,7 @@ def kafka_cluster(): def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') wait_kafka_is_available() - # print("kafka is available - running test") + # logging.debug("kafka is available - running test") yield # run test @@ -661,7 +675,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('old', messages) + kafka_produce(kafka_cluster, 'old', messages) result = '' while True: @@ -671,9 +685,9 @@ kafka_topic_old old kafka_check_result(result, True) - members = describe_consumer_group('old') + members = describe_consumer_group(kafka_cluster, 'old') assert members[0]['client_id'] == 'ClickHouse-instance-test-kafka' - # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:9092 --describe --members --group old --verbose")) + # text_desc = kafka_cluster.exec_in_container(kafka_cluster.get_container_id('kafka1'),"kafka-consumer-groups --bootstrap-server localhost:{} --describe --members --group old --verbose".format(cluster.kafka_port))) @pytest.mark.timeout(180) @@ -693,16 +707,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) # Insert couple of malformed messages. - kafka_produce('new', ['}{very_broken_message,']) - kafka_produce('new', ['}another{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) result = '' while True: @@ -712,14 +726,14 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_check_result(result, True) - members = describe_consumer_group('new') + members = describe_consumer_group(kafka_cluster, 'new') assert members[0]['client_id'] == 'instance test 1234' @pytest.mark.timeout(180) def test_kafka_issue11308(kafka_cluster): # Check that matview does respect Kafka SETTINGS - kafka_produce('issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', + kafka_produce(kafka_cluster, 'issue11308', ['{"t": 123, "e": {"x": "woof"} }', '{"t": 123, "e": {"x": "woof"} }', '{"t": 124, "e": {"x": "test"} }']) instance.query(''' @@ -768,7 +782,7 @@ def test_kafka_issue11308(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_issue4116(kafka_cluster): # Check that format_csv_delimiter parameter works now - as part of all available format settings. - kafka_produce('issue4116', ['1|foo', '2|bar', '42|answer', '100|multi\n101|row\n103|message']) + kafka_produce(kafka_cluster, 'issue4116', ['1|foo', '2|bar', '42|answer', '100|multi\n101|row\n103|message']) instance.query(''' CREATE TABLE test.kafka (a UInt64, b String) @@ -823,7 +837,7 @@ def test_kafka_consumer_hang(kafka_cluster): time.sleep(0.5) kafka_cluster.unpause_container('kafka1') - # print("Attempt to drop") + # logging.debug("Attempt to drop") instance.query('DROP TABLE test.kafka') # kafka_cluster.open_bash_shell('instance') @@ -899,7 +913,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) result = '' while True: @@ -925,7 +939,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) - kafka_produce('tsv', messages) + kafka_produce(kafka_cluster, 'tsv', messages) result = '' while True: @@ -965,12 +979,12 @@ def test_kafka_json_without_delimiter(kafka_cluster): messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('json', [messages]) + kafka_produce(kafka_cluster, 'json', [messages]) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('json', [messages]) + kafka_produce(kafka_cluster, 'json', [messages]) result = '' while True: @@ -993,9 +1007,9 @@ def test_kafka_protobuf(kafka_cluster): kafka_schema = 'kafka.proto:KeyValuePair'; ''') - kafka_produce_protobuf_messages('pb', 0, 20) - kafka_produce_protobuf_messages('pb', 20, 1) - kafka_produce_protobuf_messages('pb', 21, 29) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 0, 20) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 20, 1) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 21, 29) result = '' while True: @@ -1025,9 +1039,9 @@ SETTINGS SELECT * FROM test.kafka; ''') - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 21, 29) result = instance.query('SELECT * FROM test.kafka', ignore_error=True) expected = '''\ @@ -1096,9 +1110,9 @@ def test_kafka_protobuf_no_delimiter(kafka_cluster): kafka_schema = 'kafka.proto:KeyValuePair'; ''') - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 0, 20) - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 20, 1) - kafka_produce_protobuf_messages_no_delimeters('pb_no_delimiter', 21, 29) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 0, 20) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 20, 1) + kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, 'pb_no_delimiter', 21, 29) result = '' while True: @@ -1155,7 +1169,7 @@ def test_kafka_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mv', messages) + kafka_produce(kafka_cluster, 'mv', messages) while True: result = instance.query('SELECT * FROM test.view') @@ -1184,7 +1198,7 @@ def test_librdkafka_snappy_regression(kafka_cluster): """ # create topic with snappy compression - admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:9092'}) + admin_client = admin.AdminClient({'bootstrap.servers': 'localhost:{}'.format(kafka_cluster.kafka_port)}) topic_snappy = admin.NewTopic(topic='snappy_regression', num_partitions=1, replication_factor=1, config={ 'compression.type': 'snappy', }) @@ -1215,14 +1229,14 @@ def test_librdkafka_snappy_regression(kafka_cluster): for i in range(number_of_messages): messages.append(json.dumps({'key': i, 'value': value})) expected.append(f'{i}\t{value}') - kafka_produce('snappy_regression', messages) + kafka_produce(kafka_cluster, 'snappy_regression', messages) expected = '\n'.join(expected) while True: result = instance.query('SELECT * FROM test.kafka') rows = len(result.strip('\n').split('\n')) - print(rows) + logging.debug(rows) if rows == number_of_messages: break @@ -1252,7 +1266,7 @@ def test_kafka_materialized_view_with_subquery(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mvsq', messages) + kafka_produce(kafka_cluster, 'mvsq', messages) while True: result = instance.query('SELECT * FROM test.view') @@ -1296,7 +1310,7 @@ def test_kafka_many_materialized_views(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('mmv', messages) + kafka_produce(kafka_cluster, 'mmv', messages) while True: result1 = instance.query('SELECT * FROM test.view1') @@ -1321,7 +1335,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): kafka_messages = 1000 batch_messages = 1000 messages = [json.dumps({'key': i, 'value': 'x' * 100}) * batch_messages for i in range(kafka_messages)] - kafka_produce('flush', messages) + kafka_produce(kafka_cluster, 'flush', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -1340,7 +1354,7 @@ def test_kafka_flush_on_big_message(kafka_cluster): SELECT * FROM test.kafka; ''') - client = KafkaAdminClient(bootstrap_servers="localhost:9092") + client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) received = False while not received: try: @@ -1379,12 +1393,12 @@ def test_kafka_virtual_columns(kafka_cluster): messages = '' for i in range(25): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('virt1', [messages], 0) + kafka_produce(kafka_cluster, 'virt1', [messages], 0) messages = '' for i in range(25, 50): messages += json.dumps({'key': i, 'value': i}) + '\n' - kafka_produce('virt1', [messages], 0) + kafka_produce(kafka_cluster, 'virt1', [messages], 0) result = '' while True: @@ -1419,7 +1433,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster): messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('virt2', messages, 0) + kafka_produce(kafka_cluster, 'virt2', messages, 0) while True: result = instance.query('SELECT kafka_key, key, topic, value, offset, partition, timestamp FROM test.view') @@ -1563,7 +1577,7 @@ def test_kafka_commit_on_block_write(kafka_cluster): for _ in range(101): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - kafka_produce('block', messages) + kafka_produce(kafka_cluster, 'block', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() @@ -1608,7 +1622,7 @@ def test_kafka_commit_on_block_write(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_virtual_columns2(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="virt2_0", num_partitions=2, replication_factor=1)) topic_list.append(NewTopic(name="virt2_1", num_partitions=2, replication_factor=1)) @@ -1628,7 +1642,7 @@ def test_kafka_virtual_columns2(kafka_cluster): SELECT value, _key, _topic, _partition, _offset, toUnixTimestamp(_timestamp), toUnixTimestamp64Milli(_timestamp_ms), _headers.name, _headers.value FROM test.kafka; ''') - producer = KafkaProducer(bootstrap_servers="localhost:9092", value_serializer=producer_serializer, key_serializer=producer_serializer) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(cluster.kafka_port), value_serializer=producer_serializer, key_serializer=producer_serializer) producer.send(topic='virt2_0', value=json.dumps({'value': 1}), partition=0, key='k1', timestamp_ms=1577836801001, headers=[('content-encoding', b'base64')]) @@ -1655,7 +1669,7 @@ def test_kafka_virtual_columns2(kafka_cluster): time.sleep(10) - members = describe_consumer_group('virt2') + members = describe_consumer_group(kafka_cluster, 'virt2') # pprint.pprint(members) members[0]['client_id'] = 'ClickHouse-instance-test-kafka-0' members[1]['client_id'] = 'ClickHouse-instance-test-kafka-1' @@ -1679,7 +1693,7 @@ def test_kafka_virtual_columns2(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_produce_key_timestamp(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="insert3", num_partitions=1, replication_factor=1)) @@ -1724,7 +1738,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): result = instance.query("SELECT * FROM test.view ORDER BY value", ignore_error=True) - # print(result) + # logging.debug(result) expected = '''\ 1 1 k1 1577836801 k1 insert3 0 0 1577836801 @@ -1739,7 +1753,7 @@ def test_kafka_produce_key_timestamp(kafka_cluster): @pytest.mark.timeout(600) def test_kafka_flush_by_time(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="flush_by_time", num_partitions=1, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1770,7 +1784,7 @@ def test_kafka_flush_by_time(kafka_cluster): while not cancel.is_set(): messages = [] messages.append(json.dumps({'key': 0, 'value': 0})) - kafka_produce('flush_by_time', messages) + kafka_produce(kafka_cluster, 'flush_by_time', messages) time.sleep(0.8) kafka_thread = threading.Thread(target=produce) @@ -1806,7 +1820,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): while not cancel.is_set(): messages = [] messages.append(json.dumps({'key': 0, 'value': 0})) - kafka_produce('flush_by_block_size', messages) + kafka_produce(kafka_cluster, 'flush_by_block_size', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() @@ -1844,7 +1858,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): # more flushes can happens during test, we need to check only result of first flush (part named all_1_1_0). result = instance.query("SELECT count() FROM test.view WHERE _part='all_1_1_0'") - # print(result) + # logging.debug(result) instance.query(''' DROP TABLE test.consumer; @@ -1859,7 +1873,7 @@ def test_kafka_flush_by_block_size(kafka_cluster): @pytest.mark.timeout(600) def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions2", num_partitions=10, replication_factor=1)) @@ -1890,12 +1904,12 @@ def test_kafka_lot_of_partitions_partial_commit_of_bulk(kafka_cluster): count = count + 1 rows.append(json.dumps({'key': count, 'value': count})) messages.append("\n".join(rows)) - kafka_produce('topic_with_multiple_partitions2', messages) + kafka_produce(kafka_cluster, 'topic_with_multiple_partitions2', messages) time.sleep(30) result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) assert TSV(result) == TSV('{0}\t{0}\t{0}'.format(count)) instance.query(''' @@ -1928,7 +1942,7 @@ def test_kafka_rebalance(kafka_cluster): # time.sleep(2) - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions", num_partitions=11, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1943,14 +1957,14 @@ def test_kafka_rebalance(kafka_cluster): for _ in range(59): messages.append(json.dumps({'key': msg_index[0], 'value': msg_index[0]})) msg_index[0] += 1 - kafka_produce('topic_with_multiple_partitions', messages) + kafka_produce(kafka_cluster, 'topic_with_multiple_partitions', messages) kafka_thread = threading.Thread(target=produce) kafka_thread.start() for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): table_name = 'kafka_consumer{}'.format(consumer_index) - print(("Setting up {}".format(table_name))) + logging.debug(("Setting up {}".format(table_name))) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -1977,21 +1991,21 @@ def test_kafka_rebalance(kafka_cluster): # kafka_cluster.open_bash_shell('instance') while int( instance.query("SELECT count() FROM test.destination WHERE _consumed_by='{}'".format(table_name))) == 0: - print(("Waiting for test.kafka_consumer{} to start consume".format(consumer_index))) + logging.debug(("Waiting for test.kafka_consumer{} to start consume".format(consumer_index))) time.sleep(1) cancel.set() # I leave last one working by intent (to finish consuming after all rebalances) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS - 1): - print(("Dropping test.kafka_consumer{}".format(consumer_index))) + logging.debug(("Dropping test.kafka_consumer{}".format(consumer_index))) instance.query('DROP TABLE IF EXISTS test.kafka_consumer{}'.format(consumer_index)) while int(instance.query( "SELECT count() FROM system.tables WHERE database='test' AND name='kafka_consumer{}'".format( consumer_index))) == 1: time.sleep(1) - # print(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) + # logging.debug(instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination')) # kafka_cluster.open_bash_shell('instance') while 1: @@ -1999,9 +2013,9 @@ def test_kafka_rebalance(kafka_cluster): if messages_consumed >= msg_index[0]: break time.sleep(1) - print(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) + logging.debug(("Waiting for finishing consuming (have {}, should be {})".format(messages_consumed, msg_index[0]))) - print((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) + logging.debug((instance.query('SELECT count(), uniqExact(key), max(key) + 1 FROM test.destination'))) # Some queries to debug... # SELECT * FROM test.destination where key in (SELECT key FROM test.destination group by key having count() <> 1) @@ -2026,7 +2040,7 @@ def test_kafka_rebalance(kafka_cluster): result = int(instance.query('SELECT count() == uniqExact(key) FROM test.destination')) for consumer_index in range(NUMBER_OF_CONSURRENT_CONSUMERS): - print(("kafka_consumer{}".format(consumer_index))) + logging.debug(("kafka_consumer{}".format(consumer_index))) table_name = 'kafka_consumer{}'.format(consumer_index) instance.query(''' DROP TABLE IF EXISTS test.{0}; @@ -2045,7 +2059,7 @@ def test_kafka_rebalance(kafka_cluster): @pytest.mark.timeout(1200) def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] - kafka_produce('no_holes_when_write_suffix_failed', messages) + kafka_produce(kafka_cluster, 'no_holes_when_write_suffix_failed', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -2064,7 +2078,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): ''') messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('no_holes_when_write_suffix_failed', messages) + kafka_produce(kafka_cluster, 'no_holes_when_write_suffix_failed', messages) # init PartitionManager (it starts container) earlier pm = PartitionManager() @@ -2095,7 +2109,7 @@ def test_kafka_no_holes_when_write_suffix_failed(kafka_cluster): assert instance.contains_in_log("ZooKeeper session has been expired.: while write prefix to view") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) # kafka_cluster.open_bash_shell('instance') @@ -2143,7 +2157,7 @@ def test_exception_from_destructor(kafka_cluster): @pytest.mark.timeout(120) def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(1)] - kafka_produce('commits_of_unprocessed_messages_on_drop', messages) + kafka_produce(kafka_cluster, 'commits_of_unprocessed_messages_on_drop', messages) instance.query(''' DROP TABLE IF EXISTS test.destination; @@ -2181,7 +2195,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): ''') while int(instance.query("SELECT count() FROM test.destination")) == 0: - print("Waiting for test.kafka_consumer to start consume") + logging.debug("Waiting for test.kafka_consumer to start consume") time.sleep(1) cancel = threading.Event() @@ -2194,7 +2208,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): for _ in range(113): messages.append(json.dumps({'key': i[0], 'value': i[0]})) i[0] += 1 - kafka_produce('commits_of_unprocessed_messages_on_drop', messages) + kafka_produce(kafka_cluster, 'commits_of_unprocessed_messages_on_drop', messages) time.sleep(1) kafka_thread = threading.Thread(target=produce) @@ -2222,7 +2236,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): # SELECT key, _timestamp, _offset FROM test.destination where runningDifference(key) <> 1 ORDER BY key; result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.destination') - print(result) + logging.debug(result) instance.query(''' DROP TABLE test.kafka_consumer; @@ -2236,7 +2250,7 @@ def test_commits_of_unprocessed_messages_on_drop(kafka_cluster): @pytest.mark.timeout(120) def test_bad_reschedule(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce('test_bad_reschedule', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -2261,7 +2275,7 @@ def test_bad_reschedule(kafka_cluster): ''') while int(instance.query("SELECT count() FROM test.destination")) < 20000: - print("Waiting for consume") + logging.debug("Waiting for consume") time.sleep(1) assert int(instance.query("SELECT max(consume_ts) - min(consume_ts) FROM test.destination")) < 8 @@ -2270,7 +2284,7 @@ def test_bad_reschedule(kafka_cluster): @pytest.mark.timeout(300) def test_kafka_duplicates_when_commit_failed(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(1)] - kafka_produce('duplicates_when_commit_failed', messages) + kafka_produce(kafka_cluster, 'duplicates_when_commit_failed', messages) instance.query(''' DROP TABLE IF EXISTS test.view; @@ -2289,7 +2303,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): ''') messages = [json.dumps({'key': j + 1, 'value': 'x' * 300}) for j in range(22)] - kafka_produce('duplicates_when_commit_failed', messages) + kafka_produce(kafka_cluster, 'duplicates_when_commit_failed', messages) instance.query(''' CREATE TABLE test.view (key UInt64, value String) @@ -2325,7 +2339,7 @@ def test_kafka_duplicates_when_commit_failed(kafka_cluster): assert instance.contains_in_log("All commit attempts failed") result = instance.query('SELECT count(), uniqExact(key), max(key) FROM test.view') - print(result) + logging.debug(result) instance.query(''' DROP TABLE test.consumer; @@ -2374,7 +2388,7 @@ def test_premature_flush_on_eof(kafka_cluster): # next poll can return more data, and we should respect kafka_flush_interval_ms # and try to form bigger block messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(1)] - kafka_produce('premature_flush_on_eof', messages) + kafka_produce(kafka_cluster, 'premature_flush_on_eof', messages) instance.query(''' CREATE MATERIALIZED VIEW test.kafka_consumer TO test.destination AS @@ -2397,7 +2411,7 @@ def test_premature_flush_on_eof(kafka_cluster): # TODO: wait for messages in log: "Polled batch of 1 messages", followed by "Stalled" # produce more messages after delay - kafka_produce('premature_flush_on_eof', messages) + kafka_produce(kafka_cluster, 'premature_flush_on_eof', messages) # data was not flushed yet (it will be flushed 7.5 sec after creating MV) assert int(instance.query("SELECT count() FROM test.destination")) == 0 @@ -2417,7 +2431,7 @@ def test_premature_flush_on_eof(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_unavailable(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce('test_bad_reschedule', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) kafka_cluster.pause_container('kafka1') @@ -2451,7 +2465,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.unpause_container('kafka1') while int(instance.query("SELECT count() FROM test.destination")) < 20000: - print("Waiting for consume") + logging.debug("Waiting for consume") time.sleep(1) @@ -2503,7 +2517,7 @@ def test_kafka_csv_with_thread_per_consumer(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) result = '' while True: diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index 865afc8b162..accb855af10 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -24,7 +24,7 @@ instance = cluster.add_instance('instance', with_kerberized_kafka=True, clickhouse_path_dir="clickhouse_path" ) -kafka_id = '' # instance.cluster.kafka_docker_id +kafka_id = cluster.kerberized_kafka_docker_id # Helpers @@ -51,13 +51,14 @@ def wait_kafka_is_available(max_retries=50): else: retries += 1 if retries > max_retries: - raise "Kafka is not available" + raise Exception("Kafka is not available") print("Waiting for Kafka to start up") time.sleep(1) def producer_serializer(x): return x.encode() if isinstance(x, str) else x + def kafka_produce(topic, messages, timestamp=None): producer = KafkaProducer(bootstrap_servers="localhost:9093", value_serializer=producer_serializer) for message in messages: @@ -72,9 +73,7 @@ def kafka_produce(topic, messages, timestamp=None): @pytest.fixture(scope="module") def kafka_cluster(): try: - global kafka_id cluster.start() - kafka_id = instance.cluster.kerberized_kafka_docker_id print("kafka_id is {}".format(kafka_id)) yield cluster From 609c9b272fc3475df14828f8e5010e42019c9602 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 18 Feb 2021 16:42:31 +0300 Subject: [PATCH 0029/1060] fix --- tests/integration/helpers/cluster.py | 3 +-- tests/integration/test_storage_kafka/test.py | 8 ++++---- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0673deaa892..7409945d019 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -186,7 +186,6 @@ class ClickHouseCluster: self.kafka_host = "kafka1" self.kafka_port = get_open_port() self.kafka_docker_id = None - self.schema_registry_client = None self.schema_registry_host = "schema-registry" self.schema_registry_port = get_open_port() self.kafka_docker_id = self.get_instance_docker_id(self.kafka_host) @@ -805,7 +804,7 @@ class ClickHouseCluster: if self.with_kafka and self.base_kafka_cmd: print('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) - self.schema_registry_client = self.wait_schema_registry_to_start(30) + self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: print('Setup kerberized kafka') diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index e1b5d1068a0..24afbc8ca4e 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -42,7 +42,7 @@ cluster = ClickHouseCluster(__file__) instance = cluster.add_instance('instance', main_configs=['configs/kafka.xml', 'configs/log_conf.xml'], with_kafka=True, -# with_zookeeper=True, + with_zookeeper=True, # For Replicated Table macros={"kafka_broker":"kafka1", "kafka_topic_old":"old", "kafka_group_name_old":"old", @@ -130,7 +130,7 @@ def kafka_produce_protobuf_messages(kafka_cluster, topic, start_index, num_messa def kafka_produce_protobuf_messages_no_delimeters(kafka_cluster, topic, start_index, num_messages): data = '' - producer = KafkaProducer(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) for i in range(start_index, start_index + num_messages): msg = kafka_pb2.KeyValuePair() msg.key = i @@ -1477,7 +1477,7 @@ def test_kafka_insert(kafka_cluster): messages = [] while True: - messages.extend(kafka_consume('insert1')) + messages.extend(kafka_consume(kafka_cluster, 'insert1')) if len(messages) == 50: break @@ -1942,7 +1942,7 @@ def test_kafka_rebalance(kafka_cluster): # time.sleep(2) - admin_client = KafkaAdminClient(bootstrap_servers="localhost:".format(kafka_cluster.kafka_port)) + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="topic_with_multiple_partitions", num_partitions=11, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) From 2df4317abad43a6408f4e32cbff298f32df73ebb Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 00:21:50 +0300 Subject: [PATCH 0030/1060] wip --- .../runner/compose/docker_compose_hdfs.yml | 8 +- .../docker_compose_kerberized_kafka.yml | 2 +- tests/integration/helpers/cluster.py | 214 ++++++++++-------- tests/integration/pytest.ini | 7 + tests/integration/test_storage_hdfs/test.py | 6 - tests/integration/test_storage_kafka/test.py | 1 - .../test_storage_kerberized_kafka/test.py | 28 ++- .../integration/test_storage_mongodb/test.py | 12 +- 8 files changed, 157 insertions(+), 121 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_hdfs.yml index b8cd7f64273..96384b0a7d4 100644 --- a/docker/test/integration/runner/compose/docker_compose_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_hdfs.yml @@ -5,6 +5,10 @@ services: hostname: hdfs1 restart: always ports: - - 50075:50075 - - 50070:50070 + - ${HDFS_NAME_EXTERNAL_PORT}:${HDFS_NAME_INTERNAL_PORT} #50070 + - ${HDFS_DATA_EXTERNAL_PORT}:${HDFS_DATA_INTERNAL_PORT} #50075 entrypoint: /etc/bootstrap.sh -d + volumes: + - type: ${HDFS_FS:-tmpfs} + source: ${HDFS_LOGS:-} + target: /usr/local/hadoop/logs \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml index 1c1ab837592..710f9dacf48 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_kafka.yml @@ -25,7 +25,7 @@ services: ports: - ${KERBERIZED_KAFKA_EXTERNAL_PORT}:${KERBERIZED_KAFKA_EXTERNAL_PORT} environment: - KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://:9093 + KAFKA_LISTENERS: OUTSIDE://:19092,UNSECURED_OUTSIDE://:19093,UNSECURED_INSIDE://0.0.0.0:${KERBERIZED_KAFKA_EXTERNAL_PORT} KAFKA_ADVERTISED_LISTENERS: OUTSIDE://kerberized_kafka1:19092,UNSECURED_OUTSIDE://kerberized_kafka1:19093,UNSECURED_INSIDE://localhost:${KERBERIZED_KAFKA_EXTERNAL_PORT} # KAFKA_LISTENERS: INSIDE://kerberized_kafka1:9092,OUTSIDE://kerberized_kafka1:19092 # KAFKA_ADVERTISED_LISTENERS: INSIDE://localhost:9092,OUTSIDE://kerberized_kafka1:19092 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 7409945d019..cb5c25c260e 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -55,9 +55,9 @@ def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually - print('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) - print('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) - print('Env:\n{}\n'.format(env)) + logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) + logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) + logging.debug('Env:\n{}\n'.format(env)) raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 @@ -72,13 +72,13 @@ def get_open_port(): def subprocess_check_call(args): # Uncomment for debugging - print('run:', ' '.join(args)) + logging.info('run:' + ' '.join(args)) run_and_check(args) def subprocess_call(args): # Uncomment for debugging..; - # print('run:', ' ' . join(args)) + # logging.debug('run:', ' ' . join(args)) subprocess.call(args) def get_odbc_bridge_path(): @@ -100,7 +100,7 @@ def get_docker_compose_path(): if os.path.exists(os.path.dirname('/compose/')): return os.path.dirname('/compose/') # default in docker runner container else: - print(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) + logging.debug(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) return LOCAL_DOCKER_COMPOSE_DIR @@ -116,7 +116,7 @@ class ClickHouseCluster: def __init__(self, base_path, name=None, base_config_dir=None, server_bin_path=None, client_bin_path=None, odbc_bridge_bin_path=None, zookeeper_config_path=None, custom_dockerd_host=None): for param in list(os.environ.keys()): - print("ENV %40s %s" % (param, os.environ[param])) + logging.debug("ENV %40s %s" % (param, os.environ[param])) self.base_dir = p.dirname(base_path) self.name = name if name is not None else '' @@ -155,6 +155,7 @@ class ClickHouseCluster: self.base_kerberized_kafka_cmd = [] self.base_rabbitmq_cmd = [] self.base_cassandra_cmd = [] + self.base_redis_cmd = [] self.pre_zookeeper_commands = [] self.instances = {} self.with_zookeeper = False @@ -182,6 +183,13 @@ class ClickHouseCluster: self.minio_redirect_host = "proxy1" self.minio_redirect_port = 8080 + # available when with_hdfs == True + self.hdfs_host = "hdfs1" + self.hdfs_name_port = get_open_port() + self.hdfs_data_port = get_open_port() + self.hdfs_dir = p.abspath(p.join(self.instances_dir, "hdfs")) + self.hdfs_logs_dir = os.path.join(self.hdfs_dir, "logs") + # available when with_kafka == True self.kafka_host = "kafka1" self.kafka_port = get_open_port() @@ -194,7 +202,7 @@ class ClickHouseCluster: self.kerberized_kafka_host = "kerberized_kafka1" self.kerberized_kafka_port = get_open_port() self.kerberized_kafka_docker_id = None - self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kafka_host) + self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kerberized_kafka_host) # available when with_mongo == True self.mongo_host = "mongo1" @@ -229,7 +237,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - print("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) + logging.debug("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) def get_client_cmd(self): cmd = self.client_bin_path @@ -259,6 +267,21 @@ class ClickHouseCluster: return self.base_mysql8_cmd + def setup_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_hdfs = True + env_variables['HDFS_HOST'] = self.hdfs_host + env_variables['HDFS_NAME_EXTERNAL_PORT'] = str(self.hdfs_name_port) + env_variables['HDFS_NAME_INTERNAL_PORT'] = "50070" + env_variables['HDFS_DATA_EXTERNAL_PORT'] = str(self.hdfs_data_port) + env_variables['HDFS_DATA_INTERNAL_PORT'] = "50075" + env_variables['HDFS_LOGS'] = self.hdfs_logs_dir + env_variables['HDFS_FS'] = "bind" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) + self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] + print("HDFS BASE CMD:{}".format(self.base_hdfs_cmd)) + return self.base_hdfs_cmd + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True env_variables['KAFKA_HOST'] = self.kafka_host @@ -266,9 +289,9 @@ class ClickHouseCluster: env_variables['SCHEMA_REGISTRY_EXTERNAL_PORT'] = str(self.schema_registry_port) env_variables['SCHEMA_REGISTRY_INTERNAL_PORT'] = "8081" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')]) - self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + self.base_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kafka.yml')] - return self.base_redis_cmd + return self.base_kafka_cmd def setup_kerberized_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True @@ -276,9 +299,9 @@ class ClickHouseCluster: env_variables['KERBERIZED_KAFKA_HOST'] = self.kerberized_kafka_host env_variables['KERBERIZED_KAFKA_EXTERNAL_PORT'] = str(self.kerberized_kafka_port) self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')]) - self.base_redis_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + self.base_kerberized_kafka_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_kafka.yml')] - return self.base_redis_cmd + return self.base_kerberized_kafka_cmd def setup_redis_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True @@ -314,6 +337,16 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] return self.base_postgres_cmd + def setup_mongo_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_mongo = True + env_variables['MONGO_HOST'] = self.mongo_host + env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) + env_variables['MONGO_INTERNAL_PORT'] = "27017" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) + self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] + return self.base_mongo_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -430,11 +463,7 @@ class ClickHouseCluster: cmds.append(self.setup_rabbitmq_cmd(instance, env_variables, docker_compose_yml_dir)) if with_hdfs and not self.with_hdfs: - self.with_hdfs = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')]) - self.base_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_hdfs.yml')] - cmds.append(self.base_hdfs_cmd) + cmds.append(self.setup_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kerberized_hdfs and not self.with_kerberized_hdfs: self.with_kerberized_hdfs = True @@ -445,14 +474,7 @@ class ClickHouseCluster: cmds.append(self.base_kerberized_hdfs_cmd) if with_mongo and not self.with_mongo: - self.with_mongo = True - env_variables['MONGO_HOST'] = self.mongo_host - env_variables['MONGO_EXTERNAL_PORT'] = str(self.mongo_port) - env_variables['MONGO_INTERNAL_PORT'] = "27017" - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')]) - self.base_mongo_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] - cmds.append(self.base_mongo_cmd) + cmds.append(self.setup_mongo_cmd(instance, env_variables, docker_compose_yml_dir)) if self.with_net_trics: for cmd in cmds: @@ -486,7 +508,7 @@ class ClickHouseCluster: self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] - print("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( + logging.debug("Cluster name:{} project_name:{}. Added instance name:{} tag:{} base_cmd:{} docker_compose_yml_dir:{}".format( self.name, self.project_name, name, tag, self.base_cmd, docker_compose_yml_dir)) return instance @@ -525,10 +547,10 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["restart", service_name]) def get_instance_ip(self, instance_name): - print("get_instance_ip instance_name={}".format(instance_name)) + logging.debug("get_instance_ip instance_name={}".format(instance_name)) docker_id = self.get_instance_docker_id(instance_name) # for cont in self.docker_client.containers.list(): - # print("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) + # logging.debug("CONTAINERS LIST: ID={} NAME={} STATUS={}".format(cont.id, cont.name, cont.status)) handle = self.docker_client.containers.get(docker_id) return list(handle.attrs['NetworkSettings']['Networks'].values())[0]['IPAddress'] @@ -550,16 +572,16 @@ class ClickHouseCluster: container_info = self.docker_client.api.inspect_container(container_id) image_id = container_info.get('Image') image_info = self.docker_client.api.inspect_image(image_id) - print(("Command failed in container {}: ".format(container_id))) - pprint.pprint(container_info) - print("") - print(("Container {} uses image {}: ".format(container_id, image_id))) - pprint.pprint(image_info) - print("") + logging.debug(("Command failed in container {}: ".format(container_id))) + pprint.plogging.debug(container_info) + logging.debug("") + logging.debug(("Container {} uses image {}: ".format(container_id, image_id))) + pprint.plogging.debug(image_info) + logging.debug("") message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output) if nothrow: - print(message) + logging.debug(message) else: raise Exception(message) if not detach: @@ -582,7 +604,7 @@ class ClickHouseCluster: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) conn.close() - print("Mysql Started") + logging.debug("Mysql Started") return except Exception as ex: errors += [str(ex)] @@ -598,10 +620,10 @@ class ClickHouseCluster: try: conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql8_port) conn.close() - print("Mysql 8 Started") + logging.debug("Mysql 8 Started") return except Exception as ex: - print("Can't connect to MySQL 8 " + str(ex)) + logging.debug("Can't connect to MySQL 8 " + str(ex)) time.sleep(0.5) subprocess_call(['docker-compose', 'ps', '--services', '--all']) @@ -613,10 +635,10 @@ class ClickHouseCluster: try: conn = psycopg2.connect(host='127.0.0.1', port=self.postgres_port, user='postgres', password='mysecretpassword') conn.close() - print("Postgres Started") + logging.debug("Postgres Started") return except Exception as ex: - print("Can't connect to Postgres " + str(ex)) + logging.debug("Can't connect to Postgres " + str(ex)) time.sleep(0.5) raise Exception("Cannot wait Postgres container") @@ -628,10 +650,10 @@ class ClickHouseCluster: for instance in ['zoo1', 'zoo2', 'zoo3']: conn = self.get_kazoo_client(instance) conn.get_children('/') - print("All instances of ZooKeeper started") + logging.debug("All instances of ZooKeeper started") return except Exception as ex: - print("Can't connect to ZooKeeper " + str(ex)) + logging.debug("Can't connect to ZooKeeper " + str(ex)) time.sleep(0.5) raise Exception("Cannot wait ZooKeeper container") @@ -641,9 +663,9 @@ class ClickHouseCluster: keytab = p.abspath(p.join(self.instances['node1'].path, "secrets/clickhouse.keytab")) krb_conf = p.abspath(p.join(self.instances['node1'].path, "secrets/krb_long.conf")) hdfs_ip = self.get_instance_ip('kerberizedhdfs1') - # print("kerberizedhdfs1 ip ", hdfs_ip) + # logging.debug("kerberizedhdfs1 ip ", hdfs_ip) kdc_ip = self.get_instance_ip('hdfskerberos') - # print("kdc_ip ", kdc_ip) + # logging.debug("kdc_ip ", kdc_ip) self.hdfs_api = HDFSApi(user="root", timeout=timeout, kerberized=True, @@ -657,7 +679,7 @@ class ClickHouseCluster: hdfs_ip=hdfs_ip, kdc_ip=kdc_ip) else: - self.hdfs_api = HDFSApi(user="root", host="hdfs1") + self.hdfs_api = HDFSApi(user="root", host=self.hdfs_host) def wait_hdfs_to_start(self, timeout=60): @@ -665,10 +687,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: self.hdfs_api.write_data("/somefilewithrandomname222", "1") - print("Connected to HDFS and SafeMode disabled! ") + logging.debug("Connected to HDFS and SafeMode disabled! ") return except Exception as ex: - print("Can't connect to HDFS " + str(ex)) + logging.debug("Can't connect to HDFS " + str(ex)) time.sleep(1) raise Exception("Can't wait HDFS to start") @@ -681,10 +703,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: connection.list_database_names() - print("Connected to Mongo dbs:", connection.database_names()) + logging.debug("Connected to Mongo dbs:", connection.database_names()) return except Exception as ex: - print("Can't connect to Mongo " + str(ex)) + logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): @@ -697,7 +719,7 @@ class ClickHouseCluster: try: minio_client.list_buckets() - print("Connected to Minio.") + logging.debug("Connected to Minio.") buckets = [self.minio_bucket, self.minio_bucket_2] @@ -705,12 +727,12 @@ class ClickHouseCluster: if minio_client.bucket_exists(bucket): minio_client.remove_bucket(bucket) minio_client.make_bucket(bucket) - print("S3 bucket '%s' created", bucket) + logging.debug("S3 bucket '%s' created", bucket) self.minio_client = minio_client return except Exception as ex: - print("Can't connect to Minio: %s", str(ex)) + logging.debug("Can't connect to Minio: %s", str(ex)) time.sleep(1) raise Exception("Can't wait Minio to start") @@ -721,10 +743,10 @@ class ClickHouseCluster: while time.time() - start < timeout: try: sr_client._send_request(sr_client.url) - print("Connected to SchemaRegistry") + logging.debug("Connected to SchemaRegistry") return sr_client except Exception as ex: - print(("Can't connect to SchemaRegistry: %s", str(ex))) + logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) def wait_cassandra_to_start(self, timeout=30): @@ -740,27 +762,27 @@ class ClickHouseCluster: time.sleep(1) def start(self, destroy_dirs=True): - print("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) + logging.debug("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) if self.is_up: return # Just in case kill unstopped containers from previous launch try: - print("Trying to kill unstopped containers...") + logging.debug("Trying to kill unstopped containers...") if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) - print("Unstopped containers killed") + logging.debug("Unstopped containers killed") except: pass try: if destroy_dirs and p.exists(self.instances_dir): - print(("Removing instances dir %s", self.instances_dir)) + logging.debug(("Removing instances dir %s", self.instances_dir)) shutil.rmtree(self.instances_dir) for instance in list(self.instances.values()): - print(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) + logging.debug(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) instance.create_dir(destroy_dir=destroy_dirs) self.docker_client = docker.from_env(version=self.docker_api_version) @@ -768,7 +790,7 @@ class ClickHouseCluster: common_opts = ['up', '-d', '--force-recreate'] if self.with_zookeeper and self.base_zookeeper_cmd: - print('Setup ZooKeeper') + logging.debug('Setup ZooKeeper') env = os.environ.copy() if not self.zookeeper_use_tmpfs: env['ZK_FS'] = 'bind' @@ -787,51 +809,53 @@ class ClickHouseCluster: self.wait_zookeeper_to_start(120) if self.with_mysql and self.base_mysql_cmd: - print('Setup MySQL') + logging.debug('Setup MySQL') subprocess_check_call(self.base_mysql_cmd + common_opts) self.wait_mysql_to_start(120) if self.with_mysql8 and self.base_mysql8_cmd: - print('Setup MySQL 8') + logging.debug('Setup MySQL 8') subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start(120) if self.with_postgres and self.base_postgres_cmd: - print('Setup Postgres') + logging.debug('Setup Postgres') subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(30) if self.with_kafka and self.base_kafka_cmd: - print('Setup Kafka') + logging.debug('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: - print('Setup kerberized kafka') + logging.debug('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) self.rabbitmq_docker_id = self.get_instance_docker_id('rabbitmq1') if self.with_hdfs and self.base_hdfs_cmd: - print('Setup HDFS') + logging.debug('Setup HDFS') + os.makedirs(self.hdfs_logs_dir) subprocess_check_call(self.base_hdfs_cmd + common_opts) self.make_hdfs_api() - self.wait_hdfs_to_start(120) + self.wait_hdfs_to_start(50) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: - print('Setup kerberized HDFS') + logging.debug('Setup kerberized HDFS') run_and_check(self.base_kerberized_hdfs_cmd + common_opts) self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(timeout=300) if self.with_mongo and self.base_mongo_cmd: - print('Setup Mongo') + logging.debug('Setup Mongo') run_and_check(self.base_mongo_cmd + common_opts) self.wait_mongo_to_start(30) if self.with_redis and self.base_redis_cmd: - print('Setup Redis') + logging.debug('Setup Redis') subprocess_check_call(self.base_redis_cmd + common_opts) time.sleep(10) @@ -850,28 +874,28 @@ class ClickHouseCluster: _create_env_file(os.path.join(self.env_file), self.env_variables) clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate'] - print(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) + logging.debug(("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))) self.up_called = True subprocess.check_output(clickhouse_start_cmd) - print("ClickHouse instance created") + logging.debug("ClickHouse instance created") start_deadline = time.time() + 20.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) - print("Waiting for ClickHouse start...") + logging.debug("Waiting for ClickHouse start...") instance.wait_for_start(start_deadline) - print("ClickHouse started") + logging.debug("ClickHouse started") instance.client = Client(instance.ip_address, command=self.client_bin_path) self.is_up = True except BaseException as e: - print("Failed to start cluster: ") - print(str(e)) - print(traceback.print_exc()) + logging.debug("Failed to start cluster: ") + logging.debug(str(e)) + logging.debug(traceback.print_exc()) raise def shutdown(self, kill=True): @@ -881,7 +905,7 @@ class ClickHouseCluster: try: subprocess.check_call(self.base_cmd + ['logs'], stdout=f) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL except Exception as e: - print("Unable to get logs from docker.") + logging.debug("Unable to get logs from docker.") f.seek(0) for line in f: if SANITIZER_SIGN in line: @@ -892,14 +916,14 @@ class ClickHouseCluster: try: subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) except Exception as e: - print("Kill command failed during shutdown. {}".format(repr(e))) - print("Trying to kill forcefully") + logging.debug("Kill command failed during shutdown. {}".format(repr(e))) + logging.debug("Trying to kill forcefully") subprocess_check_call(self.base_cmd + ['kill']) try: subprocess_check_call(self.base_cmd + ['down', '--volumes']) except Exception as e: - print("Down + remove orphans failed durung shutdown. {}".format(repr(e))) + logging.debug("Down + remove orphans failed durung shutdown. {}".format(repr(e))) self.is_up = False @@ -947,7 +971,7 @@ class ClickHouseCluster: kazoo_callback(self.get_kazoo_client(zoo_instance_name)) return except KazooException as e: - print(repr(e)) + logging.debug(repr(e)) time.sleep(sleep_for) kazoo_callback(self.get_kazoo_client(zoo_instance_name)) @@ -1112,7 +1136,7 @@ class ClickHouseInstance: return result time.sleep(sleep_time) except Exception as ex: - print("Retry {} got exception {}".format(i + 1, ex)) + logging.debug("Retry {} got exception {}".format(i + 1, ex)) time.sleep(sleep_time) if result is not None: @@ -1372,16 +1396,16 @@ class ClickHouseInstance: instance_config_dir = p.abspath(p.join(self.path, 'configs')) os.makedirs(instance_config_dir) - print("Copy common default production configuration from {}".format(self.base_config_dir)) + logging.debug("Copy common default production configuration from {}".format(self.base_config_dir)) shutil.copyfile(p.join(self.base_config_dir, 'config.xml'), p.join(instance_config_dir, 'config.xml')) shutil.copyfile(p.join(self.base_config_dir, 'users.xml'), p.join(instance_config_dir, 'users.xml')) - print("Create directory for configuration generated in this helper") + logging.debug("Create directory for configuration generated in this helper") # used by all utils with any config conf_d_dir = p.abspath(p.join(instance_config_dir, 'conf.d')) os.mkdir(conf_d_dir) - print("Create directory for common tests configuration") + logging.debug("Create directory for common tests configuration") # used by server with main config.xml self.config_d_dir = p.abspath(p.join(instance_config_dir, 'config.d')) os.mkdir(self.config_d_dir) @@ -1390,14 +1414,14 @@ class ClickHouseInstance: dictionaries_dir = p.abspath(p.join(instance_config_dir, 'dictionaries')) os.mkdir(dictionaries_dir) - print("Copy common configuration from helpers") + logging.debug("Copy common configuration from helpers") # The file is named with 0_ prefix to be processed before other configuration overloads. shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), self.config_d_dir) shutil.copy(p.join(HELPERS_DIR, '0_common_instance_users.xml'), users_d_dir) if len(self.custom_dictionaries_paths): shutil.copy(p.join(HELPERS_DIR, '0_common_enable_dictionaries.xml'), self.config_d_dir) - print("Generate and write macros file") + logging.debug("Generate and write macros file") macros = self.macros.copy() macros['instance'] = self.name with open(p.join(conf_d_dir, 'macros.xml'), 'w') as macros_config: @@ -1411,7 +1435,7 @@ class ClickHouseInstance: shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config.d configs - print("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) + logging.debug("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) for path in self.custom_main_config_paths: shutil.copy(path, self.config_d_dir) @@ -1424,16 +1448,16 @@ class ClickHouseInstance: shutil.copy(path, dictionaries_dir) db_dir = p.abspath(p.join(self.path, 'database')) - print("Setup database dir {}".format(db_dir)) + logging.debug("Setup database dir {}".format(db_dir)) if self.clickhouse_path_dir is not None: - print("Database files taken from {}".format(self.clickhouse_path_dir)) + logging.debug("Database files taken from {}".format(self.clickhouse_path_dir)) shutil.copytree(self.clickhouse_path_dir, db_dir) - print("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) + logging.debug("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) else: os.mkdir(db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) - print("Setup logs dir {}".format(logs_dir)) + logging.debug("Setup logs dir {}".format(logs_dir)) os.mkdir(logs_dir) depends_on = [] @@ -1478,7 +1502,7 @@ class ClickHouseInstance: if self.stay_alive: entrypoint_cmd = CLICKHOUSE_STAY_ALIVE_COMMAND - print("Entrypoint cmd: {}".format(entrypoint_cmd)) + logging.debug("Entrypoint cmd: {}".format(entrypoint_cmd)) networks = app_net = ipv4_address = ipv6_address = net_aliases = net_alias1 = "" if self.ipv4_address is not None or self.ipv6_address is not None or self.hostname != self.name: diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index a7ca8c57da8..7d4a3ad1c29 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -4,3 +4,10 @@ norecursedirs = _instances timeout = 300 junit_duration_report = call junit_suite_name = integration +log_cli = 1 +log_cli_level = CRITICAL +log_cli_format = %(message)s +log_file = pytest.log +log_file_level = DEBUG +log_file_format = %(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) +log_file_date_format=%Y-%m-%d %H:%M:%S diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index a6c8b7e1ee9..ac3807908a8 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -4,7 +4,6 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.hdfs_api import HDFSApi -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_hdfs=True, main_configs=['configs/log_conf.xml']) @@ -13,12 +12,7 @@ node1 = cluster.add_instance('node1', with_hdfs=True, main_configs=['configs/log def started_cluster(): try: cluster.start() - yield cluster - - except Exception as ex: - print(ex) - raise ex finally: cluster.shutdown() diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 24afbc8ca4e..da5826665e5 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -638,7 +638,6 @@ def kafka_cluster(): try: cluster.start() yield cluster - finally: cluster.shutdown() diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index accb855af10..e21d18c2f49 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -37,7 +37,7 @@ def check_kafka_is_available(): kafka_id, '/usr/bin/kafka-broker-api-versions', '--bootstrap-server', - 'localhost:9093'), + 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), stdout=subprocess.PIPE) p.communicate() return p.returncode == 0 @@ -58,14 +58,26 @@ def wait_kafka_is_available(max_retries=50): def producer_serializer(x): return x.encode() if isinstance(x, str) else x + +def get_kafka_producer(port, serializer): + errors = [] + for _ in range(15): + try: + producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) + logging.debug("Kafka Connection establised: localhost:{}".format(port)) + return producer + except Exception as e: + errors += [str(e)] + time.sleep(1) -def kafka_produce(topic, messages, timestamp=None): - producer = KafkaProducer(bootstrap_servers="localhost:9093", value_serializer=producer_serializer) + raise Exception("Connection not establised, {}".format(errors)) + +def kafka_produce(kafka_cluster, topic, messages, timestamp=None): + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() - print ("Produced {} messages for topic {}".format(len(messages), topic)) - # Fixtures @@ -74,9 +86,7 @@ def kafka_produce(topic, messages, timestamp=None): def kafka_cluster(): try: cluster.start() - print("kafka_id is {}".format(kafka_id)) yield cluster - finally: cluster.shutdown() @@ -92,7 +102,7 @@ def kafka_setup_teardown(): @pytest.mark.timeout(180) # wait to build containers def test_kafka_json_as_string(kafka_cluster): - kafka_produce('kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + kafka_produce(kafka_cluster, 'kafka_json_as_string', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) instance.query(''' CREATE TABLE test.kafka (field String) @@ -116,7 +126,7 @@ def test_kafka_json_as_string(kafka_cluster): assert instance.contains_in_log("Parsing of message (topic: kafka_json_as_string, partition: 0, offset: 1) return no rows") def test_kafka_json_as_string_no_kdc(kafka_cluster): - kafka_produce('kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) + kafka_produce(kafka_cluster, 'kafka_json_as_string_no_kdc', ['{"t": 123, "e": {"x": "woof"} }', '', '{"t": 124, "e": {"x": "test"} }', '{"F1":"V1","F2":{"F21":"V21","F22":{},"F23":"V23","F24":"2019-12-24T16:28:04"},"F3":"V3"}']) kafka_cluster.pause_container('kafka_kerberos') time.sleep(45) # wait for ticket expiration diff --git a/tests/integration/test_storage_mongodb/test.py b/tests/integration/test_storage_mongodb/test.py index f75a9aac237..75af909faec 100644 --- a/tests/integration/test_storage_mongodb/test.py +++ b/tests/integration/test_storage_mongodb/test.py @@ -13,20 +13,18 @@ node = cluster.add_instance('node', with_mongo=True) def started_cluster(): try: cluster.start() - yield cluster - finally: cluster.shutdown() -def get_mongo_connection(): - connection_str = 'mongodb://root:clickhouse@localhost:27018' +def get_mongo_connection(started_cluster): + connection_str = 'mongodb://root:clickhouse@localhost:{}'.format(started_cluster.mongo_port) return pymongo.MongoClient(connection_str) def test_simple_select(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') simple_mongo_table = db['simple_table'] @@ -45,7 +43,7 @@ def test_simple_select(started_cluster): def test_complex_data_type(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') incomplete_mongo_table = db['complex_table'] @@ -64,7 +62,7 @@ def test_complex_data_type(started_cluster): def test_incorrect_data_type(started_cluster): - mongo_connection = get_mongo_connection() + mongo_connection = get_mongo_connection(started_cluster) db = mongo_connection['test'] db.add_user('root', 'clickhouse') strange_mongo_table = db['strange_table'] From ee955038c1139da47f80326459e26f47c7c426c2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 15:58:11 +0300 Subject: [PATCH 0031/1060] wip --- .../runner/compose/docker_compose_minio.yml | 2 +- tests/integration/helpers/cluster.py | 97 ++++++++++--------- tests/integration/helpers/hdfs_api.py | 97 ++++++------------- .../test_allowed_url_from_config/test.py | 6 +- .../test_redirect_url_storage/test.py | 18 ++-- tests/integration/test_storage_hdfs/test.py | 63 +++++++----- .../test_storage_kerberized_hdfs/test.py | 22 +++-- 7 files changed, 158 insertions(+), 147 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_minio.yml b/docker/test/integration/runner/compose/docker_compose_minio.yml index dbb29f9711a..73016075921 100644 --- a/docker/test/integration/runner/compose/docker_compose_minio.yml +++ b/docker/test/integration/runner/compose/docker_compose_minio.yml @@ -7,7 +7,7 @@ services: - data1-1:/data1 - ${MINIO_CERTS_DIR:-}:/certs ports: - - "9001:9001" + - ${MINIO_EXTERNAL_PORT}:${MINIO_INTERNAL_PORT} environment: MINIO_ACCESS_KEY: minio MINIO_SECRET_KEY: minio123 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index cb5c25c260e..302901159e1 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -36,7 +36,7 @@ DEFAULT_ENV_NAME = '.env' SANITIZER_SIGN = "==================" - +# to create docker-compose env file def _create_env_file(path, variables): logging.debug("Env {} stored in {}".format(variables, path)) with open(path, 'w') as f: @@ -44,13 +44,6 @@ def _create_env_file(path, variables): f.write("=".join([var, value]) + "\n") return path -def env_to_compose_args(env): - args = [] - for key, value in env.items(): - args += ["-e", "{}={}".format(key, value)] - return args - - def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) if res.returncode != 0: @@ -174,7 +167,8 @@ class ClickHouseCluster: self.with_cassandra = False self.with_minio = False - self.minio_certs_dir = None + self.minio_dir = os.path.join(self.instances_dir, "minio") + self.minio_certs_dir = None # source for certificates self.minio_host = "minio1" self.minio_bucket = "root" self.minio_bucket_2 = "root2" @@ -347,6 +341,19 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_mongo.yml')] return self.base_mongo_cmd + def setup_minio_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_minio = True + cert_d = p.join(self.minio_dir, "certs") + env_variables['MINIO_CERTS_DIR'] = cert_d + env_variables['MINIO_EXTERNAL_PORT'] = self.minio_port + env_variables['MINIO_INTERNAL_PORT'] = "9001" + env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, cert_d, 'public.crt') + + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) + self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] + return self.base_minio_cmd + def add_instance(self, name, base_config_dir=None, main_configs=None, user_configs=None, dictionaries=None, macros=None, with_zookeeper=False, with_mysql=False, with_mysql8=False, with_kafka=False, with_kerberized_kafka=False, with_rabbitmq=False, @@ -484,21 +491,13 @@ class ClickHouseCluster: cmds.append(self.setup_redis_cmd(instance, env_variables, docker_compose_yml_dir)) if with_minio and not self.with_minio: - self.with_minio = True - self.minio_certs_dir = minio_certs_dir - if self.minio_certs_dir: - env_variables['MINIO_CERTS_DIR'] = p.join(self.base_dir, self.minio_certs_dir) - # Minio client (urllib3) uses SSL_CERT_FILE for certificate validation. - env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') + cmds.append(self.setup_minio_cmd(instance, env_variables, docker_compose_yml_dir)) + + if minio_certs_dir is not None: + if self.minio_certs_dir is None: + self.minio_certs_dir = minio_certs_dir else: - # Attach empty certificates directory to ensure non-secure mode. - minio_certs_dir = p.join(self.instances_dir, 'empty_minio_certs_dir') - os.makedirs(minio_certs_dir, exist_ok=True) - env_variables['MINIO_CERTS_DIR'] = minio_certs_dir - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')]) - self.base_minio_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_minio.yml')] - cmds.append(self.base_minio_cmd) + raise Exception("Overwriting minio certs dir") if with_cassandra and not self.with_cassandra: self.with_cassandra = True @@ -659,6 +658,7 @@ class ClickHouseCluster: raise Exception("Cannot wait ZooKeeper container") def make_hdfs_api(self, timeout=60, kerberized=False): + hdfs_api = None if kerberized: keytab = p.abspath(p.join(self.instances['node1'].path, "secrets/clickhouse.keytab")) krb_conf = p.abspath(p.join(self.instances['node1'].path, "secrets/krb_long.conf")) @@ -666,27 +666,29 @@ class ClickHouseCluster: # logging.debug("kerberizedhdfs1 ip ", hdfs_ip) kdc_ip = self.get_instance_ip('hdfskerberos') # logging.debug("kdc_ip ", kdc_ip) - self.hdfs_api = HDFSApi(user="root", - timeout=timeout, - kerberized=True, - principal="root@TEST.CLICKHOUSE.TECH", - keytab=keytab, - krb_conf=krb_conf, - host="kerberizedhdfs1", - protocol="http", - proxy_port=50070, - data_port=1006, - hdfs_ip=hdfs_ip, - kdc_ip=kdc_ip) + hdfs_api = HDFSApi(user="root", + timeout=timeout, + kerberized=True, + principal="root@TEST.CLICKHOUSE.TECH", + keytab=keytab, + krb_conf=krb_conf, + host="kerberizedhdfs1", + protocol="http", + proxy_port=50070, + data_port=1006, + hdfs_ip=hdfs_ip, + kdc_ip=kdc_ip) + else: - self.hdfs_api = HDFSApi(user="root", host=self.hdfs_host) + logging.debug("Create HDFSApi host={}".format("localhost")) + hdfs_api = HDFSApi(user="root", host="localhost", data_port=self.hdfs_data_port, proxy_port=self.hdfs_name_port) + return hdfs_api - - def wait_hdfs_to_start(self, timeout=60): + def wait_hdfs_to_start(self, hdfs_api, timeout=60): start = time.time() while time.time() - start < timeout: try: - self.hdfs_api.write_data("/somefilewithrandomname222", "1") + hdfs_api.write_data("/somefilewithrandomname222", "1") logging.debug("Connected to HDFS and SafeMode disabled! ") return except Exception as ex: @@ -710,7 +712,7 @@ class ClickHouseCluster: time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): - minio_client = Minio('localhost:9001', + minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', secret_key='minio123', secure=secure) @@ -840,14 +842,14 @@ class ClickHouseCluster: logging.debug('Setup HDFS') os.makedirs(self.hdfs_logs_dir) subprocess_check_call(self.base_hdfs_cmd + common_opts) - self.make_hdfs_api() - self.wait_hdfs_to_start(50) + hdfs_api = self.make_hdfs_api() + self.wait_hdfs_to_start(hdfs_api, 120) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') run_and_check(self.base_kerberized_hdfs_cmd + common_opts) - self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(timeout=300) + hdfs_api = self.make_hdfs_api(kerberized=True) + self.wait_hdfs_to_start(hdfs_api, timeout=300) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') @@ -860,6 +862,13 @@ class ClickHouseCluster: time.sleep(10) if self.with_minio and self.base_minio_cmd: + # Copy minio certificates to minio/certs + os.mkdir(self.minio_dir) + if self.minio_certs_dir is None: + os.mkdir(os.path.join(self.minio_dir, 'certs')) + else: + shutil.copytree(self.minio_certs_dir, os.path.join(self.minio_dir, 'certs')) + minio_start_cmd = self.base_minio_cmd + common_opts logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd))) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index cb742662855..903a670aa57 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -11,15 +11,6 @@ import tempfile import logging import os -g_dns_hook = None - -def custom_getaddrinfo(*args): - # print("from custom_getaddrinfo g_dns_hook is None ", g_dns_hook is None) - ret = g_dns_hook.custom_getaddrinfo(*args) - # print("g_dns_hook.custom_getaddrinfo result", ret) - return ret - - class mk_krb_conf(object): def __init__(self, krb_conf, kdc_ip): self.krb_conf = krb_conf @@ -37,32 +28,6 @@ class mk_krb_conf(object): if self.amended_krb_conf is not None: self.amended_krb_conf.close() -# tweak dns resolution to connect to localhost where api_host is in URL -class dns_hook(object): - def __init__(self, hdfs_api): - # print("dns_hook.init ", hdfs_api.kerberized, hdfs_api.host, hdfs_api.data_port, hdfs_api.proxy_port) - self.hdfs_api = hdfs_api - def __enter__(self): - global g_dns_hook - g_dns_hook = self - # print("g_dns_hook is None ", g_dns_hook is None) - self.original_getaddrinfo = socket.getaddrinfo - socket.getaddrinfo = custom_getaddrinfo - return self - def __exit__(self, type, value, traceback): - global g_dns_hook - g_dns_hook = None - socket.getaddrinfo = self.original_getaddrinfo - def custom_getaddrinfo(self, *args): - (hostname, port) = args[:2] - # print("top of custom_getaddrinfo", hostname, port) - - if hostname == self.hdfs_api.host and (port == self.hdfs_api.data_port or port == self.hdfs_api.proxy_port): - # print("dns_hook substitute") - return [(socket.AF_INET, 1, 6, '', ("127.0.0.1", port))] - else: - return self.original_getaddrinfo(*args) - class HDFSApi(object): def __init__(self, user, timeout=100, kerberized=False, principal=None, keytab=None, krb_conf=None, @@ -83,9 +48,9 @@ class HDFSApi(object): # logging.basicConfig(level=logging.DEBUG) # logging.getLogger().setLevel(logging.DEBUG) - # requests_log = logging.getLogger("requests.packages.urllib3") - # requests_log.setLevel(logging.DEBUG) - # requests_log.propagate = True + requests_log = logging.getLogger("requests.packages.urllib3") + requests_log.setLevel(logging.DEBUG) + requests_log.propagate = True if kerberized: self._run_kinit() @@ -101,13 +66,13 @@ class HDFSApi(object): raise Exception("kerberos principal and keytab are required") with mk_krb_conf(self.krb_conf, self.kdc_ip) as instantiated_krb_conf: - # print("instantiated_krb_conf ", instantiated_krb_conf) + logging.debug("instantiated_krb_conf ", instantiated_krb_conf) os.environ["KRB5_CONFIG"] = instantiated_krb_conf cmd = "(kinit -R -t {keytab} -k {principal} || (sleep 5 && kinit -R -t {keytab} -k {principal})) ; klist".format(instantiated_krb_conf=instantiated_krb_conf, keytab=self.keytab, principal=self.principal) - # print(cmd) + logging.debug(cmd) start = time.time() @@ -123,17 +88,15 @@ class HDFSApi(object): raise Exception("Kinit running failure") def read_data(self, path, universal_newlines=True): - with dns_hook(self): - response = requests.get("{protocol}://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(protocol=self.protocol, host=self.host, port=self.proxy_port, path=path), headers={'host': 'localhost'}, allow_redirects=False, verify=False, auth=self.kerberos_auth) + logging.debug("read_data protocol:{} host:{} port:{} path: {}".format(self.protocol, self.host, self.proxy_port, path)) + response = requests.get("{protocol}://{host}:{port}/webhdfs/v1{path}?op=OPEN".format(protocol=self.protocol, host=self.host, port=self.proxy_port, path=path), headers={'host': 'localhost'}, allow_redirects=False, verify=False, auth=self.kerberos_auth) if response.status_code != 307: response.raise_for_status() # additional_params = '&'.join(response.headers['Location'].split('&')[1:2]) - url = "{location}".format(location=response.headers['Location']) - # print("redirected to ", url) - with dns_hook(self): - response_data = requests.get(url, - headers={'host': 'localhost'}, - verify=False, auth=self.kerberos_auth) + url = "{location}".format(location=response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port))) + logging.debug("redirected to {}".format(url)) + response_data = requests.get(url, headers={'host': 'localhost'}, + verify=False, auth=self.kerberos_auth) if response_data.status_code != 200: response_data.raise_for_status() if universal_newlines: @@ -142,6 +105,7 @@ class HDFSApi(object): return response_data.content def write_data(self, path, content): + logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}".format(self.protocol, self.host, self.proxy_port, path, self.user)) named_file = NamedTemporaryFile(mode='wb+') fpath = named_file.name if isinstance(content, str): @@ -149,40 +113,41 @@ class HDFSApi(object): named_file.write(content) named_file.flush() - if self.kerberized: self._run_kinit() self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) - # print(self.kerberos_auth) + logging.debug(self.kerberos_auth) + + response = requests.put( + "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', + port=self.proxy_port, + path=path, user=self.user), + allow_redirects=False, + headers={'host': 'localhost'}, + params={'overwrite' : 'true'}, + verify=False, auth=self.kerberos_auth + ) + + logging.debug("HDFS api response:{}".format(response.headers)) - with dns_hook(self): - response = requests.put( - "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host=self.host, - port=self.proxy_port, - path=path, user=self.user), - allow_redirects=False, - headers={'host': 'localhost'}, - params={'overwrite' : 'true'}, - verify=False, auth=self.kerberos_auth - ) if response.status_code != 307: - # print(response.headers) response.raise_for_status() - additional_params = '&'.join( - response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) + # additional_params = '&'.join( + # response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) - with dns_hook(self), open(fpath, mode="rb") as fh: + with open(fpath, mode="rb") as fh: file_data = fh.read() protocol = "http" # self.protocol response = requests.put( - "{location}".format(location=response.headers['Location']), + "{location}".format(location=location), data=file_data, headers={'content-type':'text/plain', 'host': 'localhost'}, params={'file': path, 'user.name' : self.user}, allow_redirects=False, verify=False, auth=self.kerberos_auth ) - # print(response) + logging.debug(response) if response.status_code != 201: response.raise_for_status() diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index 6442937c8f4..29844301dba 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -100,8 +100,10 @@ def test_table_function_remote(start_cluster): def test_redirect(start_cluster): - start_cluster.hdfs_api.write_data("/simple_storage", "1\t\n") - assert start_cluster.hdfs_api.read_data("/simple_storage") == "1\t\n" + hdfs_api = start_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\t\n") + assert hdfs_api.read_data("/simple_storage") == "1\t\n" node7.query( "CREATE TABLE table_test_7_1 (word String) ENGINE=URL('http://hdfs1:50070/webhdfs/v1/simple_storage?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0', CSV)") assert "not allowed" in node7.query_and_get_error("SET max_http_get_redirects=1; SELECT * from table_test_7_1") diff --git a/tests/integration/test_redirect_url_storage/test.py b/tests/integration/test_redirect_url_storage/test.py index f2731794d43..736fb5c409c 100644 --- a/tests/integration/test_redirect_url_storage/test.py +++ b/tests/integration/test_redirect_url_storage/test.py @@ -17,8 +17,10 @@ def started_cluster(): def test_url_without_redirect(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access datanode port directly node1.query( @@ -27,8 +29,10 @@ def test_url_without_redirect(started_cluster): def test_url_with_redirect_not_allowed(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access proxy port without allowing redirects node1.query( @@ -38,8 +42,10 @@ def test_url_with_redirect_not_allowed(started_cluster): def test_url_with_redirect_allowed(started_cluster): - started_cluster.hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + hdfs_api = started_cluster.make_hdfs_api() + + hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" # access proxy port with allowing redirects # http://localhost:50070/webhdfs/v1/b?op=OPEN&namenoderpcaddress=hdfs1:9000&offset=0 diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index ac3807908a8..3fb12b233fa 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -18,14 +18,16 @@ def started_cluster(): def test_read_write_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/simple_storage', 'TSV')") node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" def test_read_write_storage_with_globs(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table HDFSStorageWithRange (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1..5}', 'TSV')") node1.query( @@ -36,8 +38,8 @@ def test_read_write_storage_with_globs(started_cluster): "create table HDFSStorageWithAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage*', 'TSV')") for i in ["1", "2", "3"]: - started_cluster.hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") - assert started_cluster.hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" + hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") + assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" assert node1.query("select count(*) from HDFSStorageWithRange") == "3\n" assert node1.query("select count(*) from HDFSStorageWithEnum") == "3\n" @@ -67,23 +69,26 @@ def test_read_write_storage_with_globs(started_cluster): def test_read_write_table(started_cluster): - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_data("/simple_table_function") == data + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", data) + + assert hdfs_api.read_data("/simple_table_function") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data def test_write_table(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() node1.query( "create table OtherHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/other_storage', 'TSV')") node1.query("insert into OtherHDFSStorage values (10, 'tomas', 55.55), (11, 'jack', 32.54)") result = "10\ttomas\t55.55\n11\tjack\t32.54\n" - assert started_cluster.hdfs_api.read_data("/other_storage") == result + assert hdfs_api.read_data("/other_storage") == result assert node1.query("select * from OtherHDFSStorage order by id") == result @@ -110,12 +115,14 @@ def test_bad_hdfs_uri(started_cluster): @pytest.mark.timeout(800) def test_globs_in_read_table(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + some_data = "1\tSerialize\t555.222\n2\tData\t777.333\n" globs_dir = "/dir_for_test_with_globs/" files = ["dir1/dir_dir/file1", "dir2/file2", "simple_table_function", "dir/file", "some_dir/dir1/file", "some_dir/dir2/file", "some_dir/file", "table1_function", "table2_function", "table3_function"] for filename in files: - started_cluster.hdfs_api.write_data(globs_dir + filename, some_data) + hdfs_api.write_data(globs_dir + filename, some_data) test_requests = [("dir{1..5}/dir_dir/file1", 1, 1), ("*_table_functio?", 1, 1), @@ -141,58 +148,70 @@ def test_globs_in_read_table(started_cluster): def test_read_write_gzip_table(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data def test_read_write_gzip_table_with_parameter_gzip(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function", data) + + assert hdfs_api.read_gzip_data("/simple_table_function") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data def test_read_write_table_with_parameter_none(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_data("/simple_table_function.gz", data) + + assert hdfs_api.read_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster): - data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" - started_cluster.hdfs_api.write_gzip_data("/simple_table_function.gz", data) + hdfs_api = started_cluster.make_hdfs_api() - assert started_cluster.hdfs_api.read_gzip_data("/simple_table_function.gz") == data + data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n" + hdfs_api.write_gzip_data("/simple_table_function.gz", data) + + assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data assert node1.query( "select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data def test_write_gz_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query( "create table GZHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage.gz', 'TSV')") node1.query("insert into GZHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" + assert hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n" assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n" def test_write_gzip_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query( "create table GZIPHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/gzip_storage', 'TSV', 'gzip')") node1.query("insert into GZIPHDFSStorage values (1, 'Mark', 72.53)") - assert started_cluster.hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" + assert hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n" assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n" if __name__ == '__main__': diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index 1fffd7a8c12..bb4440a3e38 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -23,10 +23,12 @@ def started_cluster(): cluster.shutdown() def test_read_table(started_cluster): - data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function", data) + hdfs_api = started_cluster.make_hdfs_api() - api_read = started_cluster.hdfs_api.read_data("/simple_table_function") + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", data) + + api_read = hdfs_api.read_data("/simple_table_function") assert api_read == data select_read = node1.query("select * from hdfs('hdfs://kerberizedhdfs1:9010/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") @@ -34,10 +36,12 @@ def test_read_table(started_cluster): def test_read_write_storage(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query("create table SimpleHDFSStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage1', 'TSV')") node1.query("insert into SimpleHDFSStorage2 values (1, 'Mark', 72.53)") - api_read = started_cluster.hdfs_api.read_data("/simple_storage1") + api_read = hdfs_api.read_data("/simple_storage1") assert api_read == "1\tMark\t72.53\n" select_read = node1.query("select * from SimpleHDFSStorage2") @@ -45,12 +49,14 @@ def test_read_write_storage(started_cluster): def test_write_storage_not_expired(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query("create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')") time.sleep(45) # wait for ticket expiration node1.query("insert into SimpleHDFSStorageNotExpired values (1, 'Mark', 72.53)") - api_read = started_cluster.hdfs_api.read_data("/simple_storage_not_expired") + api_read = hdfs_api.read_data("/simple_storage_not_expired") assert api_read == "1\tMark\t72.53\n" select_read = node1.query("select * from SimpleHDFSStorageNotExpired") @@ -58,6 +64,8 @@ def test_write_storage_not_expired(started_cluster): def test_two_users(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + node1.query("create table HDFSStorOne (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/storage_user_one', 'TSV')") node1.query("insert into HDFSStorOne values (1, 'Real', 86.00)") @@ -69,8 +77,10 @@ def test_two_users(started_cluster): select_read_2 = node1.query("select * from hdfs('hdfs://suser@kerberizedhdfs1:9010/storage_user_one', 'TSV', 'id UInt64, text String, number Float64')") def test_read_table_expired(started_cluster): + hdfs_api = started_cluster.make_hdfs_api() + data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - started_cluster.hdfs_api.write_data("/simple_table_function_relogin", data) + hdfs_api.write_data("/simple_table_function_relogin", data) started_cluster.pause_container('hdfskerberos') time.sleep(45) From d45cab4228da23261c725479430cfa5bea66bf21 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 17:42:43 +0300 Subject: [PATCH 0032/1060] wip --- .../docker_compose_kerberized_hdfs.yml | 15 ++++--- tests/integration/helpers/cluster.py | 40 ++++++++++++++----- tests/integration/helpers/hdfs_api.py | 39 ++++++++++++------ tests/integration/runner | 14 ++++++- 4 files changed, 77 insertions(+), 31 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml index f2a659bce58..34fa902bf93 100644 --- a/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml +++ b/docker/test/integration/runner/compose/docker_compose_kerberized_hdfs.yml @@ -8,13 +8,16 @@ services: hostname: kerberizedhdfs1 restart: always volumes: - - ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro - - ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf - - ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro + - ${KERBERIZED_HDFS_DIR}/../../hdfs_configs/bootstrap.sh:/etc/bootstrap.sh:ro + - ${KERBERIZED_HDFS_DIR}/secrets:/usr/local/hadoop/etc/hadoop/conf + - ${KERBERIZED_HDFS_DIR}/secrets/krb_long.conf:/etc/krb5.conf:ro + - type: ${KERBERIZED_HDFS_FS:-tmpfs} + source: ${KERBERIZED_HDFS_LOGS:-} + target: /var/log/hadoop-hdfs ports: - - 1006:1006 - - 50070:50070 - - 9010:9010 + - ${KERBERIZED_HDFS_NAME_EXTERNAL_PORT}:${KERBERIZED_HDFS_NAME_INTERNAL_PORT} #50070 + - ${KERBERIZED_HDFS_DATA_EXTERNAL_PORT}:${KERBERIZED_HDFS_DATA_INTERNAL_PORT} #1006 + # - 9010:9010 depends_on: - hdfskerberos entrypoint: /etc/bootstrap.sh -d diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 302901159e1..bc9850d5176 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -184,6 +184,13 @@ class ClickHouseCluster: self.hdfs_dir = p.abspath(p.join(self.instances_dir, "hdfs")) self.hdfs_logs_dir = os.path.join(self.hdfs_dir, "logs") + # available when with_kerberized_hdfs == True + self.hdfs_kerberized_host = "kerberizedhdfs1" + self.hdfs_kerberized_name_port = get_open_port() + self.hdfs_kerberized_data_port = get_open_port() + self.hdfs_kerberized_dir = p.abspath(p.join(self.instances_dir, "kerberized_hdfs")) + self.hdfs_kerberized_logs_dir = os.path.join(self.hdfs_kerberized_dir, "logs") + # available when with_kafka == True self.kafka_host = "kafka1" self.kafka_port = get_open_port() @@ -276,6 +283,21 @@ class ClickHouseCluster: print("HDFS BASE CMD:{}".format(self.base_hdfs_cmd)) return self.base_hdfs_cmd + def setup_kerberized_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_kerberized_hdfs = True + env_variables['KERBERIZED_HDFS_HOST'] = self.hdfs_kerberized_host + env_variables['KERBERIZED_HDFS_NAME_EXTERNAL_PORT'] = str(self.hdfs_kerberized_name_port) + env_variables['KERBERIZED_HDFS_NAME_INTERNAL_PORT'] = "50070" + env_variables['KERBERIZED_HDFS_DATA_EXTERNAL_PORT'] = str(self.hdfs_kerberized_data_port) + env_variables['KERBERIZED_HDFS_DATA_INTERNAL_PORT'] = "1006" + env_variables['KERBERIZED_HDFS_LOGS'] = self.hdfs_kerberized_logs_dir + env_variables['KERBERIZED_HDFS_FS'] = "bind" + env_variables['KERBERIZED_HDFS_DIR'] = instance.path + '/' + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')]) + self.base_kerberized_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')] + return self.base_kerberized_hdfs_cmd + def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_redis = True env_variables['KAFKA_HOST'] = self.kafka_host @@ -345,7 +367,7 @@ class ClickHouseCluster: self.with_minio = True cert_d = p.join(self.minio_dir, "certs") env_variables['MINIO_CERTS_DIR'] = cert_d - env_variables['MINIO_EXTERNAL_PORT'] = self.minio_port + env_variables['MINIO_EXTERNAL_PORT'] = str(self.minio_port) env_variables['MINIO_INTERNAL_PORT'] = "9001" env_variables['SSL_CERT_FILE'] = p.join(self.base_dir, cert_d, 'public.crt') @@ -473,12 +495,7 @@ class ClickHouseCluster: cmds.append(self.setup_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) if with_kerberized_hdfs and not self.with_kerberized_hdfs: - self.with_kerberized_hdfs = True - env_variables['KERBERIZED_HDFS_DIR'] = instance.path + '/' - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')]) - self.base_kerberized_hdfs_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_kerberized_hdfs.yml')] - cmds.append(self.base_kerberized_hdfs_cmd) + cmds.append(self.setup_kerberized_hdfs_cmd(instance, env_variables, docker_compose_yml_dir)) if with_mongo and not self.with_mongo: cmds.append(self.setup_mongo_cmd(instance, env_variables, docker_compose_yml_dir)) @@ -672,10 +689,10 @@ class ClickHouseCluster: principal="root@TEST.CLICKHOUSE.TECH", keytab=keytab, krb_conf=krb_conf, - host="kerberizedhdfs1", + host="localhost", protocol="http", - proxy_port=50070, - data_port=1006, + proxy_port=self.hdfs_kerberized_name_port, + data_port=self.hdfs_kerberized_data_port, hdfs_ip=hdfs_ip, kdc_ip=kdc_ip) @@ -847,9 +864,10 @@ class ClickHouseCluster: if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') + os.makedirs(self.hdfs_kerberized_logs_dir) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(hdfs_api, timeout=300) + self.wait_hdfs_to_start(hdfs_api, timeout=30) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 903a670aa57..60b8979fce2 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -10,7 +10,6 @@ import socket import tempfile import logging import os - class mk_krb_conf(object): def __init__(self, krb_conf, kdc_ip): self.krb_conf = krb_conf @@ -51,6 +50,9 @@ class HDFSApi(object): requests_log = logging.getLogger("requests.packages.urllib3") requests_log.setLevel(logging.DEBUG) requests_log.propagate = True + kerb_log = logging.getLogger("requests_kerberos") + kerb_log.setLevel(logging.DEBUG) + kerb_log.propagate = True if kerberized: self._run_kinit() @@ -66,23 +68,28 @@ class HDFSApi(object): raise Exception("kerberos principal and keytab are required") with mk_krb_conf(self.krb_conf, self.kdc_ip) as instantiated_krb_conf: - logging.debug("instantiated_krb_conf ", instantiated_krb_conf) + logging.debug("instantiated_krb_conf {}".format(instantiated_krb_conf)) os.environ["KRB5_CONFIG"] = instantiated_krb_conf cmd = "(kinit -R -t {keytab} -k {principal} || (sleep 5 && kinit -R -t {keytab} -k {principal})) ; klist".format(instantiated_krb_conf=instantiated_krb_conf, keytab=self.keytab, principal=self.principal) - logging.debug(cmd) - start = time.time() while time.time() - start < self.timeout: try: - subprocess.call(cmd, shell=True) - print("KDC started, kinit successfully run") + res = subprocess.run(cmd, shell=True) + if res.returncode != 0: + # check_call(...) from subprocess does not print stderr, so we do it manually + logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) + logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) + logging.debug('Env:\n{}\n'.format(env)) + raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) + + logging.debug("KDC started, kinit successfully run") return except Exception as ex: - print("Can't run kinit ... waiting {}".format(str(ex))) + logging.debug("Can't run kinit ... waiting {}".format(str(ex))) time.sleep(1) raise Exception("Kinit running failure") @@ -93,9 +100,13 @@ class HDFSApi(object): if response.status_code != 307: response.raise_for_status() # additional_params = '&'.join(response.headers['Location'].split('&')[1:2]) - url = "{location}".format(location=response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port))) - logging.debug("redirected to {}".format(url)) - response_data = requests.get(url, headers={'host': 'localhost'}, + location = None + if self.kerberized: + location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port)) + else: + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) + logging.debug("redirected to {}".format(location)) + response_data = requests.get(location, headers={'host': 'localhost'}, verify=False, auth=self.kerberos_auth) if response_data.status_code != 200: response_data.raise_for_status() @@ -116,7 +127,6 @@ class HDFSApi(object): if self.kerberized: self._run_kinit() self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) - logging.debug(self.kerberos_auth) response = requests.put( "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', @@ -135,8 +145,11 @@ class HDFSApi(object): # additional_params = '&'.join( # response.headers['Location'].split('&')[1:2] + ["user.name={}".format(self.user), "overwrite=true"]) - location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) - + if self.kerberized: + location = response.headers['Location'].replace("kerberizedhdfs1:1006", "{}:{}".format(self.host, self.data_port)) + else: + location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) + with open(fpath, mode="rb") as fh: file_data = fh.read() protocol = "http" # self.protocol diff --git a/tests/integration/runner b/tests/integration/runner index 6dca7663310..ead57913217 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -142,12 +142,23 @@ if __name__ == "__main__": action="append", help="Set non-default tags for images used in docker compose recipes(yandex/my_container:my_tag)") + parser.add_argument( + "-n", "--parallel", + action="store", + dest="parallel", + help="Parallelism") + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() check_args_and_update_paths(args) + parallel_args = "" + if args.parallel: + parallel_args += "--dist=loadfile" + parallel_args += "-n {}".format(args.parallel) + net = "" if not args.disable_net_host: net = "--net=host" @@ -188,7 +199,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, @@ -197,6 +208,7 @@ if __name__ == "__main__": cases_dir=args.cases_dir, src_dir=args.src_dir, env_tags=env_tags, + parallel=parallel_args, opts=' '.join(args.pytest_args), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, From ecf928737b8adfeff05329780016c735608e6ce0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 18:08:38 +0300 Subject: [PATCH 0033/1060] fix --- tests/integration/helpers/cluster.py | 2 +- tests/integration/helpers/hdfs_api.py | 7 +++++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index bc9850d5176..88169425856 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -867,7 +867,7 @@ class ClickHouseCluster: os.makedirs(self.hdfs_kerberized_logs_dir) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) - self.wait_hdfs_to_start(hdfs_api, timeout=30) + self.wait_hdfs_to_start(hdfs_api, timeout=300) if self.with_mongo and self.base_mongo_cmd: logging.debug('Setup Mongo') diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 60b8979fce2..9bd4bf66f92 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -116,7 +116,8 @@ class HDFSApi(object): return response_data.content def write_data(self, path, content): - logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}".format(self.protocol, self.host, self.proxy_port, path, self.user)) + logging.debug("write_data protocol:{} host:{} port:{} path: {} user:{}, principal:{}".format( + self.protocol, self.host, self.proxy_port, path, self.user, self.principal)) named_file = NamedTemporaryFile(mode='wb+') fpath = named_file.name if isinstance(content, str): @@ -126,7 +127,9 @@ class HDFSApi(object): if self.kerberized: self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) + self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, + hostname_override="kerberizedhdfs1", + principal=self.principal) response = requests.put( "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', From 34fd15ca407fbc9f2619ed2e9e10f176fed75c9f Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 18:10:38 +0300 Subject: [PATCH 0034/1060] trivial --- tests/integration/test_storage_kerberized_hdfs/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_kerberized_hdfs/test.py b/tests/integration/test_storage_kerberized_hdfs/test.py index bb4440a3e38..fa520752d1c 100644 --- a/tests/integration/test_storage_kerberized_hdfs/test.py +++ b/tests/integration/test_storage_kerberized_hdfs/test.py @@ -23,7 +23,7 @@ def started_cluster(): cluster.shutdown() def test_read_table(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) data = "1\tSerialize\t555.222\n2\tData\t777.333\n" hdfs_api.write_data("/simple_table_function", data) @@ -36,7 +36,7 @@ def test_read_table(started_cluster): def test_read_write_storage(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) node1.query("create table SimpleHDFSStorage2 (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage1', 'TSV')") node1.query("insert into SimpleHDFSStorage2 values (1, 'Mark', 72.53)") @@ -49,7 +49,7 @@ def test_read_write_storage(started_cluster): def test_write_storage_not_expired(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) node1.query("create table SimpleHDFSStorageNotExpired (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/simple_storage_not_expired', 'TSV')") @@ -64,7 +64,7 @@ def test_write_storage_not_expired(started_cluster): def test_two_users(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) node1.query("create table HDFSStorOne (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://kerberizedhdfs1:9010/storage_user_one', 'TSV')") node1.query("insert into HDFSStorOne values (1, 'Real', 86.00)") @@ -77,7 +77,7 @@ def test_two_users(started_cluster): select_read_2 = node1.query("select * from hdfs('hdfs://suser@kerberizedhdfs1:9010/storage_user_one', 'TSV', 'id UInt64, text String, number Float64')") def test_read_table_expired(started_cluster): - hdfs_api = started_cluster.make_hdfs_api() + hdfs_api = started_cluster.make_hdfs_api(kerberized=True) data = "1\tSerialize\t555.222\n2\tData\t777.333\n" hdfs_api.write_data("/simple_table_function_relogin", data) From b5f59446b792e47ca117ffb1947ec962847c09c2 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Feb 2021 18:39:27 +0300 Subject: [PATCH 0035/1060] fix kerberos --- tests/integration/helpers/hdfs_api.py | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 9bd4bf66f92..8a69f6baffd 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -56,7 +56,7 @@ class HDFSApi(object): if kerberized: self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override=self.host, principal=self.principal) + self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, hostname_override="kerberizedhdfs1", principal=self.principal) #principal=self.principal, #hostname_override=self.host, principal=self.principal) # , mutual_authentication=reqkerb.REQUIRED, force_preemptive=True) @@ -106,6 +106,7 @@ class HDFSApi(object): else: location = response.headers['Location'].replace("hdfs1:50075", "{}:{}".format(self.host, self.data_port)) logging.debug("redirected to {}".format(location)) + response_data = requests.get(location, headers={'host': 'localhost'}, verify=False, auth=self.kerberos_auth) if response_data.status_code != 200: @@ -125,12 +126,6 @@ class HDFSApi(object): named_file.write(content) named_file.flush() - if self.kerberized: - self._run_kinit() - self.kerberos_auth = reqkerb.HTTPKerberosAuth(mutual_authentication=reqkerb.DISABLED, - hostname_override="kerberizedhdfs1", - principal=self.principal) - response = requests.put( "{protocol}://{host}:{port}/webhdfs/v1{path}?op=CREATE".format(protocol=self.protocol, host='localhost', port=self.proxy_port, From fc89b4fd3528c6aa467e1c3f643c92c97d199153 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sat, 20 Feb 2021 17:59:39 +0300 Subject: [PATCH 0036/1060] better --- tests/integration/helpers/cluster.py | 2 +- tests/integration/runner | 2 +- tests/integration/test_storage_s3/test.py | 193 ++++++++---------- .../test_storage_s3/test_redirect.py | 125 ++++++++++++ 4 files changed, 214 insertions(+), 108 deletions(-) create mode 100644 tests/integration/test_storage_s3/test_redirect.py diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 668838afb5c..5443659eac9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -172,7 +172,7 @@ class ClickHouseCluster: self.minio_host = "minio1" self.minio_bucket = "root" self.minio_bucket_2 = "root2" - self.minio_port = 9001 + self.minio_port = get_open_port() self.minio_client = None # type: Minio self.minio_redirect_host = "proxy1" self.minio_redirect_port = 8080 diff --git a/tests/integration/runner b/tests/integration/runner index ead57913217..99987839f56 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -157,7 +157,7 @@ if __name__ == "__main__": parallel_args = "" if args.parallel: parallel_args += "--dist=loadfile" - parallel_args += "-n {}".format(args.parallel) + parallel_args += " -n {}".format(args.parallel) net = "" if not args.disable_net_host: diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 1f445feb5a0..b9de57366bf 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -14,9 +14,10 @@ from helpers.cluster import ClickHouseCluster, ClickHouseInstance logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) +MINIO_INTERNAL_PORT = 9001 # Creates S3 bucket for tests and allows anonymous read-write access to it. -def prepare_s3_bucket(cluster): +def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. bucket_read_write_policy = {"Version": "2012-10-17", "Statement": [ @@ -50,26 +51,26 @@ def prepare_s3_bucket(cluster): } ]} - minio_client = cluster.minio_client - minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) + minio_client = started_cluster.minio_client + minio_client.set_bucket_policy(started_cluster.minio_bucket, json.dumps(bucket_read_write_policy)) - cluster.minio_restricted_bucket = "{}-with-auth".format(cluster.minio_bucket) - if minio_client.bucket_exists(cluster.minio_restricted_bucket): - minio_client.remove_bucket(cluster.minio_restricted_bucket) + started_cluster.minio_restricted_bucket = "{}-with-auth".format(started_cluster.minio_bucket) + if minio_client.bucket_exists(started_cluster.minio_restricted_bucket): + minio_client.remove_bucket(started_cluster.minio_restricted_bucket) - minio_client.make_bucket(cluster.minio_restricted_bucket) + minio_client.make_bucket(started_cluster.minio_restricted_bucket) -def put_s3_file_content(cluster, bucket, filename, data): +def put_s3_file_content(started_cluster, bucket, filename, data): buf = io.BytesIO(data) - cluster.minio_client.put_object(bucket, filename, buf, len(data)) + started_cluster.minio_client.put_object(bucket, filename, buf, len(data)) # Returns content of given S3 file as string. -def get_s3_file_content(cluster, bucket, filename, decode=True): +def get_s3_file_content(started_cluster, bucket, filename, decode=True): # type: (ClickHouseCluster, str) -> str - data = cluster.minio_client.get_object(bucket, filename) + data = started_cluster.minio_client.get_object(bucket, filename) data_str = b"" for chunk in data.stream(): data_str += chunk @@ -79,7 +80,7 @@ def get_s3_file_content(cluster, bucket, filename, decode=True): @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("restricted_dummy", main_configs=["configs/config_for_test_remote_host_filter.xml"], @@ -115,17 +116,17 @@ def run_query(instance, query, stdin=None, settings=None): ("'minio','minio123',", True), ("'wrongid','wrongkey',", False) ]) -def test_put(cluster, maybe_auth, positive): +def test_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, filename, maybe_auth, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, maybe_auth, table_format, values) try: run_query(instance, put_query) @@ -134,18 +135,18 @@ def test_put(cluster, maybe_auth, positive): raise else: assert positive - assert values_csv == get_s3_file_content(cluster, bucket, filename) + assert values_csv == get_s3_file_content(started_cluster, bucket, filename) # Test put no data to S3. @pytest.mark.parametrize("auth", [ "'minio','minio123'," ]) -def test_empty_put(cluster, auth): +def test_empty_put(started_cluster, auth): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" create_empty_table_query = """ @@ -158,13 +159,13 @@ def test_empty_put(cluster, auth): filename = "empty_put_test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') select * from empty_table".format( - cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, auth, table_format) run_query(instance, put_query) try: run_query(instance, "select count(*) from s3('http://{}:{}/{}/{}', {}'CSV', '{}')".format( - cluster.minio_host, cluster.minio_port, bucket, filename, auth, table_format)) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, auth, table_format)) assert False, "Query should be failed." except helpers.client.QueryRuntimeException as e: @@ -177,15 +178,15 @@ def test_empty_put(cluster, auth): ("'minio','minio123',", True), ("'wrongid','wrongkey',", False) ]) -def test_put_csv(cluster, maybe_auth, positive): +def test_put_csv(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( - cluster.minio_host, cluster.minio_port, bucket, filename, maybe_auth, table_format) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, maybe_auth, table_format) csv_data = "8,9,16\n11,18,13\n22,14,2\n" try: @@ -195,27 +196,27 @@ def test_put_csv(cluster, maybe_auth, positive): raise else: assert positive - assert csv_data == get_s3_file_content(cluster, bucket, filename) + assert csv_data == get_s3_file_content(started_cluster, bucket, filename) # Test put and get with S3 server redirect. -def test_put_get_with_redirect(cluster): +def test_put_get_with_redirect(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" values_csv = "1,1,1\n1,1,1\n11,11,11\n" filename = "test.csv" query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format, values) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) run_query(instance, query) - assert values_csv == get_s3_file_content(cluster, bucket, filename) + assert values_csv == get_s3_file_content(started_cluster, bucket, filename) query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/{}', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format) stdout = run_query(instance, query) assert list(map(str.split, stdout.splitlines())) == [ @@ -226,23 +227,23 @@ def test_put_get_with_redirect(cluster): # Test put with restricted S3 server redirect. -def test_put_with_zero_redirect(cluster): +def test_put_with_zero_redirect(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["s3_max_redirects"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["s3_max_redirects"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" filename = "test.csv" # Should work without redirect query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, filename, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename, table_format, values) run_query(instance, query) # Should not work with redirect query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, table_format, values) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) exception_raised = False try: run_query(instance, query) @@ -253,11 +254,11 @@ def test_put_with_zero_redirect(cluster): assert exception_raised -def test_put_get_with_globs(cluster): +def test_put_get_with_globs(started_cluster): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" for i in range(10): @@ -266,11 +267,11 @@ def test_put_get_with_globs(cluster): max_path = max(path, max_path) values = "({},{},{})".format(i, j, i + j) query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, path, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, path, table_format, values) run_query(instance, query) query = "select sum(column1), sum(column2), sum(column3), min(_file), max(_path) from s3('http://{}:{}/{}/*_{{a,b,c,d}}/%3f.csv', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, table_format) assert run_query(instance, query).splitlines() == [ "450\t450\t900\t0.csv\t{bucket}/{max_path}".format(bucket=bucket, max_path=max_path)] @@ -281,11 +282,11 @@ def test_put_get_with_globs(cluster): # ("'minio','minio123',",True), Redirect with credentials not working with nginx. ("'wrongid','wrongkey',", False) ]) -def test_multipart_put(cluster, maybe_auth, positive): +def test_multipart_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None - bucket = cluster.minio_bucket if not maybe_auth else cluster.minio_restricted_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance + bucket = started_cluster.minio_bucket if not maybe_auth else started_cluster.minio_restricted_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" # Minimum size of part is 5 Mb for Minio. @@ -303,7 +304,7 @@ def test_multipart_put(cluster, maybe_auth, positive): filename = "test_multipart.csv" put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, filename, maybe_auth, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, maybe_auth, table_format) try: run_query(instance, put_query, stdin=csv_data, settings={'s3_min_upload_part_size': min_part_size_bytes, @@ -315,23 +316,23 @@ def test_multipart_put(cluster, maybe_auth, positive): assert positive # Use proxy access logs to count number of parts uploaded to Minio. - proxy_logs = cluster.get_container_logs("proxy1") # type: str + proxy_logs = started_cluster.get_container_logs("proxy1") # type: str assert proxy_logs.count("PUT /{}/{}".format(bucket, filename)) >= 2 - assert csv_data == get_s3_file_content(cluster, bucket, filename) + assert csv_data == get_s3_file_content(started_cluster, bucket, filename) -def test_remote_host_filter(cluster): - instance = cluster.instances["restricted_dummy"] +def test_remote_host_filter(started_cluster): + instance = started_cluster.instances["restricted_dummy"] format = "column1 UInt32, column2 UInt32, column3 UInt32" query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format( - "invalid_host", cluster.minio_port, cluster.minio_bucket, format) + "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format) assert "not allowed in config.xml" in instance.query_and_get_error(query) other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format( - "invalid_host", cluster.minio_port, cluster.minio_bucket, format, other_values) + "invalid_host", MINIO_INTERNAL_PORT, started_cluster.minio_bucket, format, other_values) assert "not allowed in config.xml" in instance.query_and_get_error(query) @@ -339,8 +340,8 @@ def test_remote_host_filter(cluster): "''", # 1 arguments "'','','','','',''" # 6 arguments ]) -def test_wrong_s3_syntax(cluster, s3_storage_args): - instance = cluster.instances["dummy"] # type: ClickHouseInstance +def test_wrong_s3_syntax(started_cluster, s3_storage_args): + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance expected_err_msg = "Code: 42" # NUMBER_OF_ARGUMENTS_DOESNT_MATCH query = "create table test_table_s3_syntax (id UInt32) ENGINE = S3({})".format(s3_storage_args) @@ -348,9 +349,9 @@ def test_wrong_s3_syntax(cluster, s3_storage_args): # https://en.wikipedia.org/wiki/One_Thousand_and_One_Nights -def test_s3_glob_scheherazade(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] # type: ClickHouseInstance +def test_s3_glob_scheherazade(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" max_path = "" values = "(1, 1, 1)" @@ -361,7 +362,7 @@ def test_s3_glob_scheherazade(cluster): for i in range(start, end): path = "night_{}/tale.csv".format(i) query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( - cluster.minio_host, cluster.minio_port, bucket, path, table_format, values) + started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, path, table_format, values) run_query(instance, query) jobs.append(threading.Thread(target=add_tales, args=(night, min(night + nights_per_job, 1001)))) @@ -371,20 +372,20 @@ def test_s3_glob_scheherazade(cluster): job.join() query = "select count(), sum(column1), sum(column2), sum(column3) from s3('http://{}:{}/{}/night_*/tale.csv', 'CSV', '{}')".format( - cluster.minio_redirect_host, cluster.minio_redirect_port, bucket, table_format) + started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, table_format) assert run_query(instance, query).splitlines() == ["1001\t1001\t1001\t1001"] -def run_s3_mock(cluster): +def run_s3_mock(started_cluster): logging.info("Starting s3 mock") - container_id = cluster.get_container_id('resolver') + container_id = started_cluster.get_container_id('resolver') current_dir = os.path.dirname(__file__) - cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") - cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) + started_cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") + started_cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) # Wait for S3 mock start for attempt in range(10): - ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), + ping_response = started_cluster.exec_in_container(started_cluster.get_container_id('resolver'), ["curl", "-s", "http://resolver:8080/"], nothrow=True) if ping_response != 'OK': if attempt == 9: @@ -397,25 +398,25 @@ def run_s3_mock(cluster): logging.info("S3 mock started") -def test_custom_auth_headers(cluster): +def test_custom_auth_headers(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=cluster.minio_restricted_bucket, + bucket=started_cluster.minio_restricted_bucket, file=filename, table_format=table_format) - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance result = run_query(instance, get_query) assert result == '1\t2\t3\n' -def test_custom_auth_headers_exclusion(cluster): +def test_custom_auth_headers_exclusion(started_cluster): table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" - get_query = f"SELECT * FROM s3('http://resolver:8080/{cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" + get_query = f"SELECT * FROM s3('http://resolver:8080/{started_cluster.minio_restricted_bucket}/restricteddirectory/{filename}', 'CSV', '{table_format}')" - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance with pytest.raises(helpers.client.QueryRuntimeException) as ei: result = run_query(instance, get_query) print(result) @@ -423,33 +424,13 @@ def test_custom_auth_headers_exclusion(cluster): assert ei.value.returncode == 243 assert '403 Forbidden' in ei.value.stderr - -def test_infinite_redirect(cluster): - bucket = "redirected" - table_format = "column1 UInt32, column2 UInt32, column3 UInt32" - filename = "test.csv" - get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( - bucket=bucket, - file=filename, - table_format=table_format) - instance = cluster.instances["dummy"] # type: ClickHouseInstance - exception_raised = False - try: - run_query(instance, get_query) - except Exception as e: - assert str(e).find("Too many redirects while trying to access") != -1 - exception_raised = True - finally: - assert exception_raised - - @pytest.mark.parametrize("extension,method", [ ("bin", "gzip"), ("gz", "auto") ]) -def test_storage_s3_get_gzip(cluster, extension, method): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_get_gzip(started_cluster, extension, method): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = f"test_get_gzip.{extension}" name = "test_get_gzip" data = [ @@ -474,11 +455,11 @@ def test_storage_s3_get_gzip(cluster, extension, method): compressed = gzip.GzipFile(fileobj=buf, mode="wb") compressed.write(("\n".join(data)).encode()) compressed.close() - put_s3_file_content(cluster, bucket, filename, buf.getvalue()) + put_s3_file_content(started_cluster, bucket, filename, buf.getvalue()) try: run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( - 'http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', + 'http://{started_cluster.minio_host}:{MINIO_INTERNAL_PORT}/{bucket}/{filename}', 'CSV', '{method}')""") @@ -488,9 +469,9 @@ def test_storage_s3_get_gzip(cluster, extension, method): run_query(instance, f"DROP TABLE {name}") -def test_storage_s3_put_uncompressed(cluster): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_put_uncompressed(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = "test_put_uncompressed.bin" name = "test_put_uncompressed" data = [ @@ -512,13 +493,13 @@ def test_storage_s3_put_uncompressed(cluster): ] try: run_query(instance, "CREATE TABLE {} (name String, id UInt32) ENGINE = S3('http://{}:{}/{}/{}', 'CSV')".format( - name, cluster.minio_host, cluster.minio_port, bucket, filename)) + name, started_cluster.minio_host, MINIO_INTERNAL_PORT, bucket, filename)) run_query(instance, "INSERT INTO {} VALUES ({})".format(name, "),(".join(data))) run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["753"] - uncompressed_content = get_s3_file_content(cluster, bucket, filename) + uncompressed_content = get_s3_file_content(started_cluster, bucket, filename) assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 753 finally: run_query(instance, f"DROP TABLE {name}") @@ -528,9 +509,9 @@ def test_storage_s3_put_uncompressed(cluster): ("bin", "gzip"), ("gz", "auto") ]) -def test_storage_s3_put_gzip(cluster, extension, method): - bucket = cluster.minio_bucket - instance = cluster.instances["dummy"] +def test_storage_s3_put_gzip(started_cluster, extension, method): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] filename = f"test_put_gzip.{extension}" name = "test_put_gzip" data = [ @@ -552,7 +533,7 @@ def test_storage_s3_put_gzip(cluster, extension, method): ] try: run_query(instance, f"""CREATE TABLE {name} (name String, id UInt32) ENGINE = S3( - 'http://{cluster.minio_host}:{cluster.minio_port}/{bucket}/{filename}', + 'http://{started_cluster.minio_host}:{MINIO_INTERNAL_PORT}/{bucket}/{filename}', 'CSV', '{method}')""") @@ -560,7 +541,7 @@ def test_storage_s3_put_gzip(cluster, extension, method): run_query(instance, "SELECT sum(id) FROM {}".format(name)).splitlines() == ["708"] - buf = io.BytesIO(get_s3_file_content(cluster, bucket, filename, decode=False)) + buf = io.BytesIO(get_s3_file_content(started_cluster, bucket, filename, decode=False)) f = gzip.GzipFile(fileobj=buf, mode="rb") uncompressed_content = f.read().decode() assert sum([ int(i.split(',')[1]) for i in uncompressed_content.splitlines() ]) == 708 diff --git a/tests/integration/test_storage_s3/test_redirect.py b/tests/integration/test_storage_s3/test_redirect.py new file mode 100644 index 00000000000..ab677f46d9a --- /dev/null +++ b/tests/integration/test_storage_s3/test_redirect.py @@ -0,0 +1,125 @@ +import gzip +import json +import logging +import os +import io +import random +import threading +import time + +import helpers.client +import pytest +from helpers.cluster import ClickHouseCluster, ClickHouseInstance + +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + +# Creates S3 bucket for tests and allows anonymous read-write access to it. +def prepare_s3_bucket(cluster): + # Allows read-write access for bucket without authorization. + bucket_read_write_policy = {"Version": "2012-10-17", + "Statement": [ + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetBucketLocation", + "Resource": "arn:aws:s3:::root" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:ListBucket", + "Resource": "arn:aws:s3:::root" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:GetObject", + "Resource": "arn:aws:s3:::root/*" + }, + { + "Sid": "", + "Effect": "Allow", + "Principal": {"AWS": "*"}, + "Action": "s3:PutObject", + "Resource": "arn:aws:s3:::root/*" + } + ]} + + minio_client = cluster.minio_client + minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy)) + + cluster.minio_restricted_bucket = "{}-with-auth".format(cluster.minio_bucket) + if minio_client.bucket_exists(cluster.minio_restricted_bucket): + minio_client.remove_bucket(cluster.minio_restricted_bucket) + + minio_client.make_bucket(cluster.minio_restricted_bucket) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__, name="redirect") + cluster.add_instance("dummy", with_minio=True, main_configs=["configs/defaultS3.xml"]) + logging.info("Starting cluster...") + cluster.start() + logging.info("Cluster started") + + prepare_s3_bucket(cluster) + logging.info("S3 bucket created") + run_s3_mock(cluster) + + yield cluster + finally: + cluster.shutdown() + +def run_query(instance, query, stdin=None, settings=None): + # type: (ClickHouseInstance, str, object, dict) -> str + + logging.info("Running query '{}'...".format(query)) + result = instance.query(query, stdin=stdin, settings=settings) + logging.info("Query finished") + + return result + +def run_s3_mock(cluster): + logging.info("Starting s3 mock") + container_id = cluster.get_container_id('resolver') + current_dir = os.path.dirname(__file__) + cluster.copy_file_to_container(container_id, os.path.join(current_dir, "s3_mock", "mock_s3.py"), "mock_s3.py") + cluster.exec_in_container(container_id, ["python", "mock_s3.py"], detach=True) + + # Wait for S3 mock start + for attempt in range(10): + ping_response = cluster.exec_in_container(cluster.get_container_id('resolver'), + ["curl", "-s", "http://resolver:8080/"], nothrow=True) + if ping_response != 'OK': + if attempt == 9: + assert ping_response == 'OK', 'Expected "OK", but got "{}"'.format(ping_response) + else: + time.sleep(1) + else: + break + + logging.info("S3 mock started") + +def test_infinite_redirect(started_cluster): + bucket = "redirected" + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" + filename = "test.csv" + get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( + bucket=bucket, + file=filename, + table_format=table_format) + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance + exception_raised = False + try: + run_query(instance, get_query) + except Exception as e: + assert str(e).find("Too many redirects while trying to access") != -1 + exception_raised = True + finally: + assert exception_raised \ No newline at end of file From 775f31f65eb932a10b3f90b87c7143f03e86c831 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sat, 20 Feb 2021 18:23:09 +0300 Subject: [PATCH 0037/1060] fix --- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/runner | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5443659eac9..872d34d657c 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -589,10 +589,10 @@ class ClickHouseCluster: image_id = container_info.get('Image') image_info = self.docker_client.api.inspect_image(image_id) logging.debug(("Command failed in container {}: ".format(container_id))) - pprint.plogging.debug(container_info) + pprint.pprint(container_info) logging.debug("") logging.debug(("Container {} uses image {}: ".format(container_id, image_id))) - pprint.plogging.debug(image_info) + pprint.pprint(image_info) logging.debug("") message = 'Cmd "{}" failed in container {}. Return code {}. Output: {}'.format(' '.join(cmd), container_id, exit_code, output) diff --git a/tests/integration/runner b/tests/integration/runner index 99987839f56..f03554f179d 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -199,7 +199,8 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( + --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=120 \ + {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, bin=args.binary, From 905cc7a2bd96540114e6b9de1da2361a6d9ce871 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Feb 2021 14:46:58 +0300 Subject: [PATCH 0038/1060] fixes --- tests/integration/helpers/cluster.py | 87 ++++++++++++------- tests/integration/runner | 12 +-- .../test_alter_on_mixed_type_cluster/test.py | 1 - .../test_format_avro_confluent/test.py | 11 +-- tests/integration/test_s3_with_https/test.py | 2 +- tests/integration/test_storage_kafka/test.py | 40 ++------- .../test_storage_kerberized_kafka/test.py | 38 +------- 7 files changed, 79 insertions(+), 112 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 872d34d657c..d69321268e5 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -15,17 +15,19 @@ import traceback import urllib.parse import cassandra.cluster -import docker import psycopg2 import pymongo import pymysql import requests +from confluent_kafka.avro.cached_schema_registry_client import \ + CachedSchemaRegistryClient from dict2xml import dict2xml -from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient from kazoo.client import KazooClient from kazoo.exceptions import KazooException from minio import Minio +import docker + from .client import Client from .hdfs_api import HDFSApi @@ -38,7 +40,7 @@ SANITIZER_SIGN = "==================" # to create docker-compose env file def _create_env_file(path, variables): - logging.debug("Env {} stored in {}".format(variables, path)) + logging.debug(f"Env {variables} stored in {path}") with open(path, 'w') as f: for var, value in list(variables.items()): f.write("=".join([var, value]) + "\n") @@ -48,20 +50,19 @@ def run_and_check(args, env=None, shell=False): res = subprocess.run(args, stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=env, shell=shell) if res.returncode != 0: # check_call(...) from subprocess does not print stderr, so we do it manually - logging.debug('Stderr:\n{}\n'.format(res.stderr.decode('utf-8'))) - logging.debug('Stdout:\n{}\n'.format(res.stdout.decode('utf-8'))) - logging.debug('Env:\n{}\n'.format(env)) - raise Exception('Command {} return non-zero code {}: {}'.format(args, res.returncode, res.stderr.decode('utf-8'))) + logging.debug(f"Stderr:\n{res.stderr.decode('utf-8')}\n") + logging.debug(f"Stdout:\n{res.stdout.decode('utf-8')}\n") + logging.debug(f"Env:\n{env}\n") + raise Exception(f"Command {args} return non-zero code {res.returncode}: {res.stderr.decode('utf-8')}") # Based on https://stackoverflow.com/questions/2838244/get-open-tcp-port-in-python/2838309#2838309 def get_open_port(): - import socket - s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - s.bind(("",0)) - s.listen(1) - port = s.getsockname()[1] - s.close() - return port + s = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + s.bind(("",0)) + s.listen(1) + port = s.getsockname()[1] + s.close() + return port def subprocess_check_call(args): # Uncomment for debugging @@ -93,10 +94,23 @@ def get_docker_compose_path(): if os.path.exists(os.path.dirname('/compose/')): return os.path.dirname('/compose/') # default in docker runner container else: - logging.debug(("Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {}".format(LOCAL_DOCKER_COMPOSE_DIR))) + logging.debug(f"Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {LOCAL_DOCKER_COMPOSE_DIR}") return LOCAL_DOCKER_COMPOSE_DIR +def check_kafka_is_available(kafka_id, kafka_port): + p = subprocess.Popen(('docker', + 'exec', + '-i', + kafka_id, + '/usr/bin/kafka-broker-api-versions', + '--bootstrap-server', + f'INSIDE://localhost:{kafka_port}'), + stdout=subprocess.PIPE, stderr=subprocess.PIPE) + p.communicate() + return p.returncode == 0 + + class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -202,7 +216,6 @@ class ClickHouseCluster: # available when with_kerberozed_kafka == True self.kerberized_kafka_host = "kerberized_kafka1" self.kerberized_kafka_port = get_open_port() - self.kerberized_kafka_docker_id = None self.kerberized_kafka_docker_id = self.get_instance_docker_id(self.kerberized_kafka_host) # available when with_mongo == True @@ -238,7 +251,7 @@ class ClickHouseCluster: self.docker_client = None self.is_up = False - logging.debug("CLUSTER INIT base_config_dir:{}".format(self.base_config_dir)) + logging.debug(f"CLUSTER INIT base_config_dir:{self.base_config_dir}") def get_client_cmd(self): cmd = self.client_bin_path @@ -299,7 +312,7 @@ class ClickHouseCluster: return self.base_kerberized_hdfs_cmd def setup_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): - self.with_redis = True + self.with_kafka = True env_variables['KAFKA_HOST'] = self.kafka_host env_variables['KAFKA_EXTERNAL_PORT'] = str(self.kafka_port) env_variables['SCHEMA_REGISTRY_EXTERNAL_PORT'] = str(self.schema_registry_port) @@ -310,7 +323,7 @@ class ClickHouseCluster: return self.base_kafka_cmd def setup_kerberized_kafka_cmd(self, instance, env_variables, docker_compose_yml_dir): - self.with_redis = True + self.with_kerberized_kafka = True env_variables['KERBERIZED_KAFKA_DIR'] = instance.path + '/' env_variables['KERBERIZED_KAFKA_HOST'] = self.kerberized_kafka_host env_variables['KERBERIZED_KAFKA_EXTERNAL_PORT'] = str(self.kerberized_kafka_port) @@ -694,13 +707,25 @@ class ClickHouseCluster: proxy_port=self.hdfs_kerberized_name_port, data_port=self.hdfs_kerberized_data_port, hdfs_ip=hdfs_ip, - kdc_ip=kdc_ip) - + kdc_ip=kdc_ip) else: logging.debug("Create HDFSApi host={}".format("localhost")) hdfs_api = HDFSApi(user="root", host="localhost", data_port=self.hdfs_data_port, proxy_port=self.hdfs_name_port) return hdfs_api + def wait_kafka_is_available(self, kafka_docker_id, kafka_port, max_retries=50): + retries = 0 + while True: + if check_kafka_is_available(kafka_docker_id, kafka_port): + break + else: + retries += 1 + if retries > max_retries: + raise Exception("Kafka is not available") + logging.debug("Waiting for Kafka to start up") + time.sleep(1) + + def wait_hdfs_to_start(self, hdfs_api, timeout=60): start = time.time() while time.time() - start < timeout: @@ -722,13 +747,14 @@ class ClickHouseCluster: while time.time() - start < timeout: try: connection.list_database_names() - logging.debug("Connected to Mongo dbs:", connection.database_names()) + logging.debug("Connected to Mongo dbs: {}", connection.database_names()) return except Exception as ex: logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): + os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', secret_key='minio123', @@ -757,7 +783,7 @@ class ClickHouseCluster: raise Exception("Can't wait Minio to start") def wait_schema_registry_to_start(self, timeout=10): - sr_client = CachedSchemaRegistryClient('http://localhost:{}'.format(cluster.schema_registry_port)) + sr_client = CachedSchemaRegistryClient('http://localhost:{}'.format(self.schema_registry_port)) start = time.time() while time.time() - start < timeout: try: @@ -845,11 +871,13 @@ class ClickHouseCluster: if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') subprocess_check_call(self.base_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.wait_kafka_is_available(self.kafka_docker_id, self.kafka_port) self.wait_schema_registry_to_start(30) if self.with_kerberized_kafka and self.base_kerberized_kafka_cmd: logging.debug('Setup kerberized kafka') run_and_check(self.base_kerberized_kafka_cmd + common_opts + ['--renew-anon-volumes']) + self.wait_kafka_is_available(self.kerberized_kafka_docker_id, self.kerberized_kafka_port, 100) if self.with_rabbitmq and self.base_rabbitmq_cmd: subprocess_check_call(self.base_rabbitmq_cmd + common_opts + ['--renew-anon-volumes']) @@ -885,7 +913,7 @@ class ClickHouseCluster: if self.minio_certs_dir is None: os.mkdir(os.path.join(self.minio_dir, 'certs')) else: - shutil.copytree(self.minio_certs_dir, os.path.join(self.minio_dir, 'certs')) + shutil.copytree(os.path.join(self.base_dir, self.minio_certs_dir), os.path.join(self.minio_dir, 'certs')) minio_start_cmd = self.base_minio_cmd + common_opts @@ -1307,6 +1335,7 @@ class ClickHouseInstance: user='root') self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid())) from helpers.test_tools import assert_eq_with_retry + # wait start assert_eq_with_retry(self, "select 1", "1", retry_count=retries) @@ -1462,7 +1491,7 @@ class ClickHouseInstance: shutil.copytree(self.kerberos_secrets_dir, p.abspath(p.join(self.path, 'secrets'))) # Copy config.d configs - logging.debug("Copy custom test config files {} to {}".format(self.custom_main_config_paths, self.config_d_dir)) + logging.debug(f"Copy custom test config files {self.custom_main_config_paths} to {self.config_d_dir}") for path in self.custom_main_config_paths: shutil.copy(path, self.config_d_dir) @@ -1475,16 +1504,16 @@ class ClickHouseInstance: shutil.copy(path, dictionaries_dir) db_dir = p.abspath(p.join(self.path, 'database')) - logging.debug("Setup database dir {}".format(db_dir)) + logging.debug(f"Setup database dir {db_dir}") if self.clickhouse_path_dir is not None: - logging.debug("Database files taken from {}".format(self.clickhouse_path_dir)) + logging.debug(f"Database files taken from {self.clickhouse_path_dir}") shutil.copytree(self.clickhouse_path_dir, db_dir) - logging.debug("Database copied from {} to {}".format(self.clickhouse_path_dir, db_dir)) + logging.debug(f"Database copied from {self.clickhouse_path_dir} to {db_dir}") else: os.mkdir(db_dir) logs_dir = p.abspath(p.join(self.path, 'logs')) - logging.debug("Setup logs dir {}".format(logs_dir)) + logging.debug(f"Setup logs dir {logs_dir}") os.mkdir(logs_dir) depends_on = [] diff --git a/tests/integration/runner b/tests/integration/runner index f03554f179d..af29466e4fa 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -27,7 +27,7 @@ def check_args_and_update_paths(args): else: CLICKHOUSE_ROOT = args.clickhouse_root else: - logging.info("ClickHouse root is not set. Will use {}".format(DEFAULT_CLICKHOUSE_ROOT)) + logging.info(f"ClickHouse root is not set. Will use {DEFAULT_CLICKHOUSE_ROOT}") CLICKHOUSE_ROOT = DEFAULT_CLICKHOUSE_ROOT if not os.path.isabs(args.binary): @@ -43,23 +43,23 @@ def check_args_and_update_paths(args): args.base_configs_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.base_configs_dir)) else: args.base_configs_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, CONFIG_DIR_IN_REPO)) - logging.info("Base configs dir is not set. Will use {}".format(args.base_configs_dir)) + logging.info(f"Base configs dir is not set. Will use {args.base_configs_dir}") if args.cases_dir: if not os.path.isabs(args.cases_dir): args.cases_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.cases_dir)) else: args.cases_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, INTERGATION_DIR_IN_REPO)) - logging.info("Cases dir is not set. Will use {}".format(args.cases_dir)) + logging.info(f"Cases dir is not set. Will use {args.cases_dir}") if args.src_dir: if not os.path.isabs(args.src_dir): args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.src_dir)) else: args.src_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, SRC_DIR_IN_REPO)) - logging.info("src dir is not set. Will use {}".format(args.src_dir)) + logging.info(f"src dir is not set. Will use {args.src_dir}") - logging.info("base_configs_dir: {}, binary: {}, cases_dir: {} ".format(args.base_configs_dir, args.binary, args.cases_dir)) + logging.info(f"base_configs_dir: {args.base_configs_dir}, binary: {args.binary}, cases_dir: {args.cases_dir} ") for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: if not os.path.exists(path): @@ -183,7 +183,7 @@ if __name__ == "__main__": elif image == "yandex/clickhouse-kerberos-kdc": env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: - logging.info("Unknown image {}".format(image)) + logging.info(f"Unknown image {image}") # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 269072f239a..5e516176c0b 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -1,7 +1,6 @@ import pytest from helpers.cluster import ClickHouseCluster -from helpers.test_tools import assert_eq_with_retry cluster = ClickHouseCluster(__file__) diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index ec2ed875075..49bc7baeb39 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -3,6 +3,7 @@ import logging import avro.schema import pytest +from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegistryClient from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance @@ -11,7 +12,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) @pytest.fixture(scope="module") -def cluster(): +def started_cluster(): try: cluster = ClickHouseCluster(__file__) cluster.add_instance("dummy", with_kafka=True) @@ -37,10 +38,10 @@ def run_query(instance, query, data=None, settings=None): return result -def test_select(cluster): +def test_select(started_cluster): # type: (ClickHouseCluster) -> None - schema_registry_client = cluster.schema_registry_client + schema_registry_client = CachedSchemaRegistryClient('http://localhost:{}'.format(started_cluster.schema_registry_port)) serializer = MessageSerializer(schema_registry_client) schema = avro.schema.make_avsc_object({ @@ -62,9 +63,9 @@ def test_select(cluster): buf.write(message) data = buf.getvalue() - instance = cluster.instances["dummy"] # type: ClickHouseInstance + instance = started_cluster.instances["dummy"] # type: ClickHouseInstance schema_registry_url = "http://{}:{}".format( - cluster.schema_registry_host, + started_cluster.schema_registry_host, 8081 ) diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 58e3b3c2a3b..1a5c6e2ce12 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -20,7 +20,7 @@ def cluster(): cluster = ClickHouseCluster(__file__) cluster.add_instance("node", main_configs=["configs/config.d/storage_conf.xml", "configs/config.d/log_conf.xml", "configs/config.d/ssl.xml"], with_minio=True, - minio_certs_dir='minio_certs') + minio_certs_dir="minio_certs") logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index da5826665e5..98c3855ddd9 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -24,13 +24,12 @@ from kafka.admin import NewTopic from kafka.protocol.admin import DescribeGroupsRequest_v1 from kafka.protocol.group import MemberAssignment -""" -protoc --version -libprotoc 3.0.0 -# to create kafka_pb2.py -protoc --python_out=. kafka.proto -""" +# protoc --version +# libprotoc 3.0.0 +# # to create kafka_pb2.py +# protoc --python_out=. kafka.proto + from . import kafka_pb2 from . import social_pb2 @@ -51,36 +50,8 @@ instance = cluster.add_instance('instance', "kafka_client_id":"instance", "kafka_format_json_each_row":"JSONEachRow"}, clickhouse_path_dir='clickhouse_path') -kafka_id = cluster.kafka_docker_id -# Helpers - -def check_kafka_is_available(): - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), - stdout=subprocess.PIPE, stderr=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise Exception("Kafka is not available") - logging.debug("Waiting for Kafka to start up") - time.sleep(1) - def get_kafka_producer(port, serializer): errors = [] for _ in range(15): @@ -645,7 +616,6 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() # logging.debug("kafka is available - running test") yield # run test diff --git a/tests/integration/test_storage_kerberized_kafka/test.py b/tests/integration/test_storage_kerberized_kafka/test.py index e21d18c2f49..94d2156c2ff 100644 --- a/tests/integration/test_storage_kerberized_kafka/test.py +++ b/tests/integration/test_storage_kerberized_kafka/test.py @@ -3,6 +3,7 @@ import random import threading import time import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV @@ -24,37 +25,6 @@ instance = cluster.add_instance('instance', with_kerberized_kafka=True, clickhouse_path_dir="clickhouse_path" ) -kafka_id = cluster.kerberized_kafka_docker_id - -# Helpers - -def check_kafka_is_available(): - - # plaintext - p = subprocess.Popen(('docker', - 'exec', - '-i', - kafka_id, - '/usr/bin/kafka-broker-api-versions', - '--bootstrap-server', - 'INSIDE://{}:{}'.format("localhost", cluster.kafka_port)), - stdout=subprocess.PIPE) - p.communicate() - return p.returncode == 0 - - -def wait_kafka_is_available(max_retries=50): - retries = 0 - while True: - if check_kafka_is_available(): - break - else: - retries += 1 - if retries > max_retries: - raise Exception("Kafka is not available") - print("Waiting for Kafka to start up") - time.sleep(1) - def producer_serializer(x): return x.encode() if isinstance(x, str) else x @@ -73,8 +43,8 @@ def get_kafka_producer(port, serializer): raise Exception("Connection not establised, {}".format(errors)) def kafka_produce(kafka_cluster, topic, messages, timestamp=None): - logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) - producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer) + logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kerberized_kafka_port, topic)) + producer = get_kafka_producer(kafka_cluster.kerberized_kafka_port, producer_serializer) for message in messages: producer.send(topic=topic, value=message, timestamp_ms=timestamp) producer.flush() @@ -94,8 +64,6 @@ def kafka_cluster(): @pytest.fixture(autouse=True) def kafka_setup_teardown(): instance.query('DROP DATABASE IF EXISTS test; CREATE DATABASE test;') - wait_kafka_is_available() - print("kafka is available - running test") yield # run test # Tests From 74f6ab798f71b015dc5fd2bd06af9f9c360f0d60 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 14:52:39 +0300 Subject: [PATCH 0039/1060] Add __restrict modifier to AggregateFunctionSequenceNextNode --- .../AggregateFunctionSequenceNextNode.h | 22 +++++++++---------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index be672974ac6..c406fee557e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -177,7 +177,7 @@ public: [[maybe_unused]] auto a = new (place) Data; } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { Node * node = Node::allocate(*columns[1], row_num, arena); @@ -202,7 +202,7 @@ public: data(place).value.push_back(node, arena); } - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { if (data(rhs).value.empty()) return; @@ -241,7 +241,7 @@ public: data(place).sorted = true; } - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { writeBinary(data(place).sorted, buf); @@ -251,7 +251,7 @@ public: node->write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override { readBinary(data(place).sorted, buf); @@ -305,7 +305,7 @@ public: return 0; } - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { auto & value = data(place).value; @@ -359,7 +359,7 @@ public: a.value.push_back(v->clone(arena), arena); } - void create(AggregateDataPtr place) const override + void create(AggregateDataPtr __restrict place) const override { [[maybe_unused]] auto a = new (place) Data; } @@ -369,7 +369,7 @@ public: return Descending ? lhs_timestamp < rhs_timestamp : lhs_timestamp > rhs_timestamp; } - void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override + void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { bool is_first = true; auto & value = data(place).value; @@ -394,7 +394,7 @@ public: } } - void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override + void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override { auto & a = data(place).value; auto & b = data(rhs).value; @@ -415,7 +415,7 @@ public: } } - void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override + void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { writeBinary(data(place).sorted, buf); @@ -425,7 +425,7 @@ public: node->write(buf); } - void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override + void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override { readBinary(data(place).sorted, buf); @@ -442,7 +442,7 @@ public: value[i] = Node::read(buf, arena); } - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena *) const override + void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override { auto & value = data(place).value; From 6f28b6c4a6a61824ff97df55b0f9f880f81b5d7f Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 24 Feb 2021 14:53:24 +0300 Subject: [PATCH 0040/1060] Minor changes in AggregateFunctionSequenceNextNode --- .../parametric-functions.md | 16 +++--- .../AggregateFunctionSequenceNextNode.cpp | 50 +++++++++++-------- .../AggregateFunctionSequenceNextNode.h | 38 +++++++------- .../0_stateless/01656_sequence_next_node.sql | 2 +- 4 files changed, 57 insertions(+), 49 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 3bb22461e5d..96e0013db19 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -513,21 +513,21 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode([descending_order])(timestamp, event_column, event1, event2, event3, ...) +sequenceNextNode(descending_order)(timestamp, event_column, event1, event2, event3, ...) ``` **Parameters** - `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. -- `timestamp` — Name of the column containing the timestamp. Data types supported: Date, DateTime and other unsigned integer types. -- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: String and Nullable(String) -- `cond` — Conditions describing the chain of events. UInt8. + +**Arguments** +- `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. +- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `cond` — Conditions describing the chain of events. `UInt8` **Returned value** -Nullable(String). - -- event_column[next_index] - if the pattern is matched and next value exists. -- null - if the pattern isn’t matched or next value doesn't exist. +- `event_column[next_index]` - if the pattern is matched and next value exists. +- `NULL` - if the pattern isn’t matched or next value doesn't exist. Type: `Nullable(String)`. diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index af90c80de61..1513e5b3213 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -6,7 +6,6 @@ #include #include #include -#include "registerAggregateFunctions.h" namespace DB @@ -22,36 +21,37 @@ namespace ErrorCodes namespace { -template +template inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) { if (argument_types.size() == 2) { // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. if (descending_order) - return std::make_shared>(data_type); + return std::make_shared>(data_type); else - return std::make_shared>(data_type); + return std::make_shared>(data_type); } else { if (descending_order) - return std::make_shared>(data_type, argument_types); + return std::make_shared>(data_type, argument_types); else - return std::make_shared>(data_type, argument_types); + return std::make_shared>(data_type, argument_types); } } -template -AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, const DataTypes & argument_types, const Array & parameters) +AggregateFunctionPtr +createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, const DataTypes & argument_types, const Array & parameters) { bool descending_order = false; if (parameters.size() == 1) { auto type = parameters[0].getType(); - if (type != Field::Types::Int64 && type != Field::Types::UInt64) - throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); + bool is_correct_type = type == Field::Types::Int64 || type == Field::Types::UInt64; + if (!is_correct_type || (parameters[0].get() != 0 && parameters[0].get() != 1)) + throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); descending_order = parameters[0].get(); } @@ -60,9 +60,13 @@ AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & nam ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (argument_types.size() < 2) - throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > MaxArgs) - throw Exception("Aggregate function " + name + " requires at most 34(timestamp, value_column, 31 events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Aggregate function " + name + " requires at least two arguments.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > max_args + 2) + throw Exception("Aggregate function " + name + " requires at most " + + std::to_string(max_args + 2) + + " (timestamp, value_column, " + std::to_string(max_args) + " events) arguments.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); for (const auto i : ext::range(2, argument_types.size())) { @@ -73,15 +77,11 @@ AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & nam } if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) - throw Exception{"Illegal type " + argument_types.front().get()->getName() + throw Exception{"Illegal type " + argument_types[1].get()->getName() + " of second argument of aggregate function " + name + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - DataTypePtr data_type; - if (typeid_cast(argument_types[1].get())) - data_type = argument_types[1]; - else - data_type = std::make_shared(argument_types[1]); + DataTypePtr data_type = makeNullable(argument_types[1]); WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) @@ -102,13 +102,21 @@ AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & nam ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } +auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_args) +{ + return [max_args](const std::string & name, const DataTypes & argument_types, const Array & parameters) + { + return createAggregateFunctionSequenceNode(name, max_args, argument_types, parameters); + }; +} + } void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNode<2 + 31>, properties }); - factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNode<2>, properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(31), properties }); + factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); } } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c406fee557e..69381b2aa95 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -32,10 +32,10 @@ namespace DB template struct NodeBase { - UInt64 size; // size of payload + UInt64 size; /// size of payload DataTypeDateTime::FieldType event_time; - UInt32 events_bitset; // Bitsets of UInt32 are easy to compare. (< operator on bitsets) + UInt32 events_bitset; /// Bitsets of UInt32 are easy to compare. (< operator on bitsets) char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -95,7 +95,7 @@ struct NodeString : public NodeBase }; /// TODO : Expends SequenceNextNodeGeneralData to support other types -template +template struct SequenceNextNodeGeneralData { using Allocator = MixedAlignedArenaAllocator; @@ -130,9 +130,9 @@ struct SequenceNextNodeGeneralData /// Implementation of sequenceNextNode template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { - using Data = SequenceNextNodeGeneralData; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } @@ -142,7 +142,7 @@ class SequenceNextNodeImpl final public: SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) , events_size(arguments.size() - 2) @@ -158,11 +158,11 @@ public: const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties & /*properties*/) const override { - // This aggregate function sets insertion_requires_nullable_column on. - // Even though some values are mapped to aggregating key, it could return nulls for the below case. - // aggregated events: [A -> B -> C] - // events to find: [C -> D] - // [C -> D] is not matched to 'A -> B -> C' so that it returns null. + /// This aggregate function sets insertion_requires_nullable_column on. + /// Even though some values are mapped to aggregating key, it could return nulls for the below case. + /// aggregated events: [A -> B -> C] + /// events to find: [C -> D] + /// [C -> D] is not matched to 'A -> B -> C' so that it returns null. return std::make_shared>(nested_function, arguments, params); } @@ -174,7 +174,7 @@ public: void create(AggregateDataPtr place) const override { - [[maybe_unused]] auto a = new (place) Data; + new (place) Data; } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override @@ -218,8 +218,8 @@ public: for (UInt64 i = 0; i < new_elems; ++i) a.push_back(b[i]->clone(arena), arena); - /// either sort whole container or do so partially merging ranges afterwards - using Comparator = typename SequenceNextNodeGeneralData::Comparator; + /// Either sort whole container or do so partially merging ranges afterwards + using Comparator = typename SequenceNextNodeGeneralData::Comparator; if (!data(place).sorted && !data(rhs).sorted) std::stable_sort(std::begin(a), std::end(a), Comparator{}); @@ -326,17 +326,17 @@ public: /// Implementation of sequenceFirstNode template class SequenceFirstNodeImpl final - : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> + : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> { - using Data = SequenceNextNodeGeneralData; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } DataTypePtr & data_type; public: - SequenceFirstNodeImpl(const DataTypePtr & data_type_) - : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( + explicit SequenceFirstNodeImpl(const DataTypePtr & data_type_) + : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) { @@ -361,7 +361,7 @@ public: void create(AggregateDataPtr __restrict place) const override { - [[maybe_unused]] auto a = new (place) Data; + new (place) Data; } bool compare(const T lhs_timestamp, const T rhs_timestamp) const diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 9af59d5c8e2..c2270bc10ac 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -1,6 +1,6 @@ DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; -CREATE TABLE iF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; +CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); From bc8db50142c02f53cabfc0a2010e72b267ed941d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Feb 2021 18:14:21 +0300 Subject: [PATCH 0041/1060] no python3 in CI by defalult --- tests/integration/runner | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/runner b/tests/integration/runner index af29466e4fa..5ad37726821 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -27,7 +27,7 @@ def check_args_and_update_paths(args): else: CLICKHOUSE_ROOT = args.clickhouse_root else: - logging.info(f"ClickHouse root is not set. Will use {DEFAULT_CLICKHOUSE_ROOT}") + logging.info("ClickHouse root is not set. Will use %s" % (DEFAULT_CLICKHOUSE_ROOT)) CLICKHOUSE_ROOT = DEFAULT_CLICKHOUSE_ROOT if not os.path.isabs(args.binary): @@ -43,23 +43,23 @@ def check_args_and_update_paths(args): args.base_configs_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.base_configs_dir)) else: args.base_configs_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, CONFIG_DIR_IN_REPO)) - logging.info(f"Base configs dir is not set. Will use {args.base_configs_dir}") + logging.info("Base configs dir is not set. Will use %s" % (args.base_configs_dir)) if args.cases_dir: if not os.path.isabs(args.cases_dir): args.cases_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.cases_dir)) else: args.cases_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, INTERGATION_DIR_IN_REPO)) - logging.info(f"Cases dir is not set. Will use {args.cases_dir}") + logging.info("Cases dir is not set. Will use %s" % (args.cases_dir)) if args.src_dir: if not os.path.isabs(args.src_dir): args.src_dir = os.path.abspath(os.path.join(CURRENT_WORK_DIR, args.src_dir)) else: args.src_dir = os.path.abspath(os.path.join(CLICKHOUSE_ROOT, SRC_DIR_IN_REPO)) - logging.info(f"src dir is not set. Will use {args.src_dir}") + logging.info("src dir is not set. Will use %s" % (args.src_dir)) - logging.info(f"base_configs_dir: {args.base_configs_dir}, binary: {args.binary}, cases_dir: {args.cases_dir} ") + logging.info("base_configs_dir: %s , binary: %s, cases_dir: %s " % (args.base_configs_dir, args.binary, args.cases_dir)) for path in [args.binary, args.bridge_binary, args.base_configs_dir, args.cases_dir, CLICKHOUSE_ROOT]: if not os.path.exists(path): @@ -183,7 +183,7 @@ if __name__ == "__main__": elif image == "yandex/clickhouse-kerberos-kdc": env_tags += "-e {}={}".format("DOCKER_KERBEROS_KDC_TAG", tag) else: - logging.info(f"Unknown image {image}") + logging.info("Unknown image %s" % (image)) # create named volume which will be used inside to store images and other docker related files, # to avoid redownloading it every time From f8f475e83fc5a08a113417ff385eaacd1292ec2b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Feb 2021 21:24:16 +0300 Subject: [PATCH 0042/1060] fix --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e2e57e92187..0347eee1719 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -754,7 +754,7 @@ class ClickHouseCluster: time.sleep(1) def wait_minio_to_start(self, timeout=30, secure=False): - os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_certs_dir, 'public.crt') + os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', secret_key='minio123', From 7dc4b6d58e26462ffa5ff19ea9f86e887c7bc5eb Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 25 Feb 2021 19:31:20 +0900 Subject: [PATCH 0043/1060] Add comments and fix AggregateFunctionSequenceNextNode --- .../AggregateFunctionSequenceNextNode.h | 43 ++++++++++++++++--- 1 file changed, 36 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 69381b2aa95..9bde44be06e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,6 +27,19 @@ namespace DB { +/** + * When sorting the list of events the EMPTY_EVENTS_BITSET will be moved to the last. + * In the case of events, + * dt action + * 2020-01-01 00:00:01 'D' + * 2020-01-01 00:00:01 'A' + * 2020-01-01 00:00:01 'B' + * 2020-01-01 00:00:01 'C' + * The next node of a chain of events 'A' -> 'B' -> 'C' is expected to be the 'D'. + * Because EMPTY_EVENTS_BITSET is 0x80000000 the order of the sorted events is ['A", 'B', 'C', 'D']. The result value of this aggregation is 'D'. + * If EMPTY_EVENTS_BITSET is 0 hen the order of the sorted events is ['D', 'A', 'B', 'C']. This time, the result value is NULL. + */ +static const UInt32 EMPTY_EVENTS_BITSET = 0x80000000; /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template @@ -36,6 +49,8 @@ struct NodeBase DataTypeDateTime::FieldType event_time; UInt32 events_bitset; /// Bitsets of UInt32 are easy to compare. (< operator on bitsets) + /// Nodes in the list must be sorted in order to find a chain of events at the method getNextNodeIndex(). + /// While sorting, events_bitset is one of sorting criteria. char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -92,6 +107,11 @@ struct NodeString : public NodeBase { assert_cast(column).insertData(data(), size); } + + bool compare(const Node * rhs) const + { + return strcmp(data(), rhs->data()) < 0; + } }; /// TODO : Expends SequenceNextNodeGeneralData to support other types @@ -110,10 +130,12 @@ struct SequenceNextNodeGeneralData { if constexpr (Descending) return lhs->event_time == rhs->event_time ? - lhs->events_bitset < rhs->events_bitset : lhs->event_time > rhs->event_time; + (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) + : lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? - lhs->events_bitset < rhs->events_bitset : lhs->event_time < rhs->event_time; + (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) + : lhs->event_time < rhs->event_time; } }; @@ -163,7 +185,7 @@ public: /// aggregated events: [A -> B -> C] /// events to find: [C -> D] /// [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const @@ -194,7 +216,7 @@ public: for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) events_bitset += (1 << i); - if (events_bitset == 0) events_bitset = 0x80000000; // Any events are not matched. + if (events_bitset == 0) events_bitset = EMPTY_EVENTS_BITSET; // Any events are not matched. node->event_time = timestamp; node->events_bitset = events_bitset; @@ -278,7 +300,12 @@ public: } /// This method returns an index of next node that matched the events. - /// It is one as referring Boyer-Moore-Algorithm. + /// It is one as referring Boyer-Moore-Algorithm(https://en.wikipedia.org/wiki/Boyer%E2%80%93Moore_string-search_algorithm). + /// But, there are some differences. + /// In original Boyer-Moore-Algorithm compares strings, but this algorithm compares events_bits. + /// events_bitset consists of events_bits. + /// matched events in the chain of events are represented as a bitmask of UInt32. + /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { if (data.value.size() <= events_size) @@ -292,6 +319,8 @@ public: UInt32 j = 0; /// It checks whether the chain of events are matched or not. for (; j < events_size; ++j) + /// It compares each matched events. + /// The lower bitmask is the former matched event. if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) break; @@ -350,7 +379,7 @@ public: const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties &) const override { - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const @@ -388,7 +417,7 @@ public: { Node * node = Node::allocate(*columns[1], row_num, arena); node->event_time = timestamp; - node->events_bitset = 0x80000000; + node->events_bitset = EMPTY_EVENTS_BITSET; data(place).value.push_back(node, arena); } From 66e9098e4e8100626fd8ef3a8428b69a4272e254 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 26 Feb 2021 14:56:59 +0900 Subject: [PATCH 0044/1060] Fix comparator of NodeString --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 9bde44be06e..e188ae58045 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -110,7 +110,8 @@ struct NodeString : public NodeBase bool compare(const Node * rhs) const { - return strcmp(data(), rhs->data()) < 0; + auto cmp = strncmp(data(), rhs->data(), std::min(size, rhs->size)); + return (cmp == 0) ? size <= rhs->size : cmp < 0; } }; From ec3d63bfe471f25bf5fbe09c30cdfca7fddbcf0a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 3 Mar 2021 11:09:44 +0300 Subject: [PATCH 0045/1060] add check for fatal --- tests/integration/helpers/cluster.py | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0347eee1719..09b3731e2bb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -955,6 +955,7 @@ class ClickHouseCluster: def shutdown(self, kill=True): sanitizer_assert_instance = None + fatal_log = None if self.up_called: with open(self.docker_logs_path, "w+") as f: try: @@ -967,6 +968,15 @@ class ClickHouseCluster: sanitizer_assert_instance = line.split('|')[0].strip() break + for instance in list(self.instances.values()): + if instance.contains_in_log(SANITIZER_SIGN): + sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) + logging.ERROR(f"Sanitizer in instance {instance.name} fatal log {fatal_log}") + + if instance.contains_in_log("Fatal"): + fatal_log = instance.grep_in_log("Fatal") + logging.ERROR(f"Crash in instance {instance.name} fatal log {fatal_log}") + if kill: try: subprocess_check_call(self.base_cmd + ['stop', '--timeout', '20']) @@ -1002,6 +1012,7 @@ class ClickHouseCluster: raise Exception( "Sanitizer assert found in {} for instance {}".format(self.docker_logs_path, sanitizer_assert_instance)) + def pause_container(self, instance_name): subprocess_check_call(self.base_cmd + ['pause', instance_name]) @@ -1287,6 +1298,11 @@ class ClickHouseInstance: ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) return len(result) > 0 + def grep_in_log(self, substring): + result = self.exec_in_container( + ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) + return result + def file_exists(self, path): return self.exec_in_container( ["bash", "-c", "echo $(if [ -e '{}' ]; then echo 'yes'; else echo 'no'; fi)".format(path)]) == 'yes\n' From 34609445f75ce10117b3692d99417380a149d99d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 12:06:39 +0300 Subject: [PATCH 0046/1060] fix --- tests/integration/test_storage_kafka/test.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 1a3a6ce136a..b9635ecd738 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -50,9 +50,9 @@ instance = cluster.add_instance('instance', clickhouse_path_dir='clickhouse_path') -def get_kafka_producer(port, serializer): +def get_kafka_producer(port, serializer, retries): errors = [] - for _ in range(15): + for _ in range(retries): try: producer = KafkaProducer(bootstrap_servers="localhost:{}".format(port), value_serializer=serializer) logging.debug("Kafka Connection establised: localhost:{}".format(port)) @@ -66,7 +66,7 @@ def get_kafka_producer(port, serializer): def producer_serializer(x): return x.encode() if isinstance(x, str) else x -def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=2): +def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): logging.debug("kafka_produce server:{}:{} topic:{}".format("localhost", kafka_cluster.kafka_port, topic)) producer = get_kafka_producer(kafka_cluster.kafka_port, producer_serializer, retries) for message in messages: From 8b6a64e8b8b5ccee6c0a0f9092509f881ca86248 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 12:07:59 +0300 Subject: [PATCH 0047/1060] typo --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 24153b4236d..2f70d77eb08 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -1320,7 +1320,7 @@ class ClickHouseInstance: wait_duration = time.time() - start_time logging.debug('{} log line matching "{}" appeared in a {} seconds'.format(repetitions, regexp, wait_duration)) - return wait_duratio + return wait_duration def file_exists(self, path): return self.exec_in_container( From 8122003c98cfa99e36a8703c12da932630a029e5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 13:53:50 +0300 Subject: [PATCH 0048/1060] more fixes --- tests/integration/test_storage_kafka/test.py | 44 ++++++++++---------- 1 file changed, 23 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index b9635ecd738..f72b2b105bf 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -19,7 +19,9 @@ from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster from helpers.network import PartitionManager from helpers.test_tools import TSV -from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer +from kafka import KafkaAdminClient, KafkaProducer, KafkaConsumer, BrokerConnection +from kafka.protocol.admin import DescribeGroupsRequest_v1 +from kafka.protocol.group import MemberAssignment from kafka.admin import NewTopic @@ -75,7 +77,7 @@ def kafka_produce(kafka_cluster, topic, messages, timestamp=None, retries=15): ## just to ensure the python client / producer is working properly def kafka_producer_send_heartbeat_msg(max_retries=50): - kafka_produce('test_heartbeat_topic', ['test'], retries=max_retries) + kafka_produce(kafka_cluster, 'test_heartbeat_topic', ['test'], retries=max_retries) def kafka_consume(kafka_cluster, topic): consumer = KafkaConsumer(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port), auto_offset_reset="earliest") @@ -154,8 +156,8 @@ def kafka_check_result(result, check=False, ref_file='test_kafka_json.reference' else: return TSV(result) == TSV(reference) -def describe_consumer_group(name): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") +def describe_consumer_group(kafka_cluster, name): + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) consumer_groups = admin_client.describe_consumer_groups([name]) res = [] for member in consumer_groups[0].members: @@ -216,7 +218,7 @@ kafka_topic_old old messages = [] for i in range(50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('old', messages) + kafka_produce(kafka_cluster, 'old', messages) result = '' while True: @@ -248,16 +250,16 @@ def test_kafka_settings_new_syntax(kafka_cluster): messages = [] for i in range(25): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) # Insert couple of malformed messages. - kafka_produce('new', ['}{very_broken_message,']) - kafka_produce('new', ['}another{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}{very_broken_message,']) + kafka_produce(kafka_cluster, 'new', ['}another{very_broken_message,']) messages = [] for i in range(25, 50): messages.append(json.dumps({'key': i, 'value': i})) - kafka_produce('new', messages) + kafka_produce(kafka_cluster, 'new', messages) result = '' while True: @@ -907,7 +909,7 @@ def test_kafka_issue4116(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="consumer_hang", num_partitions=8, replication_factor=1)) @@ -959,7 +961,7 @@ def test_kafka_consumer_hang(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_consumer_hang2(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="consumer_hang2", num_partitions=1, replication_factor=1)) @@ -1010,7 +1012,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}, {i}'.format(i=i)) - kafka_produce('csv', messages) + kafka_produce(kafka_cluster, 'csv', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -1035,7 +1037,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): messages = [] for i in range(50): messages.append('{i}\t{i}'.format(i=i)) - kafka_produce('tsv', messages) + kafka_produce(kafka_cluster, 'tsv', messages) instance.query(''' CREATE TABLE test.kafka (key UInt64, value UInt64) @@ -1057,7 +1059,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): @pytest.mark.timeout(120) def test_kafka_select_empty(kafka_cluster): - admin_client = KafkaAdminClient(bootstrap_servers="localhost:9092") + admin_client = KafkaAdminClient(bootstrap_servers="localhost:{}".format(kafka_cluster.kafka_port)) topic_list = [] topic_list.append(NewTopic(name="empty", num_partitions=1, replication_factor=1)) admin_client.create_topics(new_topics=topic_list, validate_only=False) @@ -1107,9 +1109,9 @@ def test_kafka_json_without_delimiter(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_protobuf(kafka_cluster): - kafka_produce_protobuf_messages('pb', 0, 20) - kafka_produce_protobuf_messages('pb', 20, 1) - kafka_produce_protobuf_messages('pb', 21, 29) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 0, 20) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 20, 1) + kafka_produce_protobuf_messages(kafka_cluster, 'pb', 21, 29) instance.query(''' CREATE TABLE test.kafka (key UInt64, value String) @@ -1133,9 +1135,9 @@ def test_kafka_protobuf(kafka_cluster): @pytest.mark.timeout(180) def test_kafka_string_field_on_first_position_in_protobuf(kafka_cluster): # https://github.com/ClickHouse/ClickHouse/issues/12615 - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 0, 20) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 20, 1) - kafka_produce_protobuf_social('string_field_on_first_position_in_protobuf', 21, 29) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 0, 20) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 20, 1) + kafka_produce_protobuf_social(kafka_cluster, 'string_field_on_first_position_in_protobuf', 21, 29) instance.query(''' CREATE TABLE test.kafka ( @@ -1349,7 +1351,7 @@ def test_librdkafka_compression(kafka_cluster): SELECT * FROM test.kafka; '''.format(topic_name=topic_name) ) - kafka_produce(topic_name, messages) + kafka_produce(kafka_cluster, topic_name, messages) instance.wait_for_log_line("Committed offset {}".format(number_of_messages)) From f03cf47a8264ea9970fd0a614d87277b9ec16f48 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 14:22:19 +0300 Subject: [PATCH 0049/1060] fix --- tests/integration/test_storage_kafka/test.py | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index f72b2b105bf..115f1e926bc 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2525,7 +2525,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.pause_container('kafka1') instance.query(''' - CREATE TABLE test.kafka (key UInt64, value UInt64) + CREATE TABLE test.test_kafka_unavailable (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', kafka_topic_list = 'test_kafka_unavailable', @@ -2534,7 +2534,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_max_block_size = 1000, kafka_flush_interval_ms = 1000; - CREATE MATERIALIZED VIEW test.destination Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_kafka_unavailable Engine=Log AS SELECT key, now() as consume_ts, @@ -2544,10 +2544,10 @@ def test_kafka_unavailable(kafka_cluster): _offset, _partition, _timestamp - FROM test.kafka; + FROM test.test_kafka_unavailable; ''') - instance.query("SELECT * FROM test.kafka") + instance.query("SELECT * FROM test.test_kafka_unavailable") instance.wait_for_log_line('brokers are down') instance.wait_for_log_line('stalled. Reschedule', repetitions=2) @@ -2555,7 +2555,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.unpause_container('kafka1') instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination")) == 2000 + assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 2000 time.sleep(5) # needed to give time for kafka client in python test to recovery @pytest.mark.timeout(180) From 298824c9262b78df3c140d3f29ea81b6d0df3e3a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 4 Mar 2021 14:52:56 +0300 Subject: [PATCH 0050/1060] fix --- tests/integration/test_storage_kafka/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 115f1e926bc..e0c337775a4 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2525,6 +2525,8 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.pause_container('kafka1') instance.query(''' + DROP TABLE IF EXISTS test.destination_kafka_unavailable; + DROP TABLE IF EXISTS test.test_kafka_unavailable; CREATE TABLE test.test_kafka_unavailable (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', @@ -2555,7 +2557,7 @@ def test_kafka_unavailable(kafka_cluster): kafka_cluster.unpause_container('kafka1') instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 2000 + assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 5000 time.sleep(5) # needed to give time for kafka client in python test to recovery @pytest.mark.timeout(180) From fad1a4ccb3c79958699b8292da6aecc63efcca5e Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Fri, 5 Mar 2021 21:46:09 +0900 Subject: [PATCH 0051/1060] Fix comparator of sequenceNextNode for more deterministic sorting --- .../AggregateFunctionSequenceNextNode.cpp | 2 +- .../AggregateFunctionSequenceNextNode.h | 49 ++++++------------- .../01656_sequence_next_node.reference | 16 +++--- .../0_stateless/01656_sequence_next_node.sql | 37 ++++++++------ 4 files changed, 45 insertions(+), 59 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 1513e5b3213..9d1d8aaa075 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -115,7 +115,7 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_args) void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(31), properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(MAX_EVENTS_SIZE), properties }); factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index e188ae58045..b8cf7ef5f5d 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,19 +27,7 @@ namespace DB { -/** - * When sorting the list of events the EMPTY_EVENTS_BITSET will be moved to the last. - * In the case of events, - * dt action - * 2020-01-01 00:00:01 'D' - * 2020-01-01 00:00:01 'A' - * 2020-01-01 00:00:01 'B' - * 2020-01-01 00:00:01 'C' - * The next node of a chain of events 'A' -> 'B' -> 'C' is expected to be the 'D'. - * Because EMPTY_EVENTS_BITSET is 0x80000000 the order of the sorted events is ['A", 'B', 'C', 'D']. The result value of this aggregation is 'D'. - * If EMPTY_EVENTS_BITSET is 0 hen the order of the sorted events is ['D', 'A', 'B', 'C']. This time, the result value is NULL. - */ -static const UInt32 EMPTY_EVENTS_BITSET = 0x80000000; +const UInt32 MAX_EVENTS_SIZE = 64; /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template @@ -48,9 +36,7 @@ struct NodeBase UInt64 size; /// size of payload DataTypeDateTime::FieldType event_time; - UInt32 events_bitset; /// Bitsets of UInt32 are easy to compare. (< operator on bitsets) - /// Nodes in the list must be sorted in order to find a chain of events at the method getNextNodeIndex(). - /// While sorting, events_bitset is one of sorting criteria. + std::bitset events_bitset; char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -68,7 +54,8 @@ struct NodeBase buf.write(data(), size); writeBinary(event_time, buf); - writeBinary(events_bitset, buf); + UInt64 ulong_bitset = events_bitset.to_ulong(); + writeBinary(ulong_bitset, buf); } static Node * read(ReadBuffer & buf, Arena * arena) @@ -81,7 +68,9 @@ struct NodeBase buf.read(node->data(), size); readBinary(node->event_time, buf); - readBinary(node->events_bitset, buf); + UInt64 ulong_bitset; + readBinary(ulong_bitset, buf); + node->events_bitset = ulong_bitset; return node; } @@ -130,13 +119,9 @@ struct SequenceNextNodeGeneralData bool operator()(const Node * lhs, const Node * rhs) const { if constexpr (Descending) - return lhs->event_time == rhs->event_time ? - (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) - : lhs->event_time > rhs->event_time; + return lhs->event_time == rhs->event_time ? !lhs->compare(rhs) : lhs->event_time > rhs->event_time; else - return lhs->event_time == rhs->event_time ? - (lhs->events_bitset == rhs->events_bitset ? lhs->compare(rhs) : lhs->events_bitset < rhs->events_bitset) - : lhs->event_time < rhs->event_time; + return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; } }; @@ -179,7 +164,7 @@ public: AggregateFunctionPtr getOwnNullAdapter( const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, - const AggregateFunctionProperties & /*properties*/) const override + const AggregateFunctionProperties &) const override { /// This aggregate function sets insertion_requires_nullable_column on. /// Even though some values are mapped to aggregating key, it could return nulls for the below case. @@ -213,14 +198,11 @@ public: /// 0x00000000 /// + 1 (bit of event1) /// + 4 (bit of event3) - UInt32 events_bitset = 0; + node->events_bitset.reset(); for (UInt8 i = 0; i < events_size; ++i) if (assert_cast *>(columns[2 + i])->getData()[row_num]) - events_bitset += (1 << i); - if (events_bitset == 0) events_bitset = EMPTY_EVENTS_BITSET; // Any events are not matched. - + node->events_bitset.set(i); node->event_time = timestamp; - node->events_bitset = events_bitset; data(place).value.push_back(node, arena); } @@ -295,7 +277,7 @@ public: { UInt32 k = 0; for (; k < events_size - j; ++k) - if (data.value[i - j]->events_bitset & (1 << (events_size - 1 - j - k))) + if (data.value[i - j]->events_bitset.test(events_size - 1 - j - k)) return k; return k; } @@ -322,7 +304,7 @@ public: for (; j < events_size; ++j) /// It compares each matched events. /// The lower bitmask is the former matched event. - if (!(data.value[i - j]->events_bitset & (1 << (events_size - 1 - j)))) + if (data.value[i - j]->events_bitset.test(events_size - 1 - j) == false) break; /// If the chain of events are matched returns the index of result value. @@ -413,12 +395,11 @@ public: is_first = false; } - if (is_first) { Node * node = Node::allocate(*columns[1], row_num, arena); node->event_time = timestamp; - node->events_bitset = EMPTY_EVENTS_BITSET; + node->events_bitset.reset(); data(place).value.push_back(node, arena); } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 50755232cb9..da6ec2d97bf 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -124,14 +124,14 @@ (0, A) id >= 10 10 B (0, A) id >= 10 10 B (0, A) id >= 10 10 A -(0, A) id = 11 0 -(0, C) id = 11 0 -(0, B->C) id = 11 0 -(0, A->B->C) id = 11 0 -(0, A) id = 11 0 -(0, C) id = 11 0 -(0, C->B) id = 11 0 -(0, C->B->A) id = 11 0 +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, B->C) id = 11 1 +(0, A->B->C) id = 11 1 +(0, A) id = 11 1 +(0, C) id = 11 1 +(0, C->B) id = 11 1 +(0, C->B->A) id = 11 1 (0) id < 10 1 A (0) id < 10 2 A (0) id < 10 3 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index c2270bc10ac..2c16f33aa0e 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -61,14 +61,19 @@ SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS n SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); + +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; @@ -146,14 +151,14 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'C', 'D')); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('B', 'A', 'D')); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('A', 'D')); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node in ('D')); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; From 534500ef84dff4b3bb66a9d726f7658dc28433a3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 5 Mar 2021 16:39:51 +0300 Subject: [PATCH 0052/1060] wip --- .../runner/compose/docker_compose_mysql.yml | 20 +++++- tests/integration/helpers/cluster.py | 63 ++++++++++++++----- tests/integration/helpers/external_sources.py | 4 ++ tests/integration/test_mysql_protocol/test.py | 10 +-- 4 files changed, 73 insertions(+), 24 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql.yml index 3075abdc181..c9b45af6563 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql.yml @@ -5,6 +5,20 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - ${MYSQL_EXTERNAL_PORT}:${MYSQL_INTERNAL_PORT} - command: --server_id=100 --log-bin='mysql-bin-1.log' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_PORT} + command: --server_id=100 + --log-bin='mysql-bin-1.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/error.log + --general-log=ON + --general-log-file=/mysql/general.log + volumes: + - type: ${MYSQL_LOGS_FS:-tmpfs} + source: ${MYSQL_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 2f70d77eb08..588b2599a34 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -3,6 +3,7 @@ import errno import http.client import logging import os +import stat import os.path as p import pprint import pwd @@ -111,7 +112,6 @@ def check_kafka_is_available(kafka_id, kafka_port): p.communicate() return p.returncode == 0 - class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. @@ -242,11 +242,16 @@ class ClickHouseCluster: # available when with_mysql == True self.mysql_host = "mysql57" - self.mysql_port = get_open_port() + self.mysql_port = 3306 + self.mysql_ip = None + self.mysql_dir = p.abspath(p.join(self.instances_dir, "mysql")) + self.mysql_logs_dir = os.path.join(self.mysql_dir, "logs") # available when with_mysql8 == True self.mysql8_host = "mysql80" self.mysql8_port = get_open_port() + self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8")) + self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs") self.zookeeper_use_tmpfs = True @@ -263,8 +268,10 @@ class ClickHouseCluster: def setup_mysql_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_mysql = True env_variables['MYSQL_HOST'] = self.mysql_host - env_variables['MYSQL_EXTERNAL_PORT'] = str(self.mysql_port) - env_variables['MYSQL_INTERNAL_PORT'] = "3306" + env_variables['MYSQL_PORT'] = str(self.mysql_port) + env_variables['MYSQL_ROOT_HOST'] = '%' + env_variables['MYSQL_LOGS'] = self.mysql_logs_dir + env_variables['MYSQL_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')]) self.base_mysql_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql.yml')] @@ -276,6 +283,8 @@ class ClickHouseCluster: env_variables['MYSQL8_HOST'] = self.mysql8_host env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port) env_variables['MYSQL8_INTERNAL_PORT'] = "3306" + env_variables['MYSQL8_LOGS'] = self.mysql8_logs_dir + env_variables['MYSQL8_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')]) self.base_mysql8_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')] @@ -627,12 +636,13 @@ class ClickHouseCluster: ["bash", "-c", "echo {} | base64 --decode > {}".format(encodedStr, dest_path)], user='root') - def wait_mysql_to_start(self, timeout=60): + def wait_mysql_to_start(self, timeout=30): + self.mysql_ip = self.get_instance_ip('mysql57') start = time.time() errors = [] while time.time() - start < timeout: try: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=self.mysql_ip, port=self.mysql_port) conn.close() logging.debug("Mysql Started") return @@ -812,6 +822,7 @@ class ClickHouseCluster: if self.is_up: return + # Just in case kill unstopped containers from previous launch try: logging.debug("Trying to kill unstopped containers...") @@ -822,6 +833,15 @@ class ClickHouseCluster: except: pass + # # Just in case remove unused networks + # try: + # logging.debug("Trying to prune unused networks...") + + # subprocess_call(['docker', 'network', 'prune', '-f']) + # logging.debug("Networks pruned") + # except: + # pass + try: if destroy_dirs and p.exists(self.instances_dir): logging.debug(("Removing instances dir %s", self.instances_dir)) @@ -831,7 +851,7 @@ class ClickHouseCluster: logging.debug(('Setup directory for instance: {} destroy_dirs: {}'.format(instance.name, destroy_dirs))) instance.create_dir(destroy_dir=destroy_dirs) - self.docker_client = docker.from_env(version=self.docker_api_version) + self.docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180) common_opts = ['up', '-d', '--force-recreate'] @@ -856,11 +876,14 @@ class ClickHouseCluster: if self.with_mysql and self.base_mysql_cmd: logging.debug('Setup MySQL') + os.makedirs(self.mysql_logs_dir) + os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(120) + self.wait_mysql_to_start(60) if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') + os.makedirs(self.mysql8_logs_dir) subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start(120) @@ -935,7 +958,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 20.0 # seconds + start_deadline = time.time() + 30.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -970,13 +993,19 @@ class ClickHouseCluster: break for instance in list(self.instances.values()): - if instance.contains_in_log(SANITIZER_SIGN): - sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) - logging.ERROR(f"Sanitizer in instance {instance.name} fatal log {fatal_log}") + try: + if not instance.is_up: + continue + if instance.contains_in_log(SANITIZER_SIGN): + sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) + logging.ERROR(f"Sanitizer in instance {instance.name} log {sanitizer_assert_instance}") - if instance.contains_in_log("Fatal"): - fatal_log = instance.grep_in_log("Fatal") - logging.ERROR(f"Crash in instance {instance.name} fatal log {fatal_log}") + if instance.contains_in_log("Fatal"): + fatal_log = instance.grep_in_log("Fatal") + name = instance.name + logging.ERROR(f"Crash in instance {name} fatal log {fatal_log}") + except Exception as e: + logging.error(f"Failed to check fails in logs: {e}") if kill: try: @@ -1177,6 +1206,7 @@ class ClickHouseInstance: self.ipv6_address = ipv6_address self.with_installed_binary = with_installed_binary self.env_file = os.path.join(os.path.dirname(self.docker_compose_path), DEFAULT_ENV_NAME) + self.is_up = False def is_built_with_thread_sanitizer(self): @@ -1342,7 +1372,7 @@ class ClickHouseInstance: return None return None - def restart_with_latest_version(self, stop_start_wait_sec=10, callback_onstop=None, signal=15): + def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=15): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root') @@ -1410,6 +1440,7 @@ class ClickHouseInstance: sock = socket.socket(socket.AF_INET, socket.SOCK_STREAM) sock.settimeout(time_left) sock.connect((self.ip_address, 9000)) + self.is_up = True return except socket.timeout: continue diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index fe4c5269a56..66cd45583a7 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -9,6 +9,7 @@ import cassandra.cluster import pymongo import pymysql.cursors import redis +import logging from tzlocal import get_localzone @@ -59,6 +60,7 @@ class SourceMySQL(ExternalSource): } def create_mysql_conn(self): + logging.debug(f"pymysql connect {self.user}, {self.password}, {self.internal_hostname}, {self.internal_port}") self.connection = pymysql.connect( user=self.user, password=self.password, @@ -98,6 +100,8 @@ class SourceMySQL(ExternalSource): ) def prepare(self, structure, table_name, cluster): + if self.internal_hostname is None: + self.internal_hostname = cluster.mysql_ip self.create_mysql_conn() self.execute_mysql_query("create database if not exists test default character set 'utf8'") self.execute_mysql_query("drop table if exists test.{}".format(table_name)) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 7f7d59674bc..bc7cb6024c0 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -36,7 +36,7 @@ def mysql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_mysql1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') @pytest.fixture(scope='module') @@ -63,7 +63,7 @@ def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_golang1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') @@ -71,7 +71,7 @@ def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_php1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') @@ -79,7 +79,7 @@ def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_mysqljs1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') @@ -87,7 +87,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.from_env().containers.get(cluster.project_name + '_java1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') def test_mysql_client(mysql_client, server_address): From a6c7ddbcdf253a314d386b3029e0b03739f9410d Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Sun, 7 Mar 2021 22:52:23 +0900 Subject: [PATCH 0053/1060] Fix comparator --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index b8cf7ef5f5d..0f642542daf 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -100,7 +100,7 @@ struct NodeString : public NodeBase bool compare(const Node * rhs) const { auto cmp = strncmp(data(), rhs->data(), std::min(size, rhs->size)); - return (cmp == 0) ? size <= rhs->size : cmp < 0; + return (cmp == 0) ? size < rhs->size : cmp < 0; } }; From e6f90298bba0cc11a5de7730906b01d36a3e8ab5 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 9 Mar 2021 10:32:10 +0300 Subject: [PATCH 0054/1060] wip --- docker/test/integration/runner/Dockerfile | 1 - ...ompose_mysql_5_7_for_materialize_mysql.yml | 7 ++++ .../compose/docker_compose_mysql_8_0.yml | 14 ++++++-- .../integration/runner/dockerd-entrypoint.sh | 2 +- tests/integration/helpers/cluster.py | 18 ++++++---- tests/integration/helpers/network.py | 4 +-- tests/integration/runner | 2 +- .../configs/merge_tree_settings.xml | 2 +- .../test_adaptive_granularity/test.py | 5 +++ tests/integration/test_cluster_copier/test.py | 36 +++++++++---------- .../test_cluster_copier/trivial_test.py | 22 ++++++------ .../test_mysql.py | 4 +-- .../test.py | 2 +- .../test_mysql_database_engine/test.py | 16 ++++----- tests/integration/test_mysql_protocol/test.py | 5 +-- .../test_postgresql_protocol/test.py | 4 +-- 16 files changed, 86 insertions(+), 58 deletions(-) diff --git a/docker/test/integration/runner/Dockerfile b/docker/test/integration/runner/Dockerfile index 0a1ab0662a8..6b6651e5e3f 100644 --- a/docker/test/integration/runner/Dockerfile +++ b/docker/test/integration/runner/Dockerfile @@ -40,7 +40,6 @@ ENV TZ=Europe/Moscow RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone ENV DOCKER_CHANNEL stable -ENV DOCKER_VERSION 5:19.03.13~3-0~ubuntu-bionic RUN curl -fsSL https://download.docker.com/linux/ubuntu/gpg | apt-key add - RUN add-apt-repository "deb [arch=amd64] https://download.docker.com/linux/ubuntu $(lsb_release -c -s) ${DOCKER_CHANNEL}" diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml index 5aa13ba91c7..ba693fd9fb4 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_5_7_for_materialize_mysql.yml @@ -12,3 +12,10 @@ services: --gtid-mode="ON" --enforce-gtid-consistency --log-error-verbosity=3 + --log-error=/var/log/mysqld/error.log + --general-log=ON + --general-log-file=/var/log/mysqld/general.log + volumes: + - type: ${MYSQL_LOGS_FS:-tmpfs} + source: ${MYSQL_LOGS:-} + target: /var/log/mysqld/ \ No newline at end of file diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml index a7fb5d85d8b..e13076c4e2e 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_8_0.yml @@ -5,9 +5,19 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - ports: - - ${MYSQL8_EXTERNAL_PORT}:${MYSQL8_INTERNAL_PORT} + MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL8_PORT} command: --server_id=100 --log-bin='mysql-bin-1.log' --default_authentication_plugin='mysql_native_password' --default-time-zone='+3:00' --gtid-mode="ON" --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/error.log + --general-log=ON + --general-log-file=/mysql/general.log + volumes: + - type: ${MYSQL8_LOGS_FS:-tmpfs} + source: ${MYSQL8_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index c0255d3d706..9f04dde720d 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,7 +1,7 @@ #!/bin/bash set -e -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 &>/var/log/somefile & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.0.0.0/8,size=24 &>/var/log/somefile & set +e reties=0 diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 588b2599a34..eb6ecc8a39f 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -249,7 +249,8 @@ class ClickHouseCluster: # available when with_mysql8 == True self.mysql8_host = "mysql80" - self.mysql8_port = get_open_port() + self.mysql8_port = 3306 + self.mysql8_ip = None self.mysql8_dir = p.abspath(p.join(self.instances_dir, "mysql8")) self.mysql8_logs_dir = os.path.join(self.mysql8_dir, "logs") @@ -281,8 +282,8 @@ class ClickHouseCluster: def setup_mysql8_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_mysql8 = True env_variables['MYSQL8_HOST'] = self.mysql8_host - env_variables['MYSQL8_EXTERNAL_PORT'] = str(self.mysql8_port) - env_variables['MYSQL8_INTERNAL_PORT'] = "3306" + env_variables['MYSQL8_PORT'] = str(self.mysql8_port) + env_variables['MYSQL8_ROOT_HOST'] = '%' env_variables['MYSQL8_LOGS'] = self.mysql8_logs_dir env_variables['MYSQL8_LOGS_FS'] = "bind" self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_mysql_8_0.yml')]) @@ -578,7 +579,7 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - start_deadline = time.time() + 20.0 # seconds + start_deadline = time.time() + 60.0 # seconds node.wait_for_start(start_deadline) return node @@ -876,6 +877,8 @@ class ClickHouseCluster: if self.with_mysql and self.base_mysql_cmd: logging.debug('Setup MySQL') + if os.path.exists(self.mysql_dir): + shutil.rmtree(self.mysql_dir) os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) @@ -883,7 +886,10 @@ class ClickHouseCluster: if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') + if os.path.exists(self.mysql8_dir): + shutil.rmtree(self.mysql8_dir) os.makedirs(self.mysql8_logs_dir) + os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql8_cmd + common_opts) self.wait_mysql8_to_start(120) @@ -958,7 +964,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 30.0 # seconds + start_deadline = time.time() + 60.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -1372,7 +1378,7 @@ class ClickHouseInstance: return None return None - def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=15): + def restart_with_latest_version(self, stop_start_wait_sec=15, callback_onstop=None, signal=60): if not self.stay_alive: raise Exception("Cannot restart not stay alive container") self.exec_in_container(["bash", "-c", "pkill -{} clickhouse".format(signal)], user='root') diff --git a/tests/integration/helpers/network.py b/tests/integration/helpers/network.py index aa697c63d72..120ec9e873d 100644 --- a/tests/integration/helpers/network.py +++ b/tests/integration/helpers/network.py @@ -159,12 +159,12 @@ class _NetworkManager: def __init__( self, - container_expire_timeout=50, container_exit_timeout=60): + container_expire_timeout=50, container_exit_timeout=60, docker_api_version=os.environ.get("DOCKER_API_VERSION")): self.container_expire_timeout = container_expire_timeout self.container_exit_timeout = container_exit_timeout - self._docker_client = docker.from_env(version=os.environ.get("DOCKER_API_VERSION")) + self._docker_client = docker.DockerClient(base_url='unix:///var/run/docker.sock', version=docker_api_version, timeout=180) self._container = None diff --git a/tests/integration/runner b/tests/integration/runner index 5ad37726821..f3309f983a3 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -199,7 +199,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=120 \ + --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=180 \ {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, diff --git a/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml b/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml index 31ede39c318..2ee416c156e 100644 --- a/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml +++ b/tests/integration/test_adaptive_granularity/configs/merge_tree_settings.xml @@ -1,4 +1,4 @@ - + diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 12bfc22d7d9..47209908e5c 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -268,6 +268,11 @@ def test_version_single_node_update(start_dynamic_cluster, n, tables): ] ) def test_mixed_granularity_single_node(start_dynamic_cluster, node): + assert node.name == "node9" or node.name == "node10" + assert_eq_with_retry(node, + "SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'", + '0') # check that enable_mixed_granularity_parts is off by default + node.query( "INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)") node.query( diff --git a/tests/integration/test_cluster_copier/test.py b/tests/integration/test_cluster_copier/test.py index d87969630cd..abeb8739b7d 100644 --- a/tests/integration/test_cluster_copier/test.py +++ b/tests/integration/test_cluster_copier/test.py @@ -252,10 +252,10 @@ class Task_non_partitioned_table: instance.query("DROP TABLE copier_test1_1") -def execute_task(task, cmd_options): +def execute_task(started_cluster, task, cmd_options): task.start() - zk = cluster.get_kazoo_client('zoo1') + zk = started_cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) try: @@ -268,7 +268,7 @@ def execute_task(task, cmd_options): zk.create(zk_task_path + "/description", task.copier_task_config.encode()) # Run cluster-copier processes on each node - docker_api = docker.from_env().api + docker_api = started_cluster.docker_client.api copiers_exec_ids = [] cmd = ['/usr/bin/clickhouse', 'copier', @@ -280,7 +280,7 @@ def execute_task(task, cmd_options): copiers = random.sample(list(cluster.instances.keys()), 3) for instance_name in copiers: - instance = cluster.instances[instance_name] + instance = started_cluster.instances[instance_name] container = instance.get_docker_handle() instance.copy_file_to_container(os.path.join(CURRENT_TEST_DIR, "configs/config-copier.xml"), "/etc/clickhouse-server/config-copier.xml") @@ -293,7 +293,7 @@ def execute_task(task, cmd_options): # Wait for copiers stopping and check their return codes for exec_id, instance_name in zip(copiers_exec_ids, copiers): - instance = cluster.instances[instance_name] + instance = started_cluster.instances[instance_name] while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -319,9 +319,9 @@ def execute_task(task, cmd_options): ) def test_copy_simple(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--experimental-use-sample-offset', '1']) + execute_task(started_cluster, Task1(started_cluster), ['--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), []) + execute_task(started_cluster, Task1(started_cluster), []) @pytest.mark.parametrize( @@ -333,10 +333,10 @@ def test_copy_simple(started_cluster, use_sample_offset): ) def test_copy_with_recovering(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task1(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.parametrize( @@ -348,40 +348,40 @@ def test_copy_with_recovering(started_cluster, use_sample_offset): ) def test_copy_with_recovering_after_move_faults(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task1(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition(started_cluster): - execute_task(Task2(started_cluster, "test1"), []) + execute_task(started_cluster, Task2(started_cluster, "test1"), []) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering(started_cluster): - execute_task(Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task2(started_cluster, "test2"), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.timeout(600) def test_copy_month_to_week_partition_with_recovering_after_move_faults(started_cluster): - execute_task(Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, Task2(started_cluster, "test3"), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) def test_block_size(started_cluster): - execute_task(Task_test_block_size(started_cluster), []) + execute_task(started_cluster, Task_test_block_size(started_cluster), []) def test_no_index(started_cluster): - execute_task(Task_no_index(started_cluster), []) + execute_task(started_cluster, Task_no_index(started_cluster), []) def test_no_arg(started_cluster): - execute_task(Task_no_arg(started_cluster), []) + execute_task(started_cluster, Task_no_arg(started_cluster), []) def test_non_partitioned_table(started_cluster): - execute_task(Task_non_partitioned_table(started_cluster), []) + execute_task(started_cluster, Task_non_partitioned_table(started_cluster), []) if __name__ == '__main__': with contextmanager(started_cluster)() as cluster: diff --git a/tests/integration/test_cluster_copier/trivial_test.py b/tests/integration/test_cluster_copier/trivial_test.py index 8a43440ac90..717ff9d8d34 100644 --- a/tests/integration/test_cluster_copier/trivial_test.py +++ b/tests/integration/test_cluster_copier/trivial_test.py @@ -79,10 +79,10 @@ class TaskTrivial: node.query("DROP TABLE trivial") -def execute_task(task, cmd_options): +def execute_task(started_cluster, task, cmd_options): task.start() - zk = cluster.get_kazoo_client('zoo1') + zk = started_cluster.get_kazoo_client('zoo1') print("Use ZooKeeper server: {}:{}".format(zk.hosts[0][0], zk.hosts[0][1])) zk_task_path = task.zk_task_path @@ -90,7 +90,7 @@ def execute_task(task, cmd_options): zk.create(zk_task_path + "/description", task.copier_task_config) # Run cluster-copier processes on each node - docker_api = docker.from_env().api + docker_api = started_cluster.docker_client.api copiers_exec_ids = [] cmd = ['/usr/bin/clickhouse', 'copier', @@ -101,7 +101,7 @@ def execute_task(task, cmd_options): print(cmd) - for instance_name, instance in cluster.instances.items(): + for instance_name, instance in started_cluster.instances.items(): container = instance.get_docker_handle() exec_id = docker_api.exec_create(container.id, cmd, stderr=True) docker_api.exec_start(exec_id, detach=True) @@ -110,7 +110,7 @@ def execute_task(task, cmd_options): print("Copier for {} ({}) has started".format(instance.name, instance.ip_address)) # Wait for copiers stopping and check their return codes - for exec_id, instance in zip(copiers_exec_ids, iter(cluster.instances.values())): + for exec_id, instance in zip(copiers_exec_ids, iter(started_cluster.instances.values())): while True: res = docker_api.exec_inspect(exec_id) if not res['Running']: @@ -137,10 +137,10 @@ def execute_task(task, cmd_options): ) def test_trivial_copy(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) + execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), ['--experimental-use-sample-offset', '1']) else: print("AAAAA") - execute_task(TaskTrivial(started_cluster, use_sample_offset), []) + execute_task(started_cluster, TaskTrivial(started_cluster, use_sample_offset), []) @pytest.mark.parametrize( @@ -152,10 +152,10 @@ def test_trivial_copy(started_cluster, use_sample_offset): ) def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), + execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) + execute_task(started_cluster, TaskTrivial(started_cluster), ['--copy-fault-probability', str(COPYING_FAIL_PROBABILITY)]) @pytest.mark.parametrize( @@ -167,10 +167,10 @@ def test_trivial_copy_with_copy_fault(started_cluster, use_sample_offset): ) def test_trivial_copy_with_move_fault(started_cluster, use_sample_offset): if use_sample_offset: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), + execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY), '--experimental-use-sample-offset', '1']) else: - execute_task(TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) + execute_task(started_cluster, TaskTrivial(started_cluster), ['--move-fault-probability', str(MOVING_FAIL_PROBABILITY)]) if __name__ == '__main__': diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py index 119709bf229..7feba20f3a1 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_mysql.py @@ -26,7 +26,7 @@ def setup_module(module): cluster = ClickHouseCluster(__file__, name=test_name) - SOURCE = SourceMySQL("MySQL", "localhost", cluster.mysql_port, cluster.mysql_host, "3306", "root", "clickhouse") + SOURCE = SourceMySQL("MySQL", None, cluster.mysql_port, cluster.mysql_host, cluster.mysql_port, "root", "clickhouse") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() @@ -55,7 +55,7 @@ def teardown_module(module): def started_cluster(): try: cluster.start() - + simple_tester.prepare(cluster) complex_tester.prepare(cluster) ranged_tester.prepare(cluster) diff --git a/tests/integration/test_match_process_uid_against_data_owner/test.py b/tests/integration/test_match_process_uid_against_data_owner/test.py index c9f87e683c8..ed37227ab15 100644 --- a/tests/integration/test_match_process_uid_against_data_owner/test.py +++ b/tests/integration/test_match_process_uid_against_data_owner/test.py @@ -20,7 +20,7 @@ def test_different_user(): cluster.start() - docker_api = docker.from_env().api + docker_api = cluster.docker_client.api container = node.get_docker_handle() container.stop() container.start() diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index f5fb9be337d..07a4a987b59 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -21,7 +21,7 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): + def __init__(self, user, password, hostname, port): self.user = user self.port = port self.hostname = hostname @@ -55,7 +55,7 @@ class MySQLNodeInstance: def test_mysql_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("DROP DATABASE IF EXISTS test_database") mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") @@ -89,7 +89,7 @@ def test_mysql_ddl_for_mysql_database(started_cluster): def test_clickhouse_ddl_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;') @@ -114,7 +114,7 @@ def test_clickhouse_ddl_for_mysql_database(started_cluster): def test_clickhouse_dml_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") mysql_node.query( 'CREATE TABLE `test_database`.`test_table` ( `i``d` int(11) NOT NULL, PRIMARY KEY (`i``d`)) ENGINE=InnoDB;') @@ -132,7 +132,7 @@ def test_clickhouse_dml_for_mysql_database(started_cluster): def test_clickhouse_join_for_mysql_database(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") mysql_node.query("CREATE TABLE test.t1_mysql_local (" "pays VARCHAR(55) DEFAULT 'FRA' NOT NULL," @@ -155,7 +155,7 @@ def test_clickhouse_join_for_mysql_database(started_cluster): def test_bad_arguments_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node: with pytest.raises(QueryRuntimeException) as exception: mysql_node.query("CREATE DATABASE IF NOT EXISTS test_bad_arguments DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query( @@ -165,7 +165,7 @@ def test_bad_arguments_for_mysql_database_engine(started_cluster): def test_data_types_support_level_for_mysql_database_engine(started_cluster): - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, started_cluster.mysql_port)) as mysql_node: mysql_node.query("CREATE DATABASE IF NOT EXISTS test DEFAULT CHARACTER SET 'utf8'") clickhouse_node.query("CREATE DATABASE test_database ENGINE = MySQL('mysql57:3306', test, 'root', 'clickhouse')", settings={"mysql_datatypes_support_level": "decimal,datetime64"}) @@ -307,7 +307,7 @@ def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, m else: return [do_execute(q) for q in query] - with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', port=started_cluster.mysql_port)) as mysql_node: + with contextlib.closing(MySQLNodeInstance('root', 'clickhouse', started_cluster.mysql_ip, port=started_cluster.mysql_port)) as mysql_node: execute_query(mysql_node, [ "DROP DATABASE IF EXISTS ${mysql_db}", "CREATE DATABASE ${mysql_db} DEFAULT CHARACTER SET 'utf8'", diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index bc7cb6024c0..1e0d3652a91 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -5,6 +5,7 @@ import math import os import time +import logging import docker import pymysql.connections import pytest @@ -36,7 +37,7 @@ def mysql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysql1_1') @pytest.fixture(scope='module') @@ -96,7 +97,7 @@ def test_mysql_client(mysql_client, server_address): mysql --protocol tcp -h {host} -P {port} default -u user_with_double_sha1 --password=abacaba -e "SELECT 1;" '''.format(host=server_address, port=server_port), demux=True) - + logging.debug(f"test_mysql_client code:{code} stdout:{stdout}, stderr:{stderr}") assert stdout.decode() == '\n'.join(['1', '1', '']) code, (stdout, stderr) = mysql_client.exec_run(''' diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 633c02ef408..1404cf409f3 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.from_env().containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') @@ -65,7 +65,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.from_env().containers.get(cluster.project_name + '_java_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') def test_psql_is_ready(psql_server): From 07dc7e81e324072e460cb0a76ff306e457905136 Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Mar 2021 16:39:06 +0300 Subject: [PATCH 0055/1060] Remove MAX_EVENTS_SIZE from AggregateFunctionSequenceNextNode.h --- .../AggregateFunctionSequenceNextNode.cpp | 12 ++++++++---- .../AggregateFunctionSequenceNextNode.h | 17 ++++++++--------- 2 files changed, 16 insertions(+), 13 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 9d1d8aaa075..9c865f3371a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -21,6 +21,8 @@ namespace ErrorCodes namespace { +constexpr size_t MAX_EVENTS_SIZE = 64; + template inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) { @@ -28,22 +30,24 @@ inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTy { // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. if (descending_order) - return std::make_shared>(data_type); + return std::make_shared, true>>(data_type); else - return std::make_shared>(data_type); + return std::make_shared, false>>(data_type); } else { if (descending_order) - return std::make_shared>(data_type, argument_types); + return std::make_shared, true>>(data_type, argument_types); else - return std::make_shared>(data_type, argument_types); + return std::make_shared, false>>(data_type, argument_types); } } AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, const DataTypes & argument_types, const Array & parameters) { + assert(max_args <= MAX_EVENTS_SIZE); + bool descending_order = false; if (parameters.size() == 1) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 0f642542daf..9efc78aa4c5 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,16 +27,14 @@ namespace DB { -const UInt32 MAX_EVENTS_SIZE = 64; - /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl -template +template struct NodeBase { UInt64 size; /// size of payload DataTypeDateTime::FieldType event_time; - std::bitset events_bitset; + std::bitset events_bitset; char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -77,9 +75,10 @@ struct NodeBase }; /// It stores String, timestamp, bitset of matched events. -struct NodeString : public NodeBase +template +struct NodeString : public NodeBase, MaxEventsSize> { - using Node = NodeString; + using Node = NodeString; static Node * allocate(const IColumn & column, size_t row_num, Arena * arena) { @@ -94,13 +93,13 @@ struct NodeString : public NodeBase void insertInto(IColumn & column) { - assert_cast(column).insertData(data(), size); + assert_cast(column).insertData(this->data(), this->size); } bool compare(const Node * rhs) const { - auto cmp = strncmp(data(), rhs->data(), std::min(size, rhs->size)); - return (cmp == 0) ? size < rhs->size : cmp < 0; + auto cmp = strncmp(this->data(), rhs->data(), std::min(this->size, rhs->size)); + return (cmp == 0) ? this->size < rhs->size : cmp < 0; } }; From 5808bf24e2795554d0dc55d9306c3a2852710e5f Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Mar 2021 16:40:32 +0300 Subject: [PATCH 0056/1060] Fix strict weak ordering, return always lower string in sequenceNextNode --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 9efc78aa4c5..bfece896809 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -118,7 +118,7 @@ struct SequenceNextNodeGeneralData bool operator()(const Node * lhs, const Node * rhs) const { if constexpr (Descending) - return lhs->event_time == rhs->event_time ? !lhs->compare(rhs) : lhs->event_time > rhs->event_time; + return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time > rhs->event_time; else return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; } From f32a388fed0bed5f5f1b5017b018e6fcba0da5fe Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 9 Mar 2021 16:56:29 +0300 Subject: [PATCH 0057/1060] Minor changes, add comments --- .../AggregateFunctionSequenceNextNode.h | 27 +++++++++++++------ 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index bfece896809..a04f589f1e8 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -191,7 +191,6 @@ public: const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; /// The events_bitset variable stores matched events in the form of bitset. - /// It uses UInt32 instead of std::bitset because bitsets of UInt32 are easy to compare. (< operator on bitsets) /// Each Nth-bit indicates that the Nth-event are matched. /// For example, event1 and event3 is matched then the values of events_bitset is 0x00000005. /// 0x00000000 @@ -272,12 +271,23 @@ public: value[i] = Node::read(buf, arena); } + /// Calculate position of current event in target chain and shift to corresponding offset + /// Lets consider case where we search chain 'ABCD': + /// - If current event is 'X' we can skip it and perform next step from position after this 'X' + /// - If current event is 'A' we will start from this position + /// - If current event is 'B' then second position in our chain should match this 'B'. + /// And we perform next step from position one before 'B'. + /// - And so on... inline UInt32 calculateJump(const Data & data, const UInt32 i, const UInt32 j) const { - UInt32 k = 0; + /// Fast check if value is zero, not in sequence + if (data.value[i - j]->events_bitset.none()) + return events_size - j; + + UInt32 k = 1; for (; k < events_size - j; ++k) if (data.value[i - j]->events_bitset.test(events_size - 1 - j - k)) - return k; + break; return k; } @@ -285,8 +295,7 @@ public: /// It is one as referring Boyer-Moore-Algorithm(https://en.wikipedia.org/wiki/Boyer%E2%80%93Moore_string-search_algorithm). /// But, there are some differences. /// In original Boyer-Moore-Algorithm compares strings, but this algorithm compares events_bits. - /// events_bitset consists of events_bits. - /// matched events in the chain of events are represented as a bitmask of UInt32. + /// Matched events in the chain of events are represented as a bitmask. /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { @@ -299,14 +308,16 @@ public: while (i < data.value.size()) { UInt32 j = 0; - /// It checks whether the chain of events are matched or not. + /// Try to match chain of events starting from the end of this chain. for (; j < events_size; ++j) + { /// It compares each matched events. /// The lower bitmask is the former matched event. - if (data.value[i - j]->events_bitset.test(events_size - 1 - j) == false) + if (!data.value[i - j]->events_bitset.test(events_size - 1 - j)) break; + } - /// If the chain of events are matched returns the index of result value. + /// Chain of events are matched, return the index of result value. if (j == events_size) return i + 1; From d8caf62c1b4f583e655eee574fd82f69129c0881 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 10 Mar 2021 12:06:05 +0300 Subject: [PATCH 0058/1060] Update test sequence_next_node, choose lowest string for same timestamps --- .../01656_sequence_next_node.reference | 8 ++++---- .../0_stateless/01656_sequence_next_node.sql | 16 ++++++++-------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index da6ec2d97bf..94a55f2d954 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -130,8 +130,8 @@ (0, A->B->C) id = 11 1 (0, A) id = 11 1 (0, C) id = 11 1 -(0, C->B) id = 11 1 -(0, C->B->A) id = 11 1 +(0, B->C) id = 11 1 +(0, B->C->D) id = 11 1 (0) id < 10 1 A (0) id < 10 2 A (0) id < 10 3 A @@ -288,8 +288,8 @@ (0, A->B->C) id = 11 1 (0, A) id = 11 1 (0, C) id = 11 1 -(0, C->B) id = 11 1 -(0, C->B->A) id = 11 1 +(0, B->C) id = 11 1 +(0, B->C->D) id = 11 1 (0) id < 10 1 A (0) id < 10 2 A (0) id < 10 3 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 2c16f33aa0e..57b0d7b1d8e 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -70,10 +70,10 @@ SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; @@ -155,10 +155,10 @@ SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; From e41c731f0f8e368bef8b46ab33e11e0fb72f801f Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Wed, 10 Mar 2021 20:22:44 +0900 Subject: [PATCH 0059/1060] Change the behavior of sequenceNextNode. --- .../parametric-functions.md | 152 +++- .../AggregateFunctionSequenceNextNode.cpp | 84 +- .../AggregateFunctionSequenceNextNode.h | 301 +++---- .../01656_sequence_next_node.reference | 765 +++++++++++------- .../0_stateless/01656_sequence_next_node.sql | 185 +++-- 5 files changed, 866 insertions(+), 621 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 33e5766089b..dcc63ebd960 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -513,11 +513,19 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode(descending_order)(timestamp, event_column, event1, event2, event3, ...) +sequenceNextNode(direction, base)(timestamp, event_column, event1, event2, event3, ...) ``` **Parameters** -- `descending_order` - Used to sort the timestamp in ascending or descending order. 0 or 1. +- `direction` - Used to navigate to directions. + - forward : Moving forward + - backward: Moving backward + +- `base` - Used to set the base point. + - head : Set the base point to the first event + - tail : Set the base point to the last event + - first_match : Set the base point to the first matched event1 + - last_match : Set the base point to the last matched event1 **Arguments** - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. @@ -535,26 +543,158 @@ Type: `Nullable(String)`. It can be used when events are A->B->C->E->F and you want to know the event following B->C, which is E. -The query statement searching the event following B->C : +The query statement searching the event following A->B : ``` sql CREATE TABLE test_flow ( dt DateTime, id int, - action String) + page String) ENGINE = MergeTree() PARTITION BY toYYYYMMDD(dt) ORDER BY id; INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); -SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') as next_flow FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; ``` Result: ``` text ┌─id─┬─next_flow─┐ -│ 1 │ E │ +│ 1 │ C │ └────┴───────────┘ ``` + +**Behavior for `forward` and `head`** + +```SQL +ALTER TABLE test_flow DELETE WHERE 1 = 1 settings mutations_sync = 1; + +INSERT INTO test_flow VALUES (1, 1, 'Home') (2, 1, 'Gift') (3, 1, 'Exit'); +INSERT INTO test_flow VALUES (1, 2, 'Home') (2, 2, 'Home') (3, 2, 'Gift') (4, 2, 'Basket'); +INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, 'Basket'); +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page + 1970-01-01 09:00:01 1 Home // Base point, Matched with Home + 1970-01-01 09:00:02 1 Gift // Matched with Gift + 1970-01-01 09:00:03 1 Exit // The result + + 1970-01-01 09:00:01 3 Home // Base point, Matched with Home + 1970-01-01 09:00:02 3 Home // Unmatched with Gift + 1970-01-01 09:00:03 3 Gift + 1970-01-01 09:00:04 3 Basket + + 1970-01-01 09:00:01 4 Gift // Base point, Unmatched with Home + 1970-01-01 09:00:02 4 Home + 1970-01-01 09:00:03 4 Gift + 1970-01-01 09:00:04 4 Basket +``` + +**Behavior for `backward` and `tail`** + +```SQL +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift +1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Matched with Gift +1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket + +1970-01-01 09:00:01 4 Gift +1970-01-01 09:00:02 4 Home // The result +1970-01-01 09:00:03 4 Gift // Base point, Matched with Gift +1970-01-01 09:00:04 4 Basket // Base point, Matched with Basket +``` + + +**Behavior for `forward` and `first_match`** + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit // The result + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket The result + +1970-01-01 09:00:01 4 Gift // Base point +1970-01-01 09:00:02 4 Home // Thre result +1970-01-01 09:00:03 4 Gift +1970-01-01 09:00:04 4 Basket +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit // Unmatched with Home + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket // Unmatched with Home + +1970-01-01 09:00:01 4 Gift // Base point +1970-01-01 09:00:02 4 Home // Matched with Home +1970-01-01 09:00:03 4 Gift // The result +1970-01-01 09:00:04 4 Basket +``` + + +**Behavior for `backward` and `last_match`** + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home // The result +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit + +1970-01-01 09:00:01 3 Home +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket + +1970-01-01 09:00:01 4 Gift +1970-01-01 09:00:02 4 Home // The result +1970-01-01 09:00:03 4 Gift // Base point +1970-01-01 09:00:04 4 Basket +``` + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; + + dt id page +1970-01-01 09:00:01 1 Home // Matched with Home, the result is null +1970-01-01 09:00:02 1 Gift // Base point +1970-01-01 09:00:03 1 Exit + +1970-01-01 09:00:01 3 Home // The result +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket + +1970-01-01 09:00:01 4 Gift // The result +1970-01-01 09:00:02 4 Home // Matched with Home +1970-01-01 09:00:03 4 Gift // Base point +1970-01-01 09:00:04 4 Basket +``` diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 9c865f3371a..274b75b383e 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -11,6 +11,8 @@ namespace DB { +constexpr size_t MAX_EVENTS_SIZE = 64; + namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; @@ -21,26 +23,26 @@ namespace ErrorCodes namespace { -constexpr size_t MAX_EVENTS_SIZE = 64; +template +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl2(const DataTypePtr data_type, const DataTypes & argument_types, SeqBase base) +{ + if (base == HEAD) + return std::make_shared, Direction, HEAD>>(data_type, argument_types); + else if (base == TAIL) + return std::make_shared, Direction, TAIL>>(data_type, argument_types); + else if (base == FIRST_MATCH) + return std::make_shared, Direction, FIRST_MATCH>>(data_type, argument_types); + else + return std::make_shared, Direction, LAST_MATCH>>(data_type, argument_types); +} template -inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl(const DataTypePtr data_type, const DataTypes & argument_types, bool descending_order) +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl1(const DataTypePtr data_type, const DataTypes & argument_types, SeqDirection direction, SeqBase base) { - if (argument_types.size() == 2) - { - // If the number of arguments of sequenceNextNode is 2, the sequenceNextNode acts as sequenceFirstNode. - if (descending_order) - return std::make_shared, true>>(data_type); - else - return std::make_shared, false>>(data_type); - } + if (direction == FORWARD) + return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); else - { - if (descending_order) - return std::make_shared, true>>(data_type, argument_types); - else - return std::make_shared, false>>(data_type, argument_types); - } + return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); } AggregateFunctionPtr @@ -48,22 +50,32 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c { assert(max_args <= MAX_EVENTS_SIZE); - bool descending_order = false; - - if (parameters.size() == 1) - { - auto type = parameters[0].getType(); - bool is_correct_type = type == Field::Types::Int64 || type == Field::Types::UInt64; - if (!is_correct_type || (parameters[0].get() != 0 && parameters[0].get() != 1)) - throw Exception("The first parameter for aggregate function " + name + " should be 0 or 1", ErrorCodes::BAD_ARGUMENTS); - - descending_order = parameters[0].get(); - } + String param_dir = parameters.at(0).safeGet(); + SeqDirection direction; + if (param_dir.compare("forward") == 0) + direction = FORWARD; + else if (param_dir.compare("backward") == 0) + direction = BACKWARD; else - throw Exception("Incorrect number of parameters for aggregate function " + name + ", should be 1", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS}; - if (argument_types.size() < 2) + String param_base = parameters.at(1).safeGet(); + SeqBase base; + if (param_base.compare("head") == 0) + base = HEAD; + else if (param_base.compare("tail") == 0) + base = TAIL; + else if (param_base.compare("first_match") == 0) + base = FIRST_MATCH; + else if (param_base.compare("last_match") == 0) + base = LAST_MATCH; + else + throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; + + if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < 3) + throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() < 2) throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); else if (argument_types.size() > max_args + 2) @@ -89,17 +101,17 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt16) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt32) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt64) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.isDate()) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); if (timestamp_type.isDateTime()) - return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, descending_order); + return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); throw Exception{"Illegal type " + argument_types.front().get()->getName() + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index a04f589f1e8..78d19beedc4 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -27,6 +27,20 @@ namespace DB { +enum SeqDirection +{ + FORWARD = 0, + BACKWARD = 1 +}; + +enum SeqBase +{ + HEAD = 0, + TAIL = 1, + FIRST_MATCH = 2, + LAST_MATCH = 3 +}; + /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl template struct NodeBase @@ -104,7 +118,7 @@ struct NodeString : public NodeBase, MaxEventsSize> }; /// TODO : Expends SequenceNextNodeGeneralData to support other types -template +template struct SequenceNextNodeGeneralData { using Allocator = MixedAlignedArenaAllocator; @@ -117,10 +131,7 @@ struct SequenceNextNodeGeneralData { bool operator()(const Node * lhs, const Node * rhs) const { - if constexpr (Descending) - return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time > rhs->event_time; - else - return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; + return lhs->event_time == rhs->event_time ? lhs->compare(rhs) : lhs->event_time < rhs->event_time; } }; @@ -134,12 +145,12 @@ struct SequenceNextNodeGeneralData } }; -/// Implementation of sequenceNextNode -template +/// Implementation of sequenceFirstNode +template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { - using Data = SequenceNextNodeGeneralData; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } @@ -149,7 +160,7 @@ class SequenceNextNodeImpl final public: SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) , events_size(arguments.size() - 2) @@ -165,12 +176,11 @@ public: const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties &) const override { - /// This aggregate function sets insertion_requires_nullable_column on. /// Even though some values are mapped to aggregating key, it could return nulls for the below case. /// aggregated events: [A -> B -> C] /// events to find: [C -> D] /// [C -> D] is not matched to 'A -> B -> C' so that it returns null. - return std::make_shared>(nested_function, arguments, params); + return std::make_shared>(nested_function, arguments, params); } void insert(Data & a, const Node * v, Arena * arena) const @@ -222,7 +232,7 @@ public: a.push_back(b[i]->clone(arena), arena); /// Either sort whole container or do so partially merging ranges afterwards - using Comparator = typename SequenceNextNodeGeneralData::Comparator; + using Comparator = typename SequenceNextNodeGeneralData::Comparator; if (!data(place).sorted && !data(rhs).sorted) std::stable_sort(std::begin(a), std::end(a), Comparator{}); @@ -246,12 +256,35 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { + // Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node. + this->data(const_cast(place)).sort(); + writeBinary(data(place).sorted, buf); auto & value = data(place).value; - writeVarUInt(value.size(), buf); - for (auto & node : value) - node->write(buf); + + UInt64 size = std::max(static_cast(events_size + 1), value.size()); + switch (Base) + { + case HEAD: + writeVarUInt(size, buf); + for (UInt64 i = 0; i < size; ++i) + value[i]->write(buf); + break; + + case TAIL: + writeVarUInt(size, buf); + for (UInt64 i = value.size() - 1; i >= size; --i) + value[i]->write(buf); + break; + + case FIRST_MATCH: + case LAST_MATCH: + writeVarUInt(value.size(), buf); + for (auto & node : value) + node->write(buf); + break; + } } void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override @@ -271,60 +304,83 @@ public: value[i] = Node::read(buf, arena); } - /// Calculate position of current event in target chain and shift to corresponding offset - /// Lets consider case where we search chain 'ABCD': - /// - If current event is 'X' we can skip it and perform next step from position after this 'X' - /// - If current event is 'A' we will start from this position - /// - If current event is 'B' then second position in our chain should match this 'B'. - /// And we perform next step from position one before 'B'. - /// - And so on... - inline UInt32 calculateJump(const Data & data, const UInt32 i, const UInt32 j) const + inline UInt32 getBaseIndex(Data & data, bool & exist) const { - /// Fast check if value is zero, not in sequence - if (data.value[i - j]->events_bitset.none()) - return events_size - j; + switch (Base) + { + case HEAD: + exist = true; + return 0; - UInt32 k = 1; - for (; k < events_size - j; ++k) - if (data.value[i - j]->events_bitset.test(events_size - 1 - j - k)) + case TAIL: + exist = true; + return data.value.size() - 1; + + case FIRST_MATCH: + for (UInt64 i = 0; i < data.value.size(); ++i) + if (data.value[i]->events_bitset.test(0)) + { + exist = true; + return i; + } break; - return k; + + case LAST_MATCH: + for (UInt64 i = 0; i < data.value.size(); ++i) + { + auto reversed_i = data.value.size() - i - 1; + if (data.value[reversed_i]->events_bitset.test(0)) + { + exist = true; + return reversed_i; + } + } + break; + } + + exist = false; + return 0; } /// This method returns an index of next node that matched the events. - /// It is one as referring Boyer-Moore-Algorithm(https://en.wikipedia.org/wiki/Boyer%E2%80%93Moore_string-search_algorithm). - /// But, there are some differences. - /// In original Boyer-Moore-Algorithm compares strings, but this algorithm compares events_bits. - /// Matched events in the chain of events are represented as a bitmask. + /// matched events in the chain of events are represented as a bitmask. /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { + const UInt32 unmatched = data.value.size(); + if (data.value.size() <= events_size) - return 0; + return unmatched; data.sort(); - UInt32 i = events_size - 1; - while (i < data.value.size()) + bool base_existence; + UInt32 base = getBaseIndex(data, base_existence); + if (!base_existence) + return unmatched; + + if (events_size == 0) { - UInt32 j = 0; - /// Try to match chain of events starting from the end of this chain. - for (; j < events_size; ++j) - { - /// It compares each matched events. - /// The lower bitmask is the former matched event. - if (!data.value[i - j]->events_bitset.test(events_size - 1 - j)) - break; - } - - /// Chain of events are matched, return the index of result value. - if (j == events_size) - return i + 1; - - i += calculateJump(data, i, j); + return data.value.size() > 0 ? base : unmatched; } + else + { + UInt32 i = 0; + switch (Direction) + { + case FORWARD: + for (i = 0; i < events_size && base + i < data.value.size(); ++i) + if (data.value[base + i]->events_bitset.test(i) == false) + break; + return (i == events_size) ? base + i : unmatched; - return 0; + case BACKWARD: + for (i = 0; i < events_size && i < base; ++i) + if (data.value[base - i]->events_bitset.test(i) == false) + break; + return (i == events_size) ? base - i : unmatched; + } + } } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override @@ -332,7 +388,7 @@ public: auto & value = data(place).value; UInt32 event_idx = getNextNodeIndex(this->data(place)); - if (event_idx != 0 && event_idx < value.size()) + if (event_idx < value.size()) { ColumnNullable & to_concrete = assert_cast(to); value[event_idx]->insertInto(to_concrete.getNestedColumn()); @@ -345,139 +401,4 @@ public: bool allocatesMemoryInArena() const override { return true; } }; -/// Implementation of sequenceFirstNode -template -class SequenceFirstNodeImpl final - : public IAggregateFunctionDataHelper, SequenceFirstNodeImpl> -{ - using Data = SequenceNextNodeGeneralData; - static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } - static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - - DataTypePtr & data_type; - -public: - explicit SequenceFirstNodeImpl(const DataTypePtr & data_type_) - : IAggregateFunctionDataHelper, SequenceFirstNodeImpl>( - {data_type_}, {}) - , data_type(this->argument_types[0]) - { - } - - String getName() const override { return "sequenceFirstNode"; } - - DataTypePtr getReturnType() const override { return data_type; } - - AggregateFunctionPtr getOwnNullAdapter( - const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, - const AggregateFunctionProperties &) const override - { - return std::make_shared>(nested_function, arguments, params); - } - - void insert(Data & a, const Node * v, Arena * arena) const - { - ++a.total_values; - a.value.push_back(v->clone(arena), arena); - } - - void create(AggregateDataPtr __restrict place) const override - { - new (place) Data; - } - - bool compare(const T lhs_timestamp, const T rhs_timestamp) const - { - return Descending ? lhs_timestamp < rhs_timestamp : lhs_timestamp > rhs_timestamp; - } - - void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override - { - bool is_first = true; - auto & value = data(place).value; - const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; - - if (value.size() != 0) - { - if (compare(value[0]->event_time, timestamp)) - value.pop_back(); - else - is_first = false; - } - - if (is_first) - { - Node * node = Node::allocate(*columns[1], row_num, arena); - node->event_time = timestamp; - node->events_bitset.reset(); - - data(place).value.push_back(node, arena); - } - } - - void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override - { - auto & a = data(place).value; - auto & b = data(rhs).value; - - if (b.empty()) - return; - - if (a.empty()) - { - a.push_back(b[0]->clone(arena), arena); - return; - } - - if (compare(a[0]->event_time, b[0]->event_time)) - { - data(place).value.pop_back(); - a.push_back(b[0]->clone(arena), arena); - } - } - - void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override - { - writeBinary(data(place).sorted, buf); - - auto & value = data(place).value; - writeVarUInt(value.size(), buf); - for (auto & node : value) - node->write(buf); - } - - void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, Arena * arena) const override - { - readBinary(data(place).sorted, buf); - - UInt64 size; - readVarUInt(size, buf); - - if (unlikely(size == 0)) - return; - - auto & value = data(place).value; - - value.resize(size, arena); - for (UInt64 i = 0; i < size; ++i) - value[i] = Node::read(buf, arena); - } - - void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override - { - auto & value = data(place).value; - - if (value.size() > 0) - { - ColumnNullable & to_concrete = assert_cast(to); - value[0]->insertInto(to_concrete.getNestedColumn()); - to_concrete.getNullMapData().push_back(0); - } - else - to.insertDefault(); - } - - bool allocatesMemoryInArena() const override { return true; } -}; - } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 94a55f2d954..9309641f3bf 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -1,316 +1,461 @@ -(0, A) 1 B -(0, A) 2 B -(0, A) 3 B -(0, A) 4 A -(0, A) 5 B -(0, A) 6 B -(0, B) 1 C -(0, B) 2 D -(0, B) 3 \N -(0, B) 4 C -(0, B) 5 A -(0, B) 6 A -(0, C) 1 D -(0, C) 2 \N -(0, C) 3 \N -(0, C) 4 \N -(0, C) 5 \N -(0, C) 6 \N -(0, D) 1 \N -(0, D) 2 C -(0, D) 3 \N -(0, D) 4 \N -(0, D) 5 \N -(0, D) 6 \N -(0, E) 1 \N -(0, E) 2 \N -(0, E) 3 \N -(0, E) 4 \N -(0, E) 5 \N -(0, E) 6 \N -(1, A) 1 \N -(1, A) 2 \N -(1, A) 3 \N -(1, A) 4 A -(1, A) 5 B -(1, A) 6 B -(1, B) 1 A -(1, B) 2 A -(1, B) 3 A -(1, B) 4 A -(1, B) 5 A -(1, B) 6 A -(1, C) 1 B -(1, C) 2 D -(1, C) 3 \N -(1, C) 4 B -(1, C) 5 A -(1, C) 6 B -(1, D) 1 C -(1, D) 2 B -(1, D) 3 \N -(1, D) 4 \N -(1, D) 5 \N -(1, D) 6 \N -(1, E) 1 \N -(1, E) 2 \N -(1, E) 3 \N -(1, E) 4 \N -(1, E) 5 \N -(1, E) 6 \N -(0, A->B) 1 C -(0, A->B) 2 D -(0, A->B) 3 \N -(0, A->B) 4 C -(0, A->B) 5 A -(0, A->B) 6 A -(0, A->C) 1 \N -(0, A->C) 2 \N -(0, A->C) 3 \N -(0, A->C) 4 \N -(0, A->C) 5 \N -(0, A->C) 6 \N -(0, B->A) 1 \N -(0, B->A) 2 \N -(0, B->A) 3 \N -(0, B->A) 4 \N -(0, B->A) 5 C -(0, B->A) 6 B -(1, A->B) 1 \N -(1, A->B) 2 \N -(1, A->B) 3 \N -(1, A->B) 4 \N -(1, A->B) 5 A -(1, A->B) 6 A -(1, A->C) 1 \N -(1, A->C) 2 \N -(1, A->C) 3 \N -(1, A->C) 4 \N -(1, A->C) 5 \N -(1, A->C) 6 \N -(1, B->A) 1 \N -(1, B->A) 2 \N -(1, B->A) 3 \N -(1, B->A) 4 A -(1, B->A) 5 \N -(1, B->A) 6 B -(0, A->A->B) 1 \N -(0, A->A->B) 2 \N -(0, A->A->B) 3 \N -(0, A->A->B) 4 C -(0, A->A->B) 5 \N -(0, A->A->B) 6 \N -(0, B->A->A) 1 \N -(0, B->A->A) 2 \N -(0, B->A->A) 3 \N -(0, B->A->A) 4 \N -(0, B->A->A) 5 \N -(0, B->A->A) 6 \N -(1, A->A->B) 1 \N -(1, A->A->B) 2 \N -(1, A->A->B) 3 \N -(1, A->A->B) 4 \N -(1, A->A->B) 5 \N -(1, A->A->B) 6 \N -(1, B->A->A) 1 \N -(1, B->A->A) 2 \N -(1, B->A->A) 3 \N -(1, B->A->A) 4 A -(1, B->A->A) 5 \N -(1, B->A->A) 6 \N -(0, A) id >= 10 10 B -(0, A) id >= 10 10 C -(0, A) id >= 10 10 D -(0, A) id >= 10 10 B -(0, A) id >= 10 10 B -(0, A) id >= 10 10 A +(forward, head, A) 1 B +(forward, head, A) 2 \N +(forward, head, A) 3 B +(forward, head, A) 4 A +(forward, head, A) 5 B +(forward, head, A) 6 B +(forward, head, B) 1 \N +(forward, head, B) 2 B +(forward, head, B) 3 \N +(forward, head, B) 4 \N +(forward, head, B) 5 \N +(forward, head, B) 6 \N +(forward, head, C) 1 \N +(forward, head, C) 2 \N +(forward, head, C) 3 \N +(forward, head, C) 4 \N +(forward, head, C) 5 \N +(forward, head, C) 6 \N +(forward, head, D) 1 \N +(forward, head, D) 2 \N +(forward, head, D) 3 \N +(forward, head, D) 4 \N +(forward, head, D) 5 \N +(forward, head, D) 6 \N +(forward, head, E) 1 \N +(forward, head, E) 2 \N +(forward, head, E) 3 \N +(forward, head, E) 4 \N +(forward, head, E) 5 \N +(forward, head, E) 6 \N +(backward, tail, A) 1 \N +(backward, tail, A) 2 \N +(backward, tail, A) 3 \N +(backward, tail, A) 4 \N +(backward, tail, A) 5 \N +(backward, tail, A) 6 \N +(backward, tail, B) 1 \N +(backward, tail, B) 2 \N +(backward, tail, B) 3 A +(backward, tail, B) 4 \N +(backward, tail, B) 5 \N +(backward, tail, B) 6 \N +(backward, tail, C) 1 \N +(backward, tail, C) 2 D +(backward, tail, C) 3 \N +(backward, tail, C) 4 B +(backward, tail, C) 5 A +(backward, tail, C) 6 B +(backward, tail, D) 1 C +(backward, tail, D) 2 \N +(backward, tail, D) 3 \N +(backward, tail, D) 4 \N +(backward, tail, D) 5 \N +(backward, tail, D) 6 \N +(backward, tail, E) 1 \N +(backward, tail, E) 2 \N +(backward, tail, E) 3 \N +(backward, tail, E) 4 \N +(backward, tail, E) 5 \N +(backward, tail, E) 6 \N +(forward, head, A->B) 1 C +(forward, head, A->B) 2 \N +(forward, head, A->B) 3 \N +(forward, head, A->B) 4 \N +(forward, head, A->B) 5 A +(forward, head, A->B) 6 A +(forward, head, A->C) 1 \N +(forward, head, A->C) 2 \N +(forward, head, A->C) 3 \N +(forward, head, A->C) 4 \N +(forward, head, A->C) 5 \N +(forward, head, A->C) 6 \N +(forward, head, B->A) 1 \N +(forward, head, B->A) 2 \N +(forward, head, B->A) 3 \N +(forward, head, B->A) 4 \N +(forward, head, B->A) 5 \N +(forward, head, B->A) 6 \N +(backward, tail, A->B) 1 \N +(backward, tail, A->B) 2 \N +(backward, tail, A->B) 3 \N +(backward, tail, A->B) 4 \N +(backward, tail, A->B) 5 \N +(backward, tail, A->B) 6 \N +(backward, tail, A->C) 1 \N +(backward, tail, A->C) 2 \N +(backward, tail, A->C) 3 \N +(backward, tail, A->C) 4 \N +(backward, tail, A->C) 5 \N +(backward, tail, A->C) 6 \N +(backward, tail, B->A) 1 \N +(backward, tail, B->A) 2 \N +(backward, tail, B->A) 3 \N +(backward, tail, B->A) 4 \N +(backward, tail, B->A) 5 \N +(backward, tail, B->A) 6 \N +(forward, head, A->A->B) 1 \N +(forward, head, A->A->B) 2 \N +(forward, head, A->A->B) 3 \N +(forward, head, A->A->B) 4 \N +(forward, head, A->A->B) 5 \N +(forward, head, A->A->B) 6 \N +(forward, head, B->A->A) 1 \N +(forward, head, B->A->A) 2 \N +(forward, head, B->A->A) 3 \N +(forward, head, B->A->A) 4 \N +(forward, head, B->A->A) 5 \N +(forward, head, B->A->A) 6 \N +(backward, tail, A->A->B) 1 \N +(backward, tail, A->A->B) 2 \N +(backward, tail, A->A->B) 3 \N +(backward, tail, A->A->B) 4 \N +(backward, tail, A->A->B) 5 \N +(backward, tail, A->A->B) 6 \N +(backward, tail, B->A->A) 1 \N +(backward, tail, B->A->A) 2 \N +(backward, tail, B->A->A) 3 \N +(backward, tail, B->A->A) 4 \N +(backward, tail, B->A->A) 5 \N +(backward, tail, B->A->A) 6 \N +(forward, head, A) id >= 10 10 B +(forward, head, A) id >= 10 10 C +(forward, head, A) id >= 10 10 \N +(forward, head, A) id >= 10 10 \N +(backward, tail, A) id >= 10 10 B +(backward, tail, A) id >= 10 10 A (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 +(0, C) id = 11 0 +(0, B->C) id = 11 0 (0, A->B->C) id = 11 1 (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 -(0, B->C->D) id = 11 1 -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C -(0, A) 1 B -(0, A) 2 B -(0, A) 3 B -(0, A) 4 A -(0, A) 5 B -(0, A) 6 B -(0, B) 1 C -(0, B) 2 D -(0, B) 3 \N -(0, B) 4 C -(0, B) 5 A -(0, B) 6 A -(0, C) 1 D -(0, C) 2 \N -(0, C) 3 \N -(0, C) 4 \N -(0, C) 5 \N -(0, C) 6 \N -(0, D) 1 \N -(0, D) 2 C -(0, D) 3 \N -(0, D) 4 \N -(0, D) 5 \N -(0, D) 6 \N -(0, E) 1 \N -(0, E) 2 \N -(0, E) 3 \N -(0, E) 4 \N -(0, E) 5 \N -(0, E) 6 \N -(1, A) 1 \N -(1, A) 2 \N -(1, A) 3 \N -(1, A) 4 A -(1, A) 5 B -(1, A) 6 B -(1, B) 1 A -(1, B) 2 A -(1, B) 3 A -(1, B) 4 A -(1, B) 5 A -(1, B) 6 A -(1, C) 1 B -(1, C) 2 D -(1, C) 3 \N -(1, C) 4 B -(1, C) 5 A -(1, C) 6 B -(1, D) 1 C -(1, D) 2 B -(1, D) 3 \N -(1, D) 4 \N -(1, D) 5 \N -(1, D) 6 \N -(1, E) 1 \N -(1, E) 2 \N -(1, E) 3 \N -(1, E) 4 \N -(1, E) 5 \N -(1, E) 6 \N -(0, A->B) 1 C -(0, A->B) 2 D -(0, A->B) 3 \N -(0, A->B) 4 C -(0, A->B) 5 A -(0, A->B) 6 A -(0, A->C) 1 \N -(0, A->C) 2 \N -(0, A->C) 3 \N -(0, A->C) 4 \N -(0, A->C) 5 \N -(0, A->C) 6 \N -(0, B->A) 1 \N -(0, B->A) 2 \N -(0, B->A) 3 \N -(0, B->A) 4 \N -(0, B->A) 5 C -(0, B->A) 6 B -(1, A->B) 1 \N -(1, A->B) 2 \N -(1, A->B) 3 \N -(1, A->B) 4 \N -(1, A->B) 5 A -(1, A->B) 6 A -(1, A->C) 1 \N -(1, A->C) 2 \N -(1, A->C) 3 \N -(1, A->C) 4 \N -(1, A->C) 5 \N -(1, A->C) 6 \N -(1, B->A) 1 \N -(1, B->A) 2 \N -(1, B->A) 3 \N -(1, B->A) 4 A -(1, B->A) 5 \N -(1, B->A) 6 B -(0, A->A->B) 1 \N -(0, A->A->B) 2 \N -(0, A->A->B) 3 \N -(0, A->A->B) 4 C -(0, A->A->B) 5 \N -(0, A->A->B) 6 \N -(0, B->A->A) 1 \N -(0, B->A->A) 2 \N -(0, B->A->A) 3 \N -(0, B->A->A) 4 \N -(0, B->A->A) 5 \N -(0, B->A->A) 6 \N -(1, A->A->B) 1 \N -(1, A->A->B) 2 \N -(1, A->A->B) 3 \N -(1, A->A->B) 4 \N -(1, A->A->B) 5 \N -(1, A->A->B) 6 \N -(1, B->A->A) 1 \N -(1, B->A->A) 2 \N -(1, B->A->A) 3 \N -(1, B->A->A) 4 A -(1, B->A->A) 5 \N -(1, B->A->A) 6 \N -(0, A) id = 10 10 B -(0, A->B) id = 10 10 C -(0, B->C) id = 10 10 D -(0, C) id = 10 10 B -(0, D->C) id = 10 10 B -(0, C->B) id = 10 10 A +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 1 +(forward, head) id < 10 1 A +(forward, head) id < 10 2 B +(forward, head) id < 10 3 A +(forward, head) id < 10 4 A +(forward, head) id < 10 5 A +(forward, head) id < 10 6 A +(backward, tail) id < 10 1 D +(backward, tail) id < 10 2 C +(backward, tail) id < 10 3 B +(backward, tail) id < 10 4 C +(backward, tail) id < 10 5 C +(backward, tail) id < 10 6 C +(forward, first_match, A) 1 B +(forward, first_match, A) 2 \N +(forward, first_match, A) 3 B +(forward, first_match, A) 4 A +(forward, first_match, A) 5 B +(forward, first_match, A) 6 B +(forward, first_match, A) 10 B +(forward, first_match, A) 11 B +(forward, first_match, A->B) 1 C +(forward, first_match, A->B) 2 \N +(forward, first_match, A->B) 3 \N +(forward, first_match, A->B) 4 \N +(forward, first_match, A->B) 5 A +(forward, first_match, A->B) 6 A +(forward, first_match, A->B) 10 C +(forward, first_match, A->B) 11 C +(forward, first_match, A->B->C) 1 D +(forward, first_match, A->B->C) 2 \N +(forward, first_match, A->B->C) 3 \N +(forward, first_match, A->B->C) 4 \N +(forward, first_match, A->B->C) 5 \N +(forward, first_match, A->B->C) 6 \N +(forward, first_match, A->B->C) 10 D +(forward, first_match, A->B->C) 11 D +(forward, first_match, B) 1 C +(forward, first_match, B) 2 B +(forward, first_match, B) 3 \N +(forward, first_match, B) 4 C +(forward, first_match, B) 5 A +(forward, first_match, B) 6 A +(forward, first_match, B) 10 C +(forward, first_match, B) 11 C +(forward, first_match, B->B) 1 \N +(forward, first_match, B->B) 2 D +(forward, first_match, B->B) 3 \N +(forward, first_match, B->B) 4 \N +(forward, first_match, B->B) 5 \N +(forward, first_match, B->B) 6 \N +(forward, first_match, B->B) 10 \N +(forward, first_match, B->B) 11 \N +(forward, first_match, B->A) 1 \N +(forward, first_match, B->A) 2 \N +(forward, first_match, B->A) 3 \N +(forward, first_match, B->A) 4 \N +(forward, first_match, B->A) 5 C +(forward, first_match, B->A) 6 B +(forward, first_match, B->A) 10 \N +(forward, first_match, B->A) 11 \N +(backward, first_match, A) 1 \N +(backward, first_match, A) 2 \N +(backward, first_match, A) 3 \N +(backward, first_match, A) 4 \N +(backward, first_match, A) 5 \N +(backward, first_match, A) 6 \N +(backward, first_match, A) 10 \N +(backward, first_match, A) 11 \N +(backward, first_match, B) 1 A +(backward, first_match, B) 2 \N +(backward, first_match, B) 3 A +(backward, first_match, B) 4 A +(backward, first_match, B) 5 A +(backward, first_match, B) 6 A +(backward, first_match, B) 10 A +(backward, first_match, B) 11 A +(backward, first_match, B->A) 1 \N +(backward, first_match, B->A) 2 \N +(backward, first_match, B->A) 3 \N +(backward, first_match, B->A) 4 A +(backward, first_match, B->A) 5 \N +(backward, first_match, B->A) 6 \N +(backward, first_match, B->A) 10 \N +(backward, first_match, B->A) 11 \N +(backward, first_match, B->B) 1 \N +(backward, first_match, B->B) 2 \N +(backward, first_match, B->B) 3 \N +(backward, first_match, B->B) 4 \N +(backward, first_match, B->B) 5 \N +(backward, first_match, B->B) 6 \N +(backward, first_match, B->B) 10 \N +(backward, first_match, B->B) 11 \N +(forward, head, A) 1 B +(forward, head, A) 2 \N +(forward, head, A) 3 B +(forward, head, A) 4 A +(forward, head, A) 5 B +(forward, head, A) 6 B +(forward, head, B) 1 \N +(forward, head, B) 2 B +(forward, head, B) 3 \N +(forward, head, B) 4 \N +(forward, head, B) 5 \N +(forward, head, B) 6 \N +(forward, head, C) 1 \N +(forward, head, C) 2 \N +(forward, head, C) 3 \N +(forward, head, C) 4 \N +(forward, head, C) 5 \N +(forward, head, C) 6 \N +(forward, head, D) 1 \N +(forward, head, D) 2 \N +(forward, head, D) 3 \N +(forward, head, D) 4 \N +(forward, head, D) 5 \N +(forward, head, D) 6 \N +(forward, head, E) 1 \N +(forward, head, E) 2 \N +(forward, head, E) 3 \N +(forward, head, E) 4 \N +(forward, head, E) 5 \N +(forward, head, E) 6 \N +(backward, tail, A) 1 \N +(backward, tail, A) 2 \N +(backward, tail, A) 3 \N +(backward, tail, A) 4 \N +(backward, tail, A) 5 \N +(backward, tail, A) 6 \N +(backward, tail, B) 1 \N +(backward, tail, B) 2 \N +(backward, tail, B) 3 A +(backward, tail, B) 4 \N +(backward, tail, B) 5 \N +(backward, tail, B) 6 \N +(backward, tail, C) 1 \N +(backward, tail, C) 2 D +(backward, tail, C) 3 \N +(backward, tail, C) 4 B +(backward, tail, C) 5 A +(backward, tail, C) 6 B +(backward, tail, D) 1 C +(backward, tail, D) 2 \N +(backward, tail, D) 3 \N +(backward, tail, D) 4 \N +(backward, tail, D) 5 \N +(backward, tail, D) 6 \N +(backward, tail, E) 1 \N +(backward, tail, E) 2 \N +(backward, tail, E) 3 \N +(backward, tail, E) 4 \N +(backward, tail, E) 5 \N +(backward, tail, E) 6 \N +(forward, head, A->B) 1 C +(forward, head, A->B) 2 \N +(forward, head, A->B) 3 \N +(forward, head, A->B) 4 \N +(forward, head, A->B) 5 A +(forward, head, A->B) 6 A +(forward, head, A->C) 1 \N +(forward, head, A->C) 2 \N +(forward, head, A->C) 3 \N +(forward, head, A->C) 4 \N +(forward, head, A->C) 5 \N +(forward, head, A->C) 6 \N +(forward, head, B->A) 1 \N +(forward, head, B->A) 2 \N +(forward, head, B->A) 3 \N +(forward, head, B->A) 4 \N +(forward, head, B->A) 5 \N +(forward, head, B->A) 6 \N +(backward, tail, A->B) 1 \N +(backward, tail, A->B) 2 \N +(backward, tail, A->B) 3 \N +(backward, tail, A->B) 4 \N +(backward, tail, A->B) 5 \N +(backward, tail, A->B) 6 \N +(backward, tail, A->C) 1 \N +(backward, tail, A->C) 2 \N +(backward, tail, A->C) 3 \N +(backward, tail, A->C) 4 \N +(backward, tail, A->C) 5 \N +(backward, tail, A->C) 6 \N +(backward, tail, B->A) 1 \N +(backward, tail, B->A) 2 \N +(backward, tail, B->A) 3 \N +(backward, tail, B->A) 4 \N +(backward, tail, B->A) 5 \N +(backward, tail, B->A) 6 \N +(forward, head, A->A->B) 1 \N +(forward, head, A->A->B) 2 \N +(forward, head, A->A->B) 3 \N +(forward, head, A->A->B) 4 \N +(forward, head, A->A->B) 5 \N +(forward, head, A->A->B) 6 \N +(forward, head, B->A->A) 1 \N +(forward, head, B->A->A) 2 \N +(forward, head, B->A->A) 3 \N +(forward, head, B->A->A) 4 \N +(forward, head, B->A->A) 5 \N +(forward, head, B->A->A) 6 \N +(backward, tail, A->A->B) 1 \N +(backward, tail, A->A->B) 2 \N +(backward, tail, A->A->B) 3 \N +(backward, tail, A->A->B) 4 \N +(backward, tail, A->A->B) 5 \N +(backward, tail, A->A->B) 6 \N +(backward, tail, B->A->A) 1 \N +(backward, tail, B->A->A) 2 \N +(backward, tail, B->A->A) 3 \N +(backward, tail, B->A->A) 4 \N +(backward, tail, B->A->A) 5 \N +(backward, tail, B->A->A) 6 \N +(forward, head, A) id >= 10 10 B +(forward, head, A) id >= 10 10 C +(forward, head, A) id >= 10 10 \N +(forward, head, A) id >= 10 10 \N +(backward, tail, A) id >= 10 10 B +(backward, tail, A) id >= 10 10 A (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 +(0, C) id = 11 0 +(0, B->C) id = 11 0 (0, A->B->C) id = 11 1 (0, A) id = 11 1 -(0, C) id = 11 1 -(0, B->C) id = 11 1 -(0, B->C->D) id = 11 1 -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(0) id < 10 1 A -(0) id < 10 2 A -(0) id < 10 3 A -(0) id < 10 4 A -(0) id < 10 5 A -(0) id < 10 6 A -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C -(1) id < 10 1 D -(1) id < 10 2 C -(1) id < 10 3 B -(1) id < 10 4 C -(1) id < 10 5 C -(1) id < 10 6 C +(0, C) id = 11 0 +(0, C->B) id = 11 0 +(0, C->B->A) id = 11 1 +(forward, head) id < 10 1 A +(forward, head) id < 10 2 B +(forward, head) id < 10 3 A +(forward, head) id < 10 4 A +(forward, head) id < 10 5 A +(forward, head) id < 10 6 A +(backward, tail) id < 10 1 D +(backward, tail) id < 10 2 C +(backward, tail) id < 10 3 B +(backward, tail) id < 10 4 C +(backward, tail) id < 10 5 C +(backward, tail) id < 10 6 C +(forward, first_match, A) 1 B +(forward, first_match, A) 2 \N +(forward, first_match, A) 3 B +(forward, first_match, A) 4 A +(forward, first_match, A) 5 B +(forward, first_match, A) 6 B +(forward, first_match, A) 10 B +(forward, first_match, A) 11 B +(forward, first_match, A->B) 1 C +(forward, first_match, A->B) 2 \N +(forward, first_match, A->B) 3 \N +(forward, first_match, A->B) 4 \N +(forward, first_match, A->B) 5 A +(forward, first_match, A->B) 6 A +(forward, first_match, A->B) 10 C +(forward, first_match, A->B) 11 C +(forward, first_match, A->B->C) 1 D +(forward, first_match, A->B->C) 2 \N +(forward, first_match, A->B->C) 3 \N +(forward, first_match, A->B->C) 4 \N +(forward, first_match, A->B->C) 5 \N +(forward, first_match, A->B->C) 6 \N +(forward, first_match, A->B->C) 10 D +(forward, first_match, A->B->C) 11 D +(forward, first_match, B) 1 C +(forward, first_match, B) 2 B +(forward, first_match, B) 3 \N +(forward, first_match, B) 4 C +(forward, first_match, B) 5 A +(forward, first_match, B) 6 A +(forward, first_match, B) 10 C +(forward, first_match, B) 11 C +(forward, first_match, B->B) 1 \N +(forward, first_match, B->B) 2 D +(forward, first_match, B->B) 3 \N +(forward, first_match, B->B) 4 \N +(forward, first_match, B->B) 5 \N +(forward, first_match, B->B) 6 \N +(forward, first_match, B->B) 10 \N +(forward, first_match, B->B) 11 \N +(forward, first_match, B->A) 1 \N +(forward, first_match, B->A) 2 \N +(forward, first_match, B->A) 3 \N +(forward, first_match, B->A) 4 \N +(forward, first_match, B->A) 5 C +(forward, first_match, B->A) 6 B +(forward, first_match, B->A) 10 \N +(forward, first_match, B->A) 11 \N +(backward, first_match, A) 1 \N +(backward, first_match, A) 2 \N +(backward, first_match, A) 3 \N +(backward, first_match, A) 4 \N +(backward, first_match, A) 5 \N +(backward, first_match, A) 6 \N +(backward, first_match, A) 10 \N +(backward, first_match, A) 11 \N +(backward, first_match, B) 1 A +(backward, first_match, B) 2 \N +(backward, first_match, B) 3 A +(backward, first_match, B) 4 A +(backward, first_match, B) 5 A +(backward, first_match, B) 6 A +(backward, first_match, B) 10 A +(backward, first_match, B) 11 A +(backward, first_match, B->A) 1 \N +(backward, first_match, B->A) 2 \N +(backward, first_match, B->A) 3 \N +(backward, first_match, B->A) 4 A +(backward, first_match, B->A) 5 \N +(backward, first_match, B->A) 6 \N +(backward, first_match, B->A) 10 \N +(backward, first_match, B->A) 11 \N +(backward, first_match, B->B) 1 \N +(backward, first_match, B->B) 2 \N +(backward, first_match, B->B) 3 \N +(backward, first_match, B->B) 4 \N +(backward, first_match, B->B) 5 \N +(backward, first_match, B->B) 6 \N +(backward, first_match, B->B) 10 \N +(backward, first_match, B->B) 11 \N +(max_args) 1 \N +(max_args) 2 \N +(max_args) 3 \N +(max_args) 4 \N +(max_args) 5 \N +(max_args) 6 \N +(max_args) 10 \N +(max_args) 11 \N +(forward, head, A) id = 12 A diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 57b0d7b1d8e..63810265bf9 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -6,7 +6,7 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',1,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',1,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',1,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',1,'D'); -INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',2,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',2,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',2,'D'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',2,'C'); @@ -27,26 +27,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); -SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); @@ -54,31 +54,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(0, A) id >= 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; @@ -86,13 +96,13 @@ DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; DROP TABLE IF EXISTS test_sequenceNextNode; -CREATE TABLE iF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; +CREATE TABLE IF NOT EXISTS test_sequenceNextNode (dt DateTime, id int, action String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',1,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',1,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',1,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',1,'D'); -INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',2,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',2,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',2,'D'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',2,'C'); @@ -113,56 +123,73 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(0, A)', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, B)', id, sequenceNextNode(0)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, C)', id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, D)', id, sequenceNextNode(0)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, E)', id, sequenceNextNode(0)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A)', id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, B)', id, sequenceNextNode(1)(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, C)', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, D)', id, sequenceNextNode(1)(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, E)', id, sequenceNextNode(1)(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, A->C)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, B->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A->C)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, B->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, A->A->B)', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(0, B->A->A)', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, A->A->B)', id, sequenceNextNode(1)(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(1, B->A->A)', id, sequenceNextNode(1)(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(0, A) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, A->B) id = 10', id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, B->C) id = 10', id, sequenceNextNode(0)(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, D->C) id = 10', id, sequenceNextNode(1)(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; -SELECT '(0, C->B) id = 10', id, sequenceNextNode(1)(dt, action, action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id = 10 GROUP BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(0)(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C->D) id = 11', count() FROM (SELECT id, sequenceNextNode(1)(dt, action, action = 'B', action = 'C', action = 'D') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(0) id < 10', id, sequenceNextNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(0) id < 10', id, sequenceFirstNode(0)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceNextNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(1) id < 10', id, sequenceFirstNode(1)(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; + +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; + +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); +INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); + +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; DROP TABLE IF EXISTS test_sequenceNextNode; From 34ccad75dcd0cb1180388a75174901da3c5d58c2 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Wed, 10 Mar 2021 21:14:30 +0900 Subject: [PATCH 0060/1060] Remove insertion_requires_nullable_column --- src/AggregateFunctions/AggregateFunctionNull.h | 17 +++++------------ 1 file changed, 5 insertions(+), 12 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 8e638742162..821398e715e 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -228,15 +228,15 @@ public: }; -template +template class AggregateFunctionNullVariadic final : public AggregateFunctionNullBase> + AggregateFunctionNullVariadic> { public: AggregateFunctionNullVariadic(AggregateFunctionPtr nested_function_, const DataTypes & arguments, const Array & params) : AggregateFunctionNullBase>(std::move(nested_function_), arguments, params), + AggregateFunctionNullVariadic>(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size()) { if (number_of_arguments == 1) @@ -283,15 +283,8 @@ public: ColumnNullable & to_concrete = assert_cast(to); if (this->getFlag(place)) { - if constexpr (insertion_requires_nullable_column) - { - this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete, arena); - } - else - { - this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); - } + this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); + to_concrete.getNullMapData().push_back(0); } else { From 1ec927e7666391f8df9e962633f11e19510fb26f Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 11 Mar 2021 12:48:24 +0900 Subject: [PATCH 0061/1060] Fix special-build-failure and add argument validation --- .../AggregateFunctionSequenceNextNode.cpp | 16 ++++++++++------ .../AggregateFunctionSequenceNextNode.h | 6 +++--- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 274b75b383e..b8f517cba20 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -50,24 +50,28 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c { assert(max_args <= MAX_EVENTS_SIZE); + if (parameters.size() < 2) + throw Exception("Aggregate function " + name + " requires 2 parameters (direction, head)", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + String param_dir = parameters.at(0).safeGet(); SeqDirection direction; - if (param_dir.compare("forward") == 0) + if (param_dir == "forward") direction = FORWARD; - else if (param_dir.compare("backward") == 0) + else if (param_dir == "backward") direction = BACKWARD; else throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS}; String param_base = parameters.at(1).safeGet(); SeqBase base; - if (param_base.compare("head") == 0) + if (param_base == "head") base = HEAD; - else if (param_base.compare("tail") == 0) + else if (param_base == "tail") base = TAIL; - else if (param_base.compare("first_match") == 0) + else if (param_base == "first_match") base = FIRST_MATCH; - else if (param_base.compare("last_match") == 0) + else if (param_base == "last_match") base = LAST_MATCH; else throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 78d19beedc4..f22ca1d61a6 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -263,18 +263,18 @@ public: auto & value = data(place).value; - UInt64 size = std::max(static_cast(events_size + 1), value.size()); + size_t size = std::max(static_cast(events_size + 1), value.size()); switch (Base) { case HEAD: writeVarUInt(size, buf); - for (UInt64 i = 0; i < size; ++i) + for (size_t i = 0; i < size; ++i) value[i]->write(buf); break; case TAIL: writeVarUInt(size, buf); - for (UInt64 i = value.size() - 1; i >= size; --i) + for (size_t i = value.size() - 1; i >= size; --i) value[i]->write(buf); break; From bf01eeb8839b78239bc84891bf3a0a483d4bfd81 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 11 Mar 2021 16:44:02 +0900 Subject: [PATCH 0062/1060] Fix a bug in sequenceNextNode --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index f22ca1d61a6..f58469f27bb 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -263,7 +263,7 @@ public: auto & value = data(place).value; - size_t size = std::max(static_cast(events_size + 1), value.size()); + size_t size = std::min(static_cast(events_size + 1), value.size()); switch (Base) { case HEAD: From 06bb4e2c7171f775cb1443cfee6441b989161bdd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 16 Mar 2021 13:00:49 +0300 Subject: [PATCH 0063/1060] wip --- tests/integration/helpers/cluster.py | 15 +++++----- tests/integration/helpers/hdfs_api.py | 12 ++++---- tests/integration/pytest.ini | 2 +- tests/integration/runner | 2 +- tests/integration/test_MemoryTracking/test.py | 3 -- .../test_access_control_on_cluster/test.py | 2 +- .../test_adaptive_granularity/test.py | 13 +++++---- .../test.py | 4 +-- .../test_alter_on_mixed_type_cluster/test.py | 2 +- tests/integration/test_check_table/test.py | 8 +++--- .../test_compression_codec_read/test.py | 2 ++ .../integration/test_dictionaries_ddl/test.py | 4 +-- .../test_dictionaries_mysql/test.py | 14 +++++----- .../configs/dictionaries/.gitignore | 2 ++ .../test_default_reading.py | 2 +- .../test_default_string.py | 3 +- .../test_dict_get_or_default.py | 2 +- .../test_disabled_mysql_server/test.py | 8 +++--- .../test.py | 2 +- .../test_format_avro_confluent/test.py | 4 --- .../test_insert_into_distributed/test.py | 2 +- .../test_join_set_family_s3/test.py | 3 -- tests/integration/test_log_family_s3/test.py | 3 -- tests/integration/test_merge_tree_s3/test.py | 3 -- .../test_merge_tree_s3_failover/test.py | 3 -- .../test_merge_tree_s3_restore/test.py | 3 -- .../test_merge_tree_s3_with_cache/test.py | 3 -- .../test_postgresql_protocol/test.py | 2 +- .../test_profile_events_s3/test.py | 3 -- tests/integration/test_quota/test.py | 2 +- .../test_replicated_database/test.py | 28 +++++++++---------- .../test_replicated_merge_tree_s3/test.py | 3 -- tests/integration/test_s3_with_https/test.py | 3 -- tests/integration/test_s3_with_proxy/test.py | 4 --- tests/integration/test_storage_mysql/test.py | 2 +- .../test_storage_postgresql/test.py | 1 + tests/integration/test_storage_s3/test.py | 3 -- .../test_storage_s3/test_redirect.py | 3 -- 38 files changed, 72 insertions(+), 108 deletions(-) create mode 100644 tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index eb6ecc8a39f..124f77061cb 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -579,7 +579,7 @@ class ClickHouseCluster: run_and_check(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name]) node.ip_address = self.get_instance_ip(node.name) node.client = Client(node.ip_address, command=self.client_bin_path) - start_deadline = time.time() + 60.0 # seconds + start_deadline = time.time() + 120.0 # seconds node.wait_for_start(start_deadline) return node @@ -916,13 +916,15 @@ class ClickHouseCluster: if self.with_hdfs and self.base_hdfs_cmd: logging.debug('Setup HDFS') os.makedirs(self.hdfs_logs_dir) + os.chmod(self.hdfs_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api() - self.wait_hdfs_to_start(hdfs_api, 120) + self.wait_hdfs_to_start(hdfs_api, 300) if self.with_kerberized_hdfs and self.base_kerberized_hdfs_cmd: logging.debug('Setup kerberized HDFS') os.makedirs(self.hdfs_kerberized_logs_dir) + os.chmod(self.hdfs_kerberized_logs_dir, stat.S_IRWXO) run_and_check(self.base_kerberized_hdfs_cmd + common_opts) hdfs_api = self.make_hdfs_api(kerberized=True) self.wait_hdfs_to_start(hdfs_api, timeout=300) @@ -964,7 +966,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 60.0 # seconds + start_deadline = time.time() + 120.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -998,17 +1000,16 @@ class ClickHouseCluster: sanitizer_assert_instance = line.split('|')[0].strip() break - for instance in list(self.instances.values()): + for name, instance in self.instances.items(): try: if not instance.is_up: continue if instance.contains_in_log(SANITIZER_SIGN): sanitizer_assert_instance = instance.grep_in_log(SANITIZER_SIGN) - logging.ERROR(f"Sanitizer in instance {instance.name} log {sanitizer_assert_instance}") + logging.ERROR(f"Sanitizer in instance {name} log {sanitizer_assert_instance}") if instance.contains_in_log("Fatal"): fatal_log = instance.grep_in_log("Fatal") - name = instance.name logging.ERROR(f"Crash in instance {name} fatal log {fatal_log}") except Exception as e: logging.error(f"Failed to check fails in logs: {e}") @@ -1324,7 +1325,7 @@ class ClickHouseInstance: from helpers.test_tools import assert_eq_with_retry assert_eq_with_retry(self, "select 1", "1", retry_count=int(stop_wait_sec / 0.5), sleep_time=0.5) - def restart_clickhouse(self, stop_start_wait_sec=5, kill=False): + def restart_clickhouse(self, stop_start_wait_sec=15, kill=False): self.stop_clickhouse(stop_start_wait_sec, kill) self.start_clickhouse(stop_start_wait_sec) diff --git a/tests/integration/helpers/hdfs_api.py b/tests/integration/helpers/hdfs_api.py index 8a69f6baffd..5b1b4c402da 100644 --- a/tests/integration/helpers/hdfs_api.py +++ b/tests/integration/helpers/hdfs_api.py @@ -47,12 +47,12 @@ class HDFSApi(object): # logging.basicConfig(level=logging.DEBUG) # logging.getLogger().setLevel(logging.DEBUG) - requests_log = logging.getLogger("requests.packages.urllib3") - requests_log.setLevel(logging.DEBUG) - requests_log.propagate = True - kerb_log = logging.getLogger("requests_kerberos") - kerb_log.setLevel(logging.DEBUG) - kerb_log.propagate = True + # requests_log = logging.getLogger("requests.packages.urllib3") + # requests_log.setLevel(logging.INFO) + # requests_log.propagate = True + # kerb_log = logging.getLogger("requests_kerberos") + # kerb_log.setLevel(logging.DEBUG) + # kerb_log.propagate = True if kerberized: self._run_kinit() diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index 7d4a3ad1c29..47086d45b3d 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -6,7 +6,7 @@ junit_duration_report = call junit_suite_name = integration log_cli = 1 log_cli_level = CRITICAL -log_cli_format = %(message)s +log_cli_format = %%(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) log_file = pytest.log log_file_level = DEBUG log_file_format = %(asctime)s [%(levelname)8s] %(message)s (%(filename)s:%(lineno)s) diff --git a/tests/integration/runner b/tests/integration/runner index f3309f983a3..3aa3aa5ef64 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -199,7 +199,7 @@ if __name__ == "__main__": cmd = "docker run {net} {tty} --rm --name {name} --privileged --volume={bridge_bin}:/clickhouse-odbc-bridge --volume={bin}:/clickhouse \ --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ - --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=180 \ + --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=300 \ {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( net=net, tty=tty, diff --git a/tests/integration/test_MemoryTracking/test.py b/tests/integration/test_MemoryTracking/test.py index a0ad8dc519d..1ee528eaa8d 100644 --- a/tests/integration/test_MemoryTracking/test.py +++ b/tests/integration/test_MemoryTracking/test.py @@ -13,9 +13,6 @@ node = cluster.add_instance('node', main_configs=[ 'configs/asynchronous_metrics_update_period_s.xml', ]) -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope='module', autouse=True) def start_cluster(): try: diff --git a/tests/integration/test_access_control_on_cluster/test.py b/tests/integration/test_access_control_on_cluster/test.py index bc740402161..6bcf67779ef 100644 --- a/tests/integration/test_access_control_on_cluster/test.py +++ b/tests/integration/test_access_control_on_cluster/test.py @@ -18,7 +18,7 @@ def started_cluster(): def test_access_control_on_cluster(): - ch1.query_with_retry("CREATE USER Alex ON CLUSTER 'cluster'", retry_count=3) + ch1.query_with_retry("CREATE USER IF NOT EXISTS Alex ON CLUSTER 'cluster'", retry_count=5) assert ch1.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n" assert ch2.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n" assert ch3.query("SHOW CREATE USER Alex") == "CREATE USER Alex\n" diff --git a/tests/integration/test_adaptive_granularity/test.py b/tests/integration/test_adaptive_granularity/test.py index 47209908e5c..2ea2f29212f 100644 --- a/tests/integration/test_adaptive_granularity/test.py +++ b/tests/integration/test_adaptive_granularity/test.py @@ -1,5 +1,6 @@ import time +import logging import pytest from helpers.client import QueryRuntimeException, QueryTimeoutExceedException from helpers.cluster import ClickHouseCluster @@ -268,16 +269,18 @@ def test_version_single_node_update(start_dynamic_cluster, n, tables): ] ) def test_mixed_granularity_single_node(start_dynamic_cluster, node): - assert node.name == "node9" or node.name == "node10" - assert_eq_with_retry(node, - "SELECT value FROM system.merge_tree_settings WHERE name='enable_mixed_granularity_parts'", - '0') # check that enable_mixed_granularity_parts is off by default - node.query( "INSERT INTO table_with_default_granularity VALUES (toDate('2018-10-01'), 1, 333), (toDate('2018-10-02'), 2, 444)") node.query( "INSERT INTO table_with_default_granularity VALUES (toDate('2018-09-01'), 1, 333), (toDate('2018-09-02'), 2, 444)") + path_to_part = node.query( + "SELECT path FROM system.parts WHERE table = 'table_with_default_granularity' AND active=1 ORDER BY partition DESC LIMIT 1").strip() + + result = node.exec_in_container(["bash", "-c", "find {p} -name '*.mrk*'".format( + p=path_to_part)]) # check that we have non adaptive files + logging.info(f"path {path_to_part} result\n {result}") + def callback(n): new_config = """ diff --git a/tests/integration/test_adaptive_granularity_different_settings/test.py b/tests/integration/test_adaptive_granularity_different_settings/test.py index ce8f32b6ec6..f78a0ef2da6 100644 --- a/tests/integration/test_adaptive_granularity_different_settings/test.py +++ b/tests/integration/test_adaptive_granularity_different_settings/test.py @@ -37,8 +37,8 @@ def test_attach_detach(start_cluster): node2.query("INSERT INTO test VALUES (3), (4)") - node1.query("SYSTEM SYNC REPLICA test", timeout=10) - node2.query("SYSTEM SYNC REPLICA test", timeout=10) + node1.query_with_retry("SYSTEM SYNC REPLICA test", timeout=10) + node2.query_with_retry("SYSTEM SYNC REPLICA test", timeout=10) assert node1.query("SELECT COUNT() FROM test") == "4\n" assert node2.query("SELECT COUNT() FROM test") == "4\n" diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 5e516176c0b..852554f009d 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -78,7 +78,7 @@ def test_alter_replicated_on_cluster(started_cluster): assert node3.query("SELECT date FROM test_table_replicated") == '2019-10-01 00:00:00\n' assert node4.query("SELECT date FROM test_table_replicated") == '2019-10-01 00:00:00\n' - node3.query("ALTER TABLE test_table_replicated ON CLUSTER 'test_cluster_mixed' MODIFY COLUMN value String", settings={"replication_alter_partitions_sync": "2"}) + node3.query_with_retry("ALTER TABLE test_table_replicated ON CLUSTER 'test_cluster_mixed' MODIFY COLUMN value String", settings={"replication_alter_partitions_sync": "2"}) for node in [node2, node4]: node.query("INSERT INTO test_table_replicated VALUES(toDateTime('2019-10-02 00:00:00'), 2, 'Hello')") diff --git a/tests/integration/test_check_table/test.py b/tests/integration/test_check_table/test.py index d204f6c5810..b184813d24f 100644 --- a/tests/integration/test_check_table/test.py +++ b/tests/integration/test_check_table/test.py @@ -122,9 +122,9 @@ def test_check_replicated_table_simple(started_cluster): def test_check_replicated_table_corruption(started_cluster): for node in [node1, node2]: - node.query("DROP TABLE IF EXISTS replicated_mt_1") + node.query_with_retry("DROP TABLE IF EXISTS replicated_mt_1") - node.query(''' + node.query_with_retry(''' CREATE TABLE replicated_mt_1(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt_1', '{replica}') PARTITION BY toYYYYMM(date) ORDER BY id; '''.format(replica=node.name)) @@ -136,7 +136,7 @@ def test_check_replicated_table_corruption(started_cluster): assert node1.query("SELECT count() from replicated_mt_1") == "4\n" assert node2.query("SELECT count() from replicated_mt_1") == "4\n" - part_name = node1.query( + part_name = node1.query_with_retry( "SELECT name from system.parts where table = 'replicated_mt_1' and partition_id = '201901' and active = 1").strip() corrupt_data_part_on_disk(node1, "replicated_mt_1", part_name) @@ -144,7 +144,7 @@ def test_check_replicated_table_corruption(started_cluster): "check_query_single_value_result": 0}) == "{p}\t0\tPart {p} looks broken. Removing it and will try to fetch.\n".format( p=part_name) - node1.query("SYSTEM SYNC REPLICA replicated_mt_1") + node1.query_with_retry("SYSTEM SYNC REPLICA replicated_mt_1") assert node1.query("CHECK TABLE replicated_mt_1 PARTITION 201901", settings={"check_query_single_value_result": 0}) == "{}\t1\t\n".format(part_name) assert node1.query("SELECT count() from replicated_mt_1") == "4\n" diff --git a/tests/integration/test_compression_codec_read/test.py b/tests/integration/test_compression_codec_read/test.py index 0eb1f5aa867..35ae60f05ea 100644 --- a/tests/integration/test_compression_codec_read/test.py +++ b/tests/integration/test_compression_codec_read/test.py @@ -17,6 +17,8 @@ def start_cluster(): cluster.shutdown() def test_default_codec_read(start_cluster): + node1.query("DROP TABLE IF EXISTS test_18340") + node1.query(""" CREATE TABLE test_18340 ( diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 59ccc59d0af..3aa76b4af88 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -66,7 +66,7 @@ def started_cluster(): (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) + mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) execute_mysql_query(mysql_conn, "DROP DATABASE IF EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE DATABASE clickhouse") execute_mysql_query(mysql_conn, @@ -245,7 +245,7 @@ def test_file_dictionary_restrictions(started_cluster): def test_dictionary_with_where(started_cluster): - mysql_conn = create_mysql_conn("root", "clickhouse", "localhost", started_cluster.mysql_port) + mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) execute_mysql_query(mysql_conn, "CREATE DATABASE IF NOT EXISTS clickhouse") execute_mysql_query(mysql_conn, "CREATE TABLE clickhouse.special_table (key_field1 int, value1 text, PRIMARY KEY (key_field1))") diff --git a/tests/integration/test_dictionaries_mysql/test.py b/tests/integration/test_dictionaries_mysql/test.py index 90cfe53dd68..c18f3d6cf70 100644 --- a/tests/integration/test_dictionaries_mysql/test.py +++ b/tests/integration/test_dictionaries_mysql/test.py @@ -31,7 +31,7 @@ def started_cluster(): cluster.start() # Create a MySQL database - mysql_connection = get_mysql_conn() + mysql_connection = get_mysql_conn(cluster) create_mysql_db(mysql_connection, 'test') mysql_connection.close() @@ -54,7 +54,7 @@ def test_load_mysql_dictionaries(started_cluster): for n in range(0, 5): # Create MySQL tables, fill them and create CH dict tables - prepare_mysql_table('test', str(n)) + prepare_mysql_table(started_cluster, 'test', str(n)) # Check dictionaries are loaded and have correct number of elements for n in range(0, 100): @@ -72,8 +72,8 @@ def create_mysql_db(mysql_connection, name): cursor.execute("CREATE DATABASE {} DEFAULT CHARACTER SET 'utf8'".format(name)) -def prepare_mysql_table(table_name, index): - mysql_connection = get_mysql_conn() +def prepare_mysql_table(started_cluster, table_name, index): + mysql_connection = get_mysql_conn(started_cluster) # Create table create_mysql_table(mysql_connection, table_name + str(index)) @@ -89,16 +89,16 @@ def prepare_mysql_table(table_name, index): # Create CH Dictionary tables based on MySQL tables query(create_clickhouse_dictionary_table_template.format(table_name + str(index), 'dict' + str(index))) -def get_mysql_conn(): +def get_mysql_conn(started_cluster): errors = [] conn = None for _ in range(5): try: if conn is None: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=started_cluster.mysql_ip, port=started_cluster.mysql_port) else: conn.ping(reconnect=True) - logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + logging.debug(f"MySQL Connection establised: {started_cluster.mysql_ip}:{started_cluster.mysql_port}") return conn except Exception as e: errors += [str(e)] diff --git a/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore new file mode 100644 index 00000000000..d6b7ef32c84 --- /dev/null +++ b/tests/integration/test_dictionaries_redis/configs/dictionaries/.gitignore @@ -0,0 +1,2 @@ +* +!.gitignore diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py index 0c801ce3f12..46820fb4a89 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_reading.py @@ -7,7 +7,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="reading") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py index caabdf12c66..12cde30d8ed 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_default_string.py @@ -9,8 +9,7 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import TSV -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="string") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py index e794ffa5a37..789d4aeb8b3 100644 --- a/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py +++ b/tests/integration/test_dictionary_allow_read_expired_keys/test_dict_get_or_default.py @@ -7,7 +7,7 @@ from helpers.cluster import ClickHouseCluster from helpers.cluster import ClickHouseKiller from helpers.network import PartitionManager -cluster = ClickHouseCluster(__file__) +cluster = ClickHouseCluster(__file__, name="default") dictionary_node = cluster.add_instance('dictionary_node', stay_alive=True) main_node = cluster.add_instance('main_node', main_configs=['configs/enable_dictionaries.xml', diff --git a/tests/integration/test_disabled_mysql_server/test.py b/tests/integration/test_disabled_mysql_server/test.py index 2fc84ee74a5..d7977404c73 100644 --- a/tests/integration/test_disabled_mysql_server/test.py +++ b/tests/integration/test_disabled_mysql_server/test.py @@ -22,10 +22,10 @@ def started_cluster(): class MySQLNodeInstance: - def __init__(self, user='root', password='clickhouse', hostname='127.0.0.1', port=cluster.mysql_port): + def __init__(self, started_cluster, user='root', password='clickhouse'): self.user = user - self.port = port - self.hostname = hostname + self.port = cluster.mysql_port + self.hostname = cluster.mysql_ip self.password = password self.mysql_connection = None # lazy init @@ -45,7 +45,7 @@ class MySQLNodeInstance: def test_disabled_mysql_server(started_cluster): - with contextlib.closing(MySQLNodeInstance()) as mysql_node: + with contextlib.closing(MySQLNodeInstance(started_cluster)) as mysql_node: mysql_node.query("DROP DATABASE IF EXISTS test_db_disabled;") mysql_node.query("CREATE DATABASE test_db_disabled;") mysql_node.query("CREATE TABLE test_db_disabled.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") diff --git a/tests/integration/test_distributed_storage_configuration/test.py b/tests/integration/test_distributed_storage_configuration/test.py index 976fc5211a7..94beb7b57ca 100644 --- a/tests/integration/test_distributed_storage_configuration/test.py +++ b/tests/integration/test_distributed_storage_configuration/test.py @@ -17,7 +17,7 @@ node = cluster.add_instance('node', def start_cluster(): try: cluster.start() - node.query('CREATE DATABASE test ENGINE=Ordinary') # Different paths with Atomic + node.query('CREATE DATABASE IF NOT EXISTS test ENGINE=Ordinary') # Different paths with Atomic yield cluster finally: cluster.shutdown() diff --git a/tests/integration/test_format_avro_confluent/test.py b/tests/integration/test_format_avro_confluent/test.py index 49bc7baeb39..23e2d8d8c47 100644 --- a/tests/integration/test_format_avro_confluent/test.py +++ b/tests/integration/test_format_avro_confluent/test.py @@ -7,10 +7,6 @@ from confluent_kafka.avro.cached_schema_registry_client import CachedSchemaRegis from confluent_kafka.avro.serializer.message_serializer import MessageSerializer from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - @pytest.fixture(scope="module") def started_cluster(): try: diff --git a/tests/integration/test_insert_into_distributed/test.py b/tests/integration/test_insert_into_distributed/test.py index d71d1075c70..704620634cb 100644 --- a/tests/integration/test_insert_into_distributed/test.py +++ b/tests/integration/test_insert_into_distributed/test.py @@ -100,7 +100,7 @@ def test_reconnect(started_cluster): pm.heal_all() time.sleep(1) instance.query("INSERT INTO distributed VALUES (3)") - time.sleep(1) + time.sleep(5) assert remote.query("SELECT count(*) FROM local1").strip() == '3' diff --git a/tests/integration/test_join_set_family_s3/test.py b/tests/integration/test_join_set_family_s3/test.py index 2a20c8eade9..625cac48dee 100644 --- a/tests/integration/test_join_set_family_s3/test.py +++ b/tests/integration/test_join_set_family_s3/test.py @@ -4,9 +4,6 @@ import sys import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_log_family_s3/test.py b/tests/integration/test_log_family_s3/test.py index c23e7545b27..115ec47871b 100644 --- a/tests/integration/test_log_family_s3/test.py +++ b/tests/integration/test_log_family_s3/test.py @@ -4,9 +4,6 @@ import sys import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 45b3c3c65f0..0779a57ccb6 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -6,9 +6,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_merge_tree_s3_failover/test.py b/tests/integration/test_merge_tree_s3_failover/test.py index 11a1e464cb6..7d9496f41d7 100644 --- a/tests/integration/test_merge_tree_s3_failover/test.py +++ b/tests/integration/test_merge_tree_s3_failover/test.py @@ -7,9 +7,6 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - # Runs custom python-based S3 endpoint. def run_endpoint(cluster): diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 346d9aced3f..934fa1e5ecd 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -6,9 +6,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_merge_tree_s3_with_cache/test.py b/tests/integration/test_merge_tree_s3_with_cache/test.py index d5d6db2fb77..7022f90a2b9 100644 --- a/tests/integration/test_merge_tree_s3_with_cache/test.py +++ b/tests/integration/test_merge_tree_s3_with_cache/test.py @@ -3,9 +3,6 @@ import logging import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 1404cf409f3..60cf8a9239b 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=os.environ.get("DOCKER_API_VERSION"), timeout=180).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') diff --git a/tests/integration/test_profile_events_s3/test.py b/tests/integration/test_profile_events_s3/test.py index 3d65a489610..c70c29d1bf7 100644 --- a/tests/integration/test_profile_events_s3/test.py +++ b/tests/integration/test_profile_events_s3/test.py @@ -5,9 +5,6 @@ import pytest import requests from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_quota/test.py b/tests/integration/test_quota/test.py index 353d776c0f3..941c728a4de 100644 --- a/tests/integration/test_quota/test.py +++ b/tests/integration/test_quota/test.py @@ -61,7 +61,7 @@ def copy_quota_xml(local_file_name, reload_immediately=True): def started_cluster(): try: cluster.start() - + instance.query("DROP TABLE IF EXISTS test_table") instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()") instance.query("INSERT INTO test_table SELECT number FROM numbers(50)") diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index 99e7d6077f8..9b169b87fe2 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -211,21 +211,21 @@ def test_recover_staled_replica(started_cluster): with PartitionManager() as pm: pm.drop_instance_zk_connections(dummy_node) dummy_node.query_and_get_error("RENAME TABLE recover.t1 TO recover.m1") - main_node.query("RENAME TABLE recover.t1 TO recover.m1", settings=settings) - main_node.query("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings) - main_node.query("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings) - main_node.query("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings) - main_node.query("DROP TABLE recover.rmt5", settings=settings) - main_node.query("DROP DICTIONARY recover.d2", settings=settings) - main_node.query("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", settings=settings) + main_node.query_with_retry("RENAME TABLE recover.t1 TO recover.m1", settings=settings) + main_node.query_with_retry("ALTER TABLE recover.mt1 ADD COLUMN m int", settings=settings) + main_node.query_with_retry("ALTER TABLE recover.rmt1 ADD COLUMN m int", settings=settings) + main_node.query_with_retry("RENAME TABLE recover.rmt3 TO recover.rmt4", settings=settings) + main_node.query_with_retry("DROP TABLE recover.rmt5", settings=settings) + main_node.query_with_retry("DROP DICTIONARY recover.d2", settings=settings) + main_node.query_with_retry("CREATE DICTIONARY recover.d2 (n int DEFAULT 0, m int DEFAULT 1) PRIMARY KEY n SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'rmt1' PASSWORD '' DB 'recover')) LIFETIME(MIN 1 MAX 10) LAYOUT(FLAT());", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) - main_node.query("DROP TABLE recover.tmp", settings=settings) - main_node.query("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) + main_node.query_with_retry("DROP TABLE recover.tmp", settings=settings) + main_node.query_with_retry("CREATE TABLE recover.tmp AS recover.m1", settings=settings) assert main_node.query("SELECT name FROM system.tables WHERE database='recover' ORDER BY name") == "d1\nd2\nm1\nmt1\nmt2\nrmt1\nrmt2\nrmt4\nt2\ntmp\n" query = "SELECT name, uuid, create_table_query FROM system.tables WHERE database='recover' ORDER BY name" diff --git a/tests/integration/test_replicated_merge_tree_s3/test.py b/tests/integration/test_replicated_merge_tree_s3/test.py index 3b3540ef1b8..d04bdae36e2 100644 --- a/tests/integration/test_replicated_merge_tree_s3/test.py +++ b/tests/integration/test_replicated_merge_tree_s3/test.py @@ -5,9 +5,6 @@ import string import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - @pytest.fixture(scope="module") def cluster(): diff --git a/tests/integration/test_s3_with_https/test.py b/tests/integration/test_s3_with_https/test.py index 1a5c6e2ce12..6c7b47ea0b1 100644 --- a/tests/integration/test_s3_with_https/test.py +++ b/tests/integration/test_s3_with_https/test.py @@ -3,9 +3,6 @@ import logging import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - def check_proxy_logs(cluster, proxy_instance): logs = cluster.get_container_logs(proxy_instance) diff --git a/tests/integration/test_s3_with_proxy/test.py b/tests/integration/test_s3_with_proxy/test.py index 586895fffe5..7a1a2292eef 100644 --- a/tests/integration/test_s3_with_proxy/test.py +++ b/tests/integration/test_s3_with_proxy/test.py @@ -5,10 +5,6 @@ import time import pytest from helpers.cluster import ClickHouseCluster -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - - # Runs simple proxy resolver in python env container. def run_resolver(cluster): container_id = cluster.get_container_id('resolver') diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index a285b263688..aea6e0816d0 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -169,7 +169,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_host, port=cluster.mysql_port) return conn diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 13b10e0ca68..0bb7047d875 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -21,6 +21,7 @@ def get_postgres_conn(database=False): def create_postgres_db(conn, name): cursor = conn.cursor() + cursor.execute("DROP DATABASE IF EXISTS {}".format(name)) cursor.execute("CREATE DATABASE {}".format(name)) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5a81196a644..56cc7cf6e81 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -11,9 +11,6 @@ import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - MINIO_INTERNAL_PORT = 9001 # Creates S3 bucket for tests and allows anonymous read-write access to it. diff --git a/tests/integration/test_storage_s3/test_redirect.py b/tests/integration/test_storage_s3/test_redirect.py index ab677f46d9a..10a20184722 100644 --- a/tests/integration/test_storage_s3/test_redirect.py +++ b/tests/integration/test_storage_s3/test_redirect.py @@ -11,9 +11,6 @@ import helpers.client import pytest from helpers.cluster import ClickHouseCluster, ClickHouseInstance -logging.getLogger().setLevel(logging.INFO) -logging.getLogger().addHandler(logging.StreamHandler()) - # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(cluster): # Allows read-write access for bucket without authorization. From e8a435d2fb41b5bf6dddbefdeb2e898fdd26dd12 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 15 Mar 2021 18:31:50 +0900 Subject: [PATCH 0064/1060] Add base condition --- .../parametric-functions.md | 174 +++++++++----- .../AggregateFunctionSequenceNextNode.cpp | 46 ++-- .../AggregateFunctionSequenceNextNode.h | 44 ++-- .../01656_sequence_next_node.reference | 30 +++ .../0_stateless/01656_sequence_next_node.sql | 221 ++++++++++-------- 5 files changed, 331 insertions(+), 184 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index dcc63ebd960..3bff7fa795c 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -513,7 +513,7 @@ Returns a value of next event that matched an event chain. **Syntax** ``` sql -sequenceNextNode(direction, base)(timestamp, event_column, event1, event2, event3, ...) +sequenceNextNode(direction, base)(timestamp, base_condition, event_column, event1, event2, event3, ...) ``` **Parameters** @@ -529,11 +529,11 @@ sequenceNextNode(direction, base)(timestamp, event_column, event1, event2, event **Arguments** - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. -- `event_column` - Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `base_condition` — Condition that the base point must fulfill. +- `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` - `cond` — Conditions describing the chain of events. `UInt8` **Returned value** - - `event_column[next_index]` - if the pattern is matched and next value exists. - `NULL` - if the pattern isn’t matched or next value doesn't exist. @@ -556,7 +556,7 @@ ORDER BY id; INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); -SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'A', page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; ``` Result: @@ -578,123 +578,181 @@ INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, ``` ```SQL -SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'Home', page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Base point, Matched with Home 1970-01-01 09:00:02 1 Gift // Matched with Gift 1970-01-01 09:00:03 1 Exit // The result - 1970-01-01 09:00:01 3 Home // Base point, Matched with Home - 1970-01-01 09:00:02 3 Home // Unmatched with Gift - 1970-01-01 09:00:03 3 Gift - 1970-01-01 09:00:04 3 Basket + 1970-01-01 09:00:01 2 Home // Base point, Matched with Home + 1970-01-01 09:00:02 2 Home // Unmatched with Gift + 1970-01-01 09:00:03 2 Gift + 1970-01-01 09:00:04 2 Basket - 1970-01-01 09:00:01 4 Gift // Base point, Unmatched with Home - 1970-01-01 09:00:02 4 Home - 1970-01-01 09:00:03 4 Gift - 1970-01-01 09:00:04 4 Basket + 1970-01-01 09:00:01 3 Gift // Base point, Unmatched with Home + 1970-01-01 09:00:02 3 Home + 1970-01-01 09:00:03 3 Gift + 1970-01-01 09:00:04 3 Basket ``` **Behavior for `backward` and `tail`** ```SQL -SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'tail')(dt, page = 'Basket', page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home 1970-01-01 09:00:02 1 Gift 1970-01-01 09:00:03 1 Exit // Base point, Unmatched with Basket -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home // The result -1970-01-01 09:00:03 3 Gift // Matched with Gift -1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home // The result +1970-01-01 09:00:03 2 Gift // Matched with Gift +1970-01-01 09:00:04 2 Basket // Base point, Matched with Basket -1970-01-01 09:00:01 4 Gift -1970-01-01 09:00:02 4 Home // The result -1970-01-01 09:00:03 4 Gift // Base point, Matched with Gift -1970-01-01 09:00:04 4 Basket // Base point, Matched with Basket +1970-01-01 09:00:01 3 Gift +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point, Matched with Gift +1970-01-01 09:00:04 3 Basket // Base point, Matched with Basket ``` **Behavior for `forward` and `first_match`** ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit // The result -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket The result +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket The result -1970-01-01 09:00:01 4 Gift // Base point -1970-01-01 09:00:02 4 Home // Thre result -1970-01-01 09:00:03 4 Gift -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift // Base point +1970-01-01 09:00:02 3 Home // Thre result +1970-01-01 09:00:03 3 Gift +1970-01-01 09:00:04 3 Basket ``` ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit // Unmatched with Home -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket // Unmatched with Home +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket // Unmatched with Home -1970-01-01 09:00:01 4 Gift // Base point -1970-01-01 09:00:02 4 Home // Matched with Home -1970-01-01 09:00:03 4 Gift // The result -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift // Base point +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // The result +1970-01-01 09:00:04 3 Basket ``` **Behavior for `backward` and `last_match`** ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // The result 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit -1970-01-01 09:00:01 3 Home -1970-01-01 09:00:02 3 Home // The result -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket +1970-01-01 09:00:01 2 Home +1970-01-01 09:00:02 2 Home // The result +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket -1970-01-01 09:00:01 4 Gift -1970-01-01 09:00:02 4 Home // The result -1970-01-01 09:00:03 4 Gift // Base point -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift +1970-01-01 09:00:02 3 Home // The result +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket ``` ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Matched with Home, the result is null 1970-01-01 09:00:02 1 Gift // Base point 1970-01-01 09:00:03 1 Exit -1970-01-01 09:00:01 3 Home // The result -1970-01-01 09:00:02 3 Home // Matched with Home -1970-01-01 09:00:03 3 Gift // Base point -1970-01-01 09:00:04 3 Basket +1970-01-01 09:00:01 2 Home // The result +1970-01-01 09:00:02 2 Home // Matched with Home +1970-01-01 09:00:03 2 Gift // Base point +1970-01-01 09:00:04 2 Basket -1970-01-01 09:00:01 4 Gift // The result -1970-01-01 09:00:02 4 Home // Matched with Home -1970-01-01 09:00:03 4 Gift // Base point -1970-01-01 09:00:04 4 Basket +1970-01-01 09:00:01 3 Gift // The result +1970-01-01 09:00:02 3 Home // Matched with Home +1970-01-01 09:00:03 3 Gift // Base point +1970-01-01 09:00:04 3 Basket +``` + + +**Behavior for `base_condition`** + +```SQL +CREATE TABLE test_flow_basecond +( + `dt` DateTime, + `id` int, + `page` String, + `ref` String +) +ENGINE = MergeTree +PARTITION BY toYYYYMMDD(dt) +ORDER BY id + +INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3, 1, 'B', 'ref2') (4, 1, 'B', 'ref1'); +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'head')(dt, ref = 'ref1', page, page = 'A') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'. + 1970-01-01 09:00:02 1 A ref3 + 1970-01-01 09:00:03 1 B ref2 + 1970-01-01 09:00:04 1 B ref1 + ``` + +```SQL +SELECT id, sequenceNextNode('backward', 'tail')(dt, ref = 'ref4', page, page = 'B') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 + 1970-01-01 09:00:02 1 A ref3 + 1970-01-01 09:00:03 1 B ref2 + 1970-01-01 09:00:04 1 B ref1 // The tail can't be base point becasue the ref column of the tail unmatched with 'ref4'. +``` + +```SQL +SELECT id, sequenceNextNode('forward', 'first_match')(dt, ref = 'ref3', page, page = 'A') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'. + 1970-01-01 09:00:02 1 A ref3 // Base point + 1970-01-01 09:00:03 1 B ref2 // The result + 1970-01-01 09:00:04 1 B ref1 +``` + +```SQL +SELECT id, sequenceNextNode('backward', 'last_match')(dt, ref = 'ref2', page, page = 'B') FROM test_flow_basecond GROUP BY id; + + dt id page ref + 1970-01-01 09:00:01 1 A ref4 + 1970-01-01 09:00:02 1 A ref3 // The result + 1970-01-01 09:00:03 1 B ref2 // Base point + 1970-01-01 09:00:04 1 B ref1 // This row can't be base point becasue the ref column unmatched with 'ref2'. ``` diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index b8f517cba20..c31337e25c6 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -12,6 +12,7 @@ namespace DB { constexpr size_t MAX_EVENTS_SIZE = 64; +constexpr size_t MIN_REQUIRED_ARGS = 3; namespace ErrorCodes { @@ -27,13 +28,13 @@ template inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl2(const DataTypePtr data_type, const DataTypes & argument_types, SeqBase base) { if (base == HEAD) - return std::make_shared, Direction, HEAD>>(data_type, argument_types); + return std::make_shared, Direction, HEAD, MIN_REQUIRED_ARGS>>(data_type, argument_types); else if (base == TAIL) - return std::make_shared, Direction, TAIL>>(data_type, argument_types); + return std::make_shared, Direction, TAIL, MIN_REQUIRED_ARGS>>(data_type, argument_types); else if (base == FIRST_MATCH) - return std::make_shared, Direction, FIRST_MATCH>>(data_type, argument_types); + return std::make_shared, Direction, FIRST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); else - return std::make_shared, Direction, LAST_MATCH>>(data_type, argument_types); + return std::make_shared, Direction, LAST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); } template @@ -46,9 +47,9 @@ inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl1(const DataT } AggregateFunctionPtr -createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, const DataTypes & argument_types, const Array & parameters) +createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, const DataTypes & argument_types, const Array & parameters) { - assert(max_args <= MAX_EVENTS_SIZE); + assert(max_events <= MAX_EVENTS_SIZE); if (parameters.size() < 2) throw Exception("Aggregate function " + name + " requires 2 parameters (direction, head)", @@ -76,19 +77,26 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c else throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; - if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < 3) - throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() < 2) + if (argument_types.size() < MIN_REQUIRED_ARGS) throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > max_args + 2) + else if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < MIN_REQUIRED_ARGS + 1) + throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + else if (argument_types.size() > max_events + MIN_REQUIRED_ARGS) throw Exception("Aggregate function " + name + " requires at most " + - std::to_string(max_args + 2) + - " (timestamp, value_column, " + std::to_string(max_args) + " events) arguments.", + std::to_string(max_events + MIN_REQUIRED_ARGS) + + " (timestamp, value_column, " + std::to_string(max_events) + " events) arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (const auto i : ext::range(2, argument_types.size())) + if (const auto * cond_arg = argument_types[1].get()) + { + if (!isUInt8(cond_arg)) + throw Exception{"Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " + + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + + for (const auto i : ext::range(MIN_REQUIRED_ARGS, argument_types.size())) { const auto * cond_arg = argument_types[i].get(); if (!isUInt8(cond_arg)) @@ -96,12 +104,12 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) + if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String) throw Exception{"Illegal type " + argument_types[1].get()->getName() + " of second argument of aggregate function " + name + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - DataTypePtr data_type = makeNullable(argument_types[1]); + DataTypePtr data_type = makeNullable(argument_types[2]); WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) @@ -122,11 +130,11 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_args, c ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } -auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_args) +auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) { - return [max_args](const std::string & name, const DataTypes & argument_types, const Array & parameters) + return [max_events](const std::string & name, const DataTypes & argument_types, const Array & parameters) { - return createAggregateFunctionSequenceNode(name, max_args, argument_types, parameters); + return createAggregateFunctionSequenceNode(name, max_events, argument_types, parameters); }; } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index f58469f27bb..a37816cfa4a 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -49,6 +49,7 @@ struct NodeBase DataTypeDateTime::FieldType event_time; std::bitset events_bitset; + bool can_be_base; char * data() { return reinterpret_cast(this) + sizeof(Node); } @@ -68,6 +69,7 @@ struct NodeBase writeBinary(event_time, buf); UInt64 ulong_bitset = events_bitset.to_ulong(); writeBinary(ulong_bitset, buf); + writeBinary(can_be_base, buf); } static Node * read(ReadBuffer & buf, Arena * arena) @@ -83,6 +85,7 @@ struct NodeBase UInt64 ulong_bitset; readBinary(ulong_bitset, buf); node->events_bitset = ulong_bitset; + readBinary(node->can_be_base, buf); return node; } @@ -146,13 +149,15 @@ struct SequenceNextNodeGeneralData }; /// Implementation of sequenceFirstNode -template +template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } + static constexpr size_t EventColumn = 2; + static constexpr size_t BaseCondition = 1; DataTypePtr & data_type; UInt8 events_size; @@ -160,10 +165,10 @@ class SequenceNextNodeImpl final public: SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( + : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( {data_type_}, {}) , data_type(this->argument_types[0]) - , events_size(arguments.size() - 2) + , events_size(arguments.size() - MinRequiredArgs) , max_elems(max_elems_) { } @@ -196,7 +201,7 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - Node * node = Node::allocate(*columns[1], row_num, arena); + Node * node = Node::allocate(*columns[EventColumn], row_num, arena); const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; @@ -208,10 +213,12 @@ public: /// + 4 (bit of event3) node->events_bitset.reset(); for (UInt8 i = 0; i < events_size; ++i) - if (assert_cast *>(columns[2 + i])->getData()[row_num]) + if (assert_cast *>(columns[MinRequiredArgs + i])->getData()[row_num]) node->events_bitset.set(i); node->event_time = timestamp; + node->can_be_base = assert_cast *>(columns[BaseCondition])->getData()[row_num]; + data(place).value.push_back(node, arena); } @@ -306,19 +313,31 @@ public: inline UInt32 getBaseIndex(Data & data, bool & exist) const { + exist = false; + if (data.value.size() == 0) + return 0; + switch (Base) { case HEAD: - exist = true; - return 0; + if (data.value[0]->can_be_base) + { + exist = true; + return 0; + } + break; case TAIL: - exist = true; - return data.value.size() - 1; + if (data.value[data.value.size() - 1]->can_be_base) + { + exist = true; + return data.value.size() - 1; + } + break; case FIRST_MATCH: for (UInt64 i = 0; i < data.value.size(); ++i) - if (data.value[i]->events_bitset.test(0)) + if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base) { exist = true; return i; @@ -329,7 +348,7 @@ public: for (UInt64 i = 0; i < data.value.size(); ++i) { auto reversed_i = data.value.size() - i - 1; - if (data.value[reversed_i]->events_bitset.test(0)) + if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base) { exist = true; return reversed_i; @@ -338,7 +357,6 @@ public: break; } - exist = false; return 0; } diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node.reference index 9309641f3bf..739f19818a6 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.reference +++ b/tests/queries/0_stateless/01656_sequence_next_node.reference @@ -459,3 +459,33 @@ (max_args) 10 \N (max_args) 11 \N (forward, head, A) id = 12 A +(forward, head, 1) 1 A +(forward, head, 1) 2 \N +(forward, head, 1) 3 \N +(forward, head, 1, A) 1 B +(forward, head, 1, A) 2 \N +(forward, head, 1, A) 3 \N +(forward, head, 1, A->B) 1 C +(forward, head, 1, A->B) 2 \N +(forward, head, 1, A->B) 3 \N +(backward, tail, 1) 1 \N +(backward, tail, 1) 2 A +(backward, tail, 1) 3 \N +(backward, tail, 1, A) 1 \N +(backward, tail, 1, A) 2 B +(backward, tail, 1, A) 3 \N +(backward, tail, 1, A->B) 1 \N +(backward, tail, 1, A->B) 2 C +(backward, tail, 1, A->B) 3 \N +(forward, first_match, 1, B) 1 C +(forward, first_match, 1, B) 2 A +(forward, first_match, 1, B) 3 D +(forward, first_match, 1, B->C) 1 D +(forward, first_match, 1, B->C) 2 \N +(forward, first_match, 1, B->C) 3 \N +(backward, first_match, 1, B) 1 A +(backward, first_match, 1, B) 2 C +(backward, first_match, 1, B) 3 B +(backward, first_match, 1, B->C) 1 \N +(backward, first_match, 1, B->C) 2 \N +(backward, first_match, 1, B->C) 3 \N diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node.sql index 63810265bf9..c03abed8a54 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node.sql @@ -27,26 +27,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); @@ -54,41 +54,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; @@ -123,73 +123,106 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); -SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; DROP TABLE IF EXISTS test_sequenceNextNode; + +DROP TABLE IF EXISTS test_base_condition; + +CREATE TABLE IF NOT EXISTS test_base_condition (dt DateTime, id int, action String, referrer String) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; + +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',1,'A','1'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',1,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',1,'C','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',1,'D','4'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',2,'D','4'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',2,'C','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',2,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',2,'A','1'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:01',3,'B','10'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3'); +INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4'); + +SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; + +DROP TABLE IF EXISTS test_base_condition; From a4e9cf26b0c5f95621148e7d60f3c8ea268324c4 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 19 Mar 2021 17:43:20 +0900 Subject: [PATCH 0065/1060] Fix reverse indexing of TAIL in serialize() --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index a37816cfa4a..c0ecdadf8b1 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -281,8 +281,8 @@ public: case TAIL: writeVarUInt(size, buf); - for (size_t i = value.size() - 1; i >= size; --i) - value[i]->write(buf); + for (size_t i = 0; i < size; ++i) + value[value.size() - size + i]->write(buf); break; case FIRST_MATCH: From a67f066f0a9bb25bbdd63f4a455a99debe6c5e81 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:44:03 +0300 Subject: [PATCH 0066/1060] fixes --- tests/integration/helpers/cluster.py | 3 +- .../materialize_with_ddl.py | 376 +++++++++--------- .../test_materialize_mysql_database/test.py | 54 ++- 3 files changed, 213 insertions(+), 220 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 124f77061cb..fbf5e425426 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -656,10 +656,11 @@ class ClickHouseCluster: raise Exception("Cannot wait MySQL container") def wait_mysql8_to_start(self, timeout=60): + self.mysql8_ip = self.get_instance_ip('mysql80') start = time.time() while time.time() - start < timeout: try: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=self.mysql8_port) + conn = pymysql.connect(user='root', password='clickhouse', host=self.mysql8_ip, port=self.mysql8_port) conn.close() logging.debug("Mysql 8 Started") return diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 7e711d62576..c2f46a4b986 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -3,7 +3,7 @@ import time import pymysql.cursors import pytest from helpers.network import PartitionManager -import pytest +import logging from helpers.client import QueryRuntimeException from helpers.cluster import get_docker_compose_path, run_and_check import random @@ -12,7 +12,7 @@ import threading from multiprocessing.dummy import Pool from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=60, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=1, interval_seconds=3): lastest_result = '' for i in range(retry_count): @@ -21,10 +21,10 @@ def check_query(clickhouse_node, query, result_set, retry_count=60, interval_sec if result_set == lastest_result: return - print(lastest_result) + logging.debug(f"latest_result{lastest_result}") time.sleep(interval_seconds) except Exception as e: - print(("check_query retry {} exception {}".format(i + 1, e))) + logging.debug(f"check_query retry {i+1} exception {e}") time.sleep(interval_seconds) else: assert clickhouse_node.query(query) == result_set @@ -118,275 +118,275 @@ def dml_with_materialize_mysql_database(clickhouse_node, mysql_node, service_nam def materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + mysql_node.query("DROP DATABASE IF EXISTS test_database_dt") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_dt") + mysql_node.query("CREATE DATABASE test_database_dt DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_dt.test_table_1 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_1 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") - clickhouse_node.query("CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") + clickhouse_node.query("CREATE DATABASE test_database_dt ENGINE = MaterializeMySQL('{}:3306', 'test_database_dt', 'root', 'clickhouse')".format(service_name)) + assert "test_database_dt" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dt.test_table_1 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") + mysql_node.query("CREATE TABLE test_database_dt.test_table_2 (`key` INT NOT NULL PRIMARY KEY, _datetime DateTime(6), _timestamp TIMESTAMP(3), _decimal DECIMAL(65, 30)) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(1, '2020-01-01 01:02:03.999999', '2020-01-01 01:02:03.999', " + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(2, '2020-01-01 01:02:03.000000', '2020-01-01 01:02:03.000', ." + ('0' * 29) + "1)") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(3, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.99', -" + ('9' * 35) + "." + ('9' * 30) + ")") + mysql_node.query("INSERT INTO test_database_dt.test_table_2 VALUES(4, '2020-01-01 01:02:03.9999', '2020-01-01 01:02:03.9999', -." + ('0' * 29) + "1)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY key FORMAT TSV", + check_query(clickhouse_node, "SELECT * FROM test_database_dt.test_table_2 ORDER BY key FORMAT TSV", "1\t2020-01-01 01:02:03.999999\t2020-01-01 01:02:03.999\t" + ('9' * 35) + "." + ('9' * 30) + "\n" "2\t2020-01-01 01:02:03.000000\t2020-01-01 01:02:03.000\t0." + ('0' * 29) + "1\n" "3\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:03.990\t-" + ('9' * 35) + "." + ('9' * 30) + "\n" "4\t2020-01-01 01:02:03.999900\t2020-01-01 01:02:04.000\t-0." + ('0' * 29) + "1\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") - + clickhouse_node.query("DROP DATABASE test_database_dt") + mysql_node.query("DROP DATABASE test_database_dt") def drop_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_drop") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_drop") + mysql_node.query("CREATE DATABASE test_database_drop DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_drop.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("DROP TABLE test_database.test_table_1;") + mysql_node.query("DROP TABLE test_database_drop.test_table_1;") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database_drop.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") + mysql_node.query("TRUNCATE TABLE test_database_drop.test_table_2;") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_drop ENGINE = MaterializeMySQL('{}:3306', 'test_database_drop', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + assert "test_database_drop" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_drop.test_table_2 VALUES(1), (2), (3), (4), (5), (6)") + mysql_node.query("CREATE TABLE test_database_drop.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_drop FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") - mysql_node.query("DROP TABLE test_database.test_table_1;") - mysql_node.query("TRUNCATE TABLE test_database.test_table_2;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "") + mysql_node.query("DROP TABLE test_database_drop.test_table_1;") + mysql_node.query("TRUNCATE TABLE test_database_drop.test_table_2;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_drop FORMAT TSV", "test_table_2\n") + check_query(clickhouse_node, "SELECT * FROM test_database_drop.test_table_2 ORDER BY id FORMAT TSV", "") + + clickhouse_node.query("DROP DATABASE test_database_drop") + mysql_node.query("DROP DATABASE test_database_drop") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") def create_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_create") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_create") + mysql_node.query("CREATE DATABASE test_database_create DEFAULT CHARACTER SET 'utf8'") # existed before the mapping was created - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("CREATE TABLE test_database_create.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") # it already has some data - mysql_node.query("INSERT INTO test_database.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") + mysql_node.query("INSERT INTO test_database_create.test_table_1 VALUES(1), (2), (3), (5), (6), (7);") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_create ENGINE = MaterializeMySQL('{}:3306', 'test_database_create', 'root', 'clickhouse')".format( service_name)) # Check for pre-existing status - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_1 ORDER BY id FORMAT TSV", + assert "test_database_create" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SELECT * FROM test_database_create.test_table_1 ORDER BY id FORMAT TSV", "1\n2\n3\n5\n6\n7\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("CREATE TABLE test_database_create.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("INSERT INTO test_database_create.test_table_2 VALUES(1), (2), (3), (4), (5), (6);") + check_query(clickhouse_node, "SELECT * FROM test_database_create.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n6\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_create") + mysql_node.query("DROP DATABASE test_database_create") def rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_rename") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_rename") + mysql_node.query("CREATE DATABASE test_database_rename DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_rename.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("RENAME TABLE test_database.test_table_1 TO test_database.test_table_2") + mysql_node.query("RENAME TABLE test_database_rename.test_table_1 TO test_database_rename.test_table_2") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_rename ENGINE = MaterializeMySQL('{}:3306', 'test_database_rename', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_2\n") - mysql_node.query("RENAME TABLE test_database.test_table_2 TO test_database.test_table_1") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") + assert "test_database_rename" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename FORMAT TSV", "test_table_2\n") + mysql_node.query("RENAME TABLE test_database_rename.test_table_2 TO test_database_rename.test_table_1") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename FORMAT TSV", "test_table_1\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_rename") + mysql_node.query("DROP DATABASE test_database_rename") def alter_add_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") - mysql_node.query("CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + mysql_node.query("DROP DATABASE IF EXISTS test_database_add") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_add") + mysql_node.query("CREATE DATABASE test_database_add DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("CREATE TABLE test_database_add.test_table_1 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") - mysql_node.query("ALTER TABLE test_database.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_1 INT NOT NULL") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_2 INT NOT NULL FIRST") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1") + mysql_node.query("ALTER TABLE test_database_add.test_table_1 ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( "0" if service_name == "mysql57" else "(id)")) # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_add ENGINE = MaterializeMySQL('{}:3306', 'test_database_add', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_add" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "DESC test_database_add.test_table_1 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("CREATE TABLE test_database_add.test_table_2 (id INT NOT NULL PRIMARY KEY) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_add FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_add.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") + "ALTER TABLE test_database_add.test_table_2 ADD COLUMN add_column_1 INT NOT NULL, ADD COLUMN add_column_2 INT NOT NULL FIRST") mysql_node.query( - "ALTER TABLE test_database.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( + "ALTER TABLE test_database_add.test_table_2 ADD COLUMN add_column_3 INT NOT NULL AFTER add_column_1, ADD COLUMN add_column_4 INT NOT NULL DEFAULT " + ( "0" if service_name == "mysql57" else "(id)")) default_expression = "DEFAULT\t0" if service_name == "mysql57" else "DEFAULT\tid" - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + check_query(clickhouse_node, "DESC test_database_add.test_table_2 FORMAT TSV", "add_column_2\tInt32\t\t\t\t\t\nid\tInt32\t\t\t\t\t\nadd_column_1\tInt32\t\t\t\t\t\nadd_column_3\tInt32\t\t\t\t\t\nadd_column_4\tInt32\t" + default_expression + "\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_add.test_table_2 VALUES(1, 2, 3, 4, 5), (6, 7, 8, 9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database_add.test_table_2 ORDER BY id FORMAT TSV", "1\t2\t3\t4\t5\n6\t7\t8\t9\t10\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_add") + mysql_node.query("DROP DATABASE test_database_add") def alter_drop_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_drop") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_drop") + mysql_node.query("CREATE DATABASE test_database_alter_drop DEFAULT CHARACTER SET 'utf8'") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_drop.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column") + mysql_node.query("ALTER TABLE test_database_alter_drop.test_table_1 DROP COLUMN drop_column") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_drop ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_drop', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_drop" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_drop FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_drop.test_table_2 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_drop FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 DROP COLUMN drop_column") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_drop.test_table_2 DROP COLUMN drop_column") + check_query(clickhouse_node, "DESC test_database_alter_drop.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1), (2), (3), (4), (5)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("INSERT INTO test_database_alter_drop.test_table_2 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_drop.test_table_2 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_drop") + mysql_node.query("DROP DATABASE test_database_alter_drop") def alter_rename_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_rename") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_rename") + mysql_node.query("CREATE DATABASE test_database_alter_rename DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_rename.test_table_1 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 RENAME COLUMN rename_column TO new_column_name") + mysql_node.query("ALTER TABLE test_database_alter_rename.test_table_1 RENAME COLUMN rename_column TO new_column_name") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_rename ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_rename', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_rename" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_rename.test_table_2 (id INT NOT NULL PRIMARY KEY, rename_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nrename_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 RENAME COLUMN rename_column TO new_column_name") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_rename.test_table_2 RENAME COLUMN rename_column TO new_column_name") + check_query(clickhouse_node, "DESC test_database_alter_rename.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nnew_column_name\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", + mysql_node.query("INSERT INTO test_database_alter_rename.test_table_2 VALUES(1, 2), (3, 4), (5, 6), (7, 8), (9, 10)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_rename.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t4\n5\t6\n7\t8\n9\t10\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_rename") + mysql_node.query("DROP DATABASE test_database_alter_rename") def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_alter_modify") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_alter_modify") + mysql_node.query("CREATE DATABASE test_database_alter_modify DEFAULT CHARACTER SET 'utf8'") # maybe should test rename primary key? mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + "CREATE TABLE test_database_alter_modify.test_table_1 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - mysql_node.query("ALTER TABLE test_database.test_table_1 MODIFY COLUMN modify_column INT") + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_1 MODIFY COLUMN modify_column INT") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_alter_modify ENGINE = MaterializeMySQL('{}:3306', 'test_database_alter_modify', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + assert "test_database_alter_modify" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_modify FORMAT TSV", "test_table_1\n") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_2\n") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + "CREATE TABLE test_database_alter_modify.test_table_2 (id INT NOT NULL PRIMARY KEY, modify_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_alter_modify FORMAT TSV", "test_table_1\ntest_table_2\n") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT FIRST") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT FIRST") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "modify_column\tNullable(Int32)\t\t\t\t\t\nid\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("ALTER TABLE test_database.test_table_2 MODIFY COLUMN modify_column INT AFTER id") - check_query(clickhouse_node, "DESC test_database.test_table_2 FORMAT TSV", + mysql_node.query("ALTER TABLE test_database_alter_modify.test_table_2 MODIFY COLUMN modify_column INT AFTER id") + check_query(clickhouse_node, "DESC test_database_alter_modify.test_table_2 FORMAT TSV", "id\tInt32\t\t\t\t\t\nmodify_column\tNullable(Int32)\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_2 VALUES(1, 2), (3, NULL)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") + mysql_node.query("INSERT INTO test_database_alter_modify.test_table_2 VALUES(1, 2), (3, NULL)") + check_query(clickhouse_node, "SELECT * FROM test_database_alter_modify.test_table_2 ORDER BY id FORMAT TSV", "1\t2\n3\t\\N\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_alter_modify") + mysql_node.query("DROP DATABASE test_database_alter_modify") # TODO: need ClickHouse support ALTER TABLE table_name ADD COLUMN column_name, RENAME COLUMN column_name TO new_column_name; @@ -394,53 +394,53 @@ def alter_modify_column_with_materialize_mysql_database(clickhouse_node, mysql_n # pass def alter_rename_table_with_materialize_mysql_database(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database DEFAULT CHARACTER SET 'utf8'") + mysql_node.query("DROP DATABASE IF EXISTS test_database_rename_table") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_rename_table") + mysql_node.query("CREATE DATABASE test_database_rename_table DEFAULT CHARACTER SET 'utf8'") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") + "CREATE TABLE test_database_rename_table.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT) ENGINE = InnoDB;") mysql_node.query( - "ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_3") + "ALTER TABLE test_database_rename_table.test_table_1 DROP COLUMN drop_column, RENAME TO test_database_rename_table.test_table_2, RENAME TO test_database_rename_table.test_table_3") # create mapping clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_rename_table ENGINE = MaterializeMySQL('{}:3306', 'test_database_rename_table', 'root', 'clickhouse')".format( service_name)) - assert "test_database" in clickhouse_node.query("SHOW DATABASES") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_3 FORMAT TSV", + assert "test_database_rename_table" in clickhouse_node.query("SHOW DATABASES") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_3\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_3 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "CREATE TABLE test_database.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_1\ntest_table_3\n") - check_query(clickhouse_node, "DESC test_database.test_table_1 FORMAT TSV", + "CREATE TABLE test_database_rename_table.test_table_1 (id INT NOT NULL PRIMARY KEY, drop_column INT NOT NULL) ENGINE = InnoDB;") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_1\ntest_table_3\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_1 FORMAT TSV", "id\tInt32\t\t\t\t\t\ndrop_column\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") mysql_node.query( - "ALTER TABLE test_database.test_table_1 DROP COLUMN drop_column, RENAME TO test_database.test_table_2, RENAME TO test_database.test_table_4") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table_3\ntest_table_4\n") - check_query(clickhouse_node, "DESC test_database.test_table_4 FORMAT TSV", + "ALTER TABLE test_database_rename_table.test_table_1 DROP COLUMN drop_column, RENAME TO test_database_rename_table.test_table_2, RENAME TO test_database_rename_table.test_table_4") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_rename_table FORMAT TSV", "test_table_3\ntest_table_4\n") + check_query(clickhouse_node, "DESC test_database_rename_table.test_table_4 FORMAT TSV", "id\tInt32\t\t\t\t\t\n_sign\tInt8\tMATERIALIZED\t1\t\t\t\n_version\tUInt64\tMATERIALIZED\t1\t\t\t\n") - mysql_node.query("INSERT INTO test_database.test_table_4 VALUES(1), (2), (3), (4), (5)") - check_query(clickhouse_node, "SELECT * FROM test_database.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") + mysql_node.query("INSERT INTO test_database_rename_table.test_table_4 VALUES(1), (2), (3), (4), (5)") + check_query(clickhouse_node, "SELECT * FROM test_database_rename_table.test_table_4 ORDER BY id FORMAT TSV", "1\n2\n3\n4\n5\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_rename_table") + mysql_node.query("DROP DATABASE test_database_rename_table") def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name): - mysql_node.query("DROP DATABASE IF EXISTS test_database") - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") - mysql_node.query("CREATE DATABASE test_database") + mysql_node.query("DROP DATABASE IF EXISTS test_database_event") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_event") + mysql_node.query("CREATE DATABASE test_database_event") mysql_node.query("RESET MASTER") - mysql_node.query("CREATE TABLE test_database.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(1)") + mysql_node.query("CREATE TABLE test_database_event.t1(a INT NOT NULL PRIMARY KEY, b VARCHAR(255) DEFAULT 'BEGIN')") + mysql_node.query("INSERT INTO test_database_event.t1(a) VALUES(1)") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format( + "CREATE DATABASE test_database_event ENGINE = MaterializeMySQL('{}:3306', 'test_database_event', 'root', 'clickhouse')".format( service_name)) # Reject one empty GTID QUERY event with 'BEGIN' and 'COMMIT' @@ -456,13 +456,13 @@ def query_event_with_empty_transaction(clickhouse_node, mysql_node, service_name # Reject one 'BEGIN' QUERY event and 'COMMIT' XID event. mysql_node.query("/* start */ begin /* end */") - mysql_node.query("INSERT INTO test_database.t1(a) VALUES(2)") + mysql_node.query("INSERT INTO test_database_event.t1(a) VALUES(2)") mysql_node.query("/* start */ commit /* end */") - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "t1\n") - check_query(clickhouse_node, "SELECT * FROM test_database.t1 ORDER BY a FORMAT TSV", "1\tBEGIN\n2\tBEGIN\n") - clickhouse_node.query("DROP DATABASE test_database") - mysql_node.query("DROP DATABASE test_database") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_event FORMAT TSV", "t1\n") + check_query(clickhouse_node, "SELECT * FROM test_database_event.t1 ORDER BY a FORMAT TSV", "1\tBEGIN\n2\tBEGIN\n") + clickhouse_node.query("DROP DATABASE test_database_event") + mysql_node.query("DROP DATABASE test_database_event") def select_without_columns(clickhouse_node, mysql_node, service_name): @@ -592,23 +592,23 @@ def drop_instance_mysql_connections(clickhouse_node, pm, action='REJECT'): time.sleep(5) def network_partition_test(clickhouse_node, mysql_node, service_name): - clickhouse_node.query("DROP DATABASE IF EXISTS test_database") + clickhouse_node.query("DROP DATABASE IF EXISTS test_database_network") clickhouse_node.query("DROP DATABASE IF EXISTS test") - mysql_node.query("DROP DATABASE IF EXISTS test_database") + mysql_node.query("DROP DATABASE IF EXISTS test_database_network") mysql_node.query("DROP DATABASE IF EXISTS test") - mysql_node.query("CREATE DATABASE test_database;") - mysql_node.query("CREATE TABLE test_database.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") + mysql_node.query("CREATE DATABASE test_database_network;") + mysql_node.query("CREATE TABLE test_database_network.test_table ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") mysql_node.query("CREATE DATABASE test;") clickhouse_node.query( - "CREATE DATABASE test_database ENGINE = MaterializeMySQL('{}:3306', 'test_database', 'root', 'clickhouse')".format(service_name)) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table", '') + "CREATE DATABASE test_database_network ENGINE = MaterializeMySQL('{}:3306', 'test_database_network', 'root', 'clickhouse')".format(service_name)) + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table", '') with PartitionManager() as pm: drop_instance_mysql_connections(clickhouse_node, pm) - mysql_node.query('INSERT INTO test_database.test_table VALUES(1)') - check_query(clickhouse_node, "SELECT * FROM test_database.test_table", '') + mysql_node.query('INSERT INTO test_database_network.test_table VALUES(1)') + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table", '') with pytest.raises(QueryRuntimeException) as exception: clickhouse_node.query( @@ -618,18 +618,18 @@ def network_partition_test(clickhouse_node, mysql_node, service_name): restore_instance_mysql_connections(clickhouse_node, pm) - check_query(clickhouse_node, "SELECT * FROM test_database.test_table FORMAT TSV", '1\n') + check_query(clickhouse_node, "SELECT * FROM test_database_network.test_table FORMAT TSV", '1\n') clickhouse_node.query( "CREATE DATABASE test ENGINE = MaterializeMySQL('{}:3306', 'test', 'root', 'clickhouse')".format(service_name)) - check_query(clickhouse_node, "SHOW TABLES FROM test_database FORMAT TSV", "test_table\n") + check_query(clickhouse_node, "SHOW TABLES FROM test_database_network FORMAT TSV", "test_table\n") mysql_node.query("CREATE TABLE test.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") check_query(clickhouse_node, "SHOW TABLES FROM test FORMAT TSV", "test\n") - clickhouse_node.query("DROP DATABASE test_database") + clickhouse_node.query("DROP DATABASE test_database_network") clickhouse_node.query("DROP DATABASE test") - mysql_node.query("DROP DATABASE test_database") + mysql_node.query("DROP DATABASE test_database_network") mysql_node.query("DROP DATABASE test") diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index f617d5f958f..3f256fd5d67 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -31,7 +31,7 @@ def started_cluster(): class MySQLConnection: - def __init__(self, port, user='root', password='clickhouse', ip_address='127.0.0.1', docker_compose=None, project_name=cluster.project_name): + def __init__(self, port, user='root', password='clickhouse', ip_address=None, docker_compose=None, project_name=cluster.project_name): self.user = user self.port = port self.ip_address = ip_address @@ -77,39 +77,31 @@ class MySQLConnection: cursor.execute(executio_query) return cursor.fetchall() - def start_and_wait(self): - run_and_check(['docker-compose', - '-p', cluster.project_name, - '-f', self.docker_compose, - 'up', '--no-recreate', '-d', - ]) - self.wait_mysql_to_start(120) - def close(self): if self.mysql_connection is not None: self.mysql_connection.close() @pytest.fixture(scope="module") def started_mysql_5_7(): - mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', '127.0.0.1') + mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', cluster.mysql_ip) yield mysql_node @pytest.fixture(scope="module") def started_mysql_8_0(): - mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', '127.0.0.1') + mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', cluster.mysql8_ip) yield mysql8_node -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @@ -121,7 +113,7 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.create_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @@ -132,73 +124,73 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) +@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") From c3c409556eef70f9391c699eda7dcc338e2139c1 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:44:28 +0300 Subject: [PATCH 0067/1060] support tests list in runner --- tests/integration/runner | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/tests/integration/runner b/tests/integration/runner index 3aa3aa5ef64..1c1aed2f6db 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -148,6 +148,14 @@ if __name__ == "__main__": dest="parallel", help="Parallelism") + parser.add_argument( + "-t", "--tests_list", + action="store", + nargs='+', + default=[], + dest="tests_list", + help="List of tests to run") + parser.add_argument('pytest_args', nargs='*', help="args for pytest command") args = parser.parse_args() @@ -200,7 +208,7 @@ if __name__ == "__main__": --volume={base_cfg}:/clickhouse-config --volume={cases_dir}:/ClickHouse/tests/integration \ --volume={src_dir}/Server/grpc_protos:/ClickHouse/src/Server/grpc_protos \ --volume={name}_volume:/var/lib/docker -e DOCKER_CLIENT_TIMEOUT=120 -e COMPOSE_HTTP_TIMEOUT=300 \ - {env_tags} -e PYTEST_OPTS='{parallel} {opts}' {img} {command}".format( + {env_tags} -e PYTEST_OPTS='{parallel} {opts} {tests_list}' {img} {command}".format( net=net, tty=tty, bin=args.binary, @@ -211,6 +219,7 @@ if __name__ == "__main__": env_tags=env_tags, parallel=parallel_args, opts=' '.join(args.pytest_args), + tests_list=' '.join(args.tests_list), img=DIND_INTEGRATION_TESTS_IMAGE_NAME + ":" + args.docker_image_version, name=CONTAINER_NAME, command=args.command From 8f53d1344cbf5260bbc0884f330aa966c0d8fffb Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:48:28 +0300 Subject: [PATCH 0068/1060] better test cases names --- .../integration/test_dictionaries_ddl/test.py | 8 +- .../test_distributed_over_distributed/test.py | 11 +- tests/integration/test_multiple_disks/test.py | 50 +++--- .../test_mysql_database_engine/test.py | 84 ++++----- .../test.py | 12 +- tests/integration/test_ttl_move/test.py | 168 +++++++++--------- tests/integration/test_ttl_replicated/test.py | 4 +- 7 files changed, 170 insertions(+), 167 deletions(-) diff --git a/tests/integration/test_dictionaries_ddl/test.py b/tests/integration/test_dictionaries_ddl/test.py index 3aa76b4af88..ea0de044e6a 100644 --- a/tests/integration/test_dictionaries_ddl/test.py +++ b/tests/integration/test_dictionaries_ddl/test.py @@ -60,10 +60,10 @@ def started_cluster(): @pytest.mark.parametrize("clickhouse,name,layout", [ - (node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), - (node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), - (node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())'), - (node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))'), + pytest.param(node1, 'complex_node1_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())', id="complex_node1_hashed"), + pytest.param(node1, 'complex_node1_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))', id="complex_node1_cache"), + pytest.param(node2, 'complex_node2_hashed', 'LAYOUT(COMPLEX_KEY_HASHED())', id="complex_node2_hashed"), + pytest.param(node2, 'complex_node2_cache', 'LAYOUT(COMPLEX_KEY_CACHE(SIZE_IN_CELLS 10))', id="complex_node2_cache"), ]) def test_create_and_select_mysql(started_cluster, clickhouse, name, layout): mysql_conn = create_mysql_conn("root", "clickhouse", started_cluster.mysql_ip, started_cluster.mysql_port) diff --git a/tests/integration/test_distributed_over_distributed/test.py b/tests/integration/test_distributed_over_distributed/test.py index 410a03a6af1..ae86a70f31b 100644 --- a/tests/integration/test_distributed_over_distributed/test.py +++ b/tests/integration/test_distributed_over_distributed/test.py @@ -2,7 +2,6 @@ # (just in case, with real separate instances). - import pytest from helpers.cluster import ClickHouseCluster @@ -51,9 +50,13 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("node", list(NODES.values())) -@pytest.mark.parametrize("source", - ["distributed_over_distributed_table", "cluster('test_cluster', default, distributed_table)"]) +@pytest.mark.parametrize("node,source", [ + pytest.param(NODES["node1"], "distributed_over_distributed_table", id="dod_node1"), + pytest.param(NODES["node1"], "cluster('test_cluster', default, distributed_table)", id="cluster_node1"), + pytest.param(NODES["node2"], "distributed_over_distributed_table", id="dod_node2"), + pytest.param(NODES["node2"], "cluster('test_cluster', default, distributed_table)", id="cluster_node2"), +] +) class TestDistributedOverDistributedSuite: def test_select_with_order_by_node(self, started_cluster, node, source): assert node.query("SELECT * FROM {source} ORDER BY node, key".format(source=source)) \ diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 24ee6c0493b..1d672f8f7ce 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -290,8 +290,8 @@ def test_query_parser(start_cluster): @pytest.mark.parametrize("name,engine", [ - ("test_alter_policy", "MergeTree()"), - ("replicated_test_alter_policy", "ReplicatedMergeTree('/clickhouse/test_alter_policy', '1')",), + pytest.param("test_alter_policy", "MergeTree()", id="mt"), + pytest.param("replicated_test_alter_policy", "ReplicatedMergeTree('/clickhouse/test_alter_policy', '1')", id="replicated"), ]) def test_alter_policy(start_cluster, name, engine): try: @@ -364,8 +364,8 @@ def test_no_warning_about_zero_max_data_part_size(start_cluster): @pytest.mark.parametrize("name,engine", [ - ("mt_on_jbod", "MergeTree()"), - ("replicated_mt_on_jbod", "ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')",), + pytest.param("mt_on_jbod", "MergeTree()", id="mt"), + pytest.param("replicated_mt_on_jbod", "ReplicatedMergeTree('/clickhouse/replicated_mt_on_jbod', '1')", id="replicated"), ]) def test_round_robin(start_cluster, name, engine): try: @@ -400,8 +400,8 @@ def test_round_robin(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("mt_with_huge_part", "MergeTree()"), - ("replicated_mt_with_huge_part", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')",), + pytest.param("mt_with_huge_part", "MergeTree()", id="mt"), + pytest.param("replicated_mt_with_huge_part", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_huge_part', '1')", id="replicated"), ]) def test_max_data_part_size(start_cluster, name, engine): try: @@ -427,8 +427,8 @@ def test_max_data_part_size(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("mt_with_overflow", "MergeTree()"), - ("replicated_mt_with_overflow", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')",), + pytest.param("mt_with_overflow", "MergeTree()", id="mt"), + pytest.param("replicated_mt_with_overflow", "ReplicatedMergeTree('/clickhouse/replicated_mt_with_overflow', '1')", id="replicated"), ]) def test_jbod_overflow(start_cluster, name, engine): try: @@ -480,8 +480,8 @@ def test_jbod_overflow(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("moving_mt", "MergeTree()"), - ("moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')",), + pytest.param("moving_mt", "MergeTree()", id="mt"), + pytest.param("moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/moving_replicated_mt', '1')", id="replicated"), ]) def test_background_move(start_cluster, name, engine): try: @@ -530,8 +530,8 @@ def test_background_move(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("stopped_moving_mt", "MergeTree()"), - ("stopped_moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')",), + pytest.param("stopped_moving_mt", "MergeTree()", id="mt"), + pytest.param("stopped_moving_replicated_mt", "ReplicatedMergeTree('/clickhouse/stopped_moving_replicated_mt', '1')", id="replicated"), ]) def test_start_stop_moves(start_cluster, name, engine): try: @@ -633,7 +633,7 @@ def get_paths_for_partition_from_part_log(node, table, partition_id): @pytest.mark.parametrize("name,engine", [ - ("altering_mt", "MergeTree()"), + pytest.param("altering_mt", "MergeTree()", id="mt"), # ("altering_replicated_mt","ReplicatedMergeTree('/clickhouse/altering_replicated_mt', '1')",), # SYSTEM STOP MERGES doesn't disable merges assignments ]) @@ -827,9 +827,9 @@ def produce_alter_move(node, name): @pytest.mark.parametrize("name,engine", [ - ("concurrently_altering_mt", "MergeTree()"), - ("concurrently_altering_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')",), + pytest.param("concurrently_altering_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_altering_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_altering_replicated_mt', '1')", id="replicated"), ]) def test_concurrent_alter_move(start_cluster, name, engine): try: @@ -882,9 +882,9 @@ def test_concurrent_alter_move(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("concurrently_dropping_mt", "MergeTree()"), - ("concurrently_dropping_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')",), + pytest.param("concurrently_dropping_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_dropping_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_dropping_replicated_mt', '1')", id="replicated"), ]) def test_concurrent_alter_move_and_drop(start_cluster, name, engine): try: @@ -935,8 +935,8 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("detach_attach_mt", "MergeTree()"), - ("replicated_detach_attach_mt", "ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')",), + pytest.param("detach_attach_mt", "MergeTree()", id="mt"), + pytest.param("replicated_detach_attach_mt", "ReplicatedMergeTree('/clickhouse/replicated_detach_attach_mt', '1')", id="replicated"), ]) def test_detach_attach(start_cluster, name, engine): try: @@ -966,8 +966,8 @@ def test_detach_attach(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("mutating_mt", "MergeTree()"), - ("replicated_mutating_mt", "ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')",), + pytest.param("mutating_mt", "MergeTree()", id="mt"), + pytest.param("replicated_mutating_mt", "ReplicatedMergeTree('/clickhouse/replicated_mutating_mt', '1')", id="replicated"), ]) def test_mutate_to_another_disk(start_cluster, name, engine): try: @@ -1012,8 +1012,8 @@ def test_mutate_to_another_disk(start_cluster, name, engine): @pytest.mark.parametrize("name,engine", [ - ("alter_modifying_mt", "MergeTree()"), - ("replicated_alter_modifying_mt", "ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')",), + pytest.param("alter_modifying_mt", "MergeTree()", id="mt"), + pytest.param("replicated_alter_modifying_mt", "ReplicatedMergeTree('/clickhouse/replicated_alter_modifying_mt', '1')", id="replicated"), ]) def test_concurrent_alter_modify(start_cluster, name, engine): try: diff --git a/tests/integration/test_mysql_database_engine/test.py b/tests/integration/test_mysql_database_engine/test.py index 07a4a987b59..5e6486e31c6 100644 --- a/tests/integration/test_mysql_database_engine/test.py +++ b/tests/integration/test_mysql_database_engine/test.py @@ -220,60 +220,60 @@ timestamp_values_no_subsecond = ["'2015-05-18 07:40:01'", "'2019-09-16 19:20:11' # ("common_types", "FLOAT", "Nullable(Float32)", float_values, ""), # ("common_types", "FLOAT UNSIGNED", "Nullable(Float32)", float_values, ""), - ("common_types", "INT", "Nullable(Int32)", int32_values, ""), - ("common_types", "INT NOT NULL", "Int32", int32_values, ""), - ("common_types", "INT UNSIGNED NOT NULL", "UInt32", uint32_values, ""), - ("common_types", "INT UNSIGNED", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10)", "Nullable(Int32)", int32_values, ""), - ("common_types", "INT(10) NOT NULL", "Int32", int32_values, ""), - ("common_types", "INT(10) UNSIGNED NOT NULL", "UInt32", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INT(10) UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, ""), - ("common_types", "INTEGER", "Nullable(Int32)", int32_values, ""), - ("common_types", "INTEGER UNSIGNED", "Nullable(UInt32)", uint32_values, ""), + pytest.param("common_types", "INT", "Nullable(Int32)", int32_values, "", id="common_types_1"), + pytest.param("common_types", "INT NOT NULL", "Int32", int32_values, "", id="common_types_2"), + pytest.param("common_types", "INT UNSIGNED NOT NULL", "UInt32", uint32_values, "", id="common_types_3"), + pytest.param("common_types", "INT UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_4"), + pytest.param("common_types", "INT UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, "", id="common_types_5"), + pytest.param("common_types", "INT UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, "", id="common_types_6"), + pytest.param("common_types", "INT(10)", "Nullable(Int32)", int32_values, "", id="common_types_7"), + pytest.param("common_types", "INT(10) NOT NULL", "Int32", int32_values, "", id="common_types_8"), + pytest.param("common_types", "INT(10) UNSIGNED NOT NULL", "UInt32", uint32_values, "", id="common_types_8"), + pytest.param("common_types", "INT(10) UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_9"), + pytest.param("common_types", "INT(10) UNSIGNED DEFAULT NULL", "Nullable(UInt32)", uint32_values, "", id="common_types_10"), + pytest.param("common_types", "INT(10) UNSIGNED DEFAULT '1'", "Nullable(UInt32)", uint32_values, "", id="common_types_11"), + pytest.param("common_types", "INTEGER", "Nullable(Int32)", int32_values, "", id="common_types_12"), + pytest.param("common_types", "INTEGER UNSIGNED", "Nullable(UInt32)", uint32_values, "", id="common_types_13"), - ("common_types", "MEDIUMINT", "Nullable(Int32)", mint_values, ""), - ("common_types", "MEDIUMINT UNSIGNED", "Nullable(UInt32)", umint_values, ""), + pytest.param("common_types", "MEDIUMINT", "Nullable(Int32)", mint_values, "", id="common_types_14"), + pytest.param("common_types", "MEDIUMINT UNSIGNED", "Nullable(UInt32)", umint_values, "", id="common_types_15"), - ("common_types", "SMALLINT", "Nullable(Int16)", int16_values, ""), - ("common_types", "SMALLINT UNSIGNED", "Nullable(UInt16)", uint16_values, ""), + pytest.param("common_types", "SMALLINT", "Nullable(Int16)", int16_values, "", id="common_types_16"), + pytest.param("common_types", "SMALLINT UNSIGNED", "Nullable(UInt16)", uint16_values, "", id="common_types_17"), - ("common_types", "TINYINT", "Nullable(Int8)", int8_values, ""), - ("common_types", "TINYINT UNSIGNED", "Nullable(UInt8)", uint8_values, ""), + pytest.param("common_types", "TINYINT", "Nullable(Int8)", int8_values, "", id="common_types_18"), + pytest.param("common_types", "TINYINT UNSIGNED", "Nullable(UInt8)", uint8_values, "", id="common_types_19"), - ("common_types", "VARCHAR(10)", "Nullable(String)", string_values, ""), + pytest.param("common_types", "VARCHAR(10)", "Nullable(String)", string_values, "", id="common_types_20"), - ("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values, - "decimal,datetime64"), - ("decimal_default_nullable", "decimal", "Nullable(Decimal(10, 0))", decimal_values, - "decimal,datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "Decimal(18, 6)", decimal_values, - "decimal,datetime64"), - ("decimal_38_6", "decimal(38, 6) NOT NULL", "Decimal(38, 6)", decimal_values, - "decimal,datetime64"), + pytest.param("decimal_default", "decimal NOT NULL", "Decimal(10, 0)", decimal_values, + "decimal,datetime64", id="decimal_1"), + pytest.param("decimal_default_nullable", "decimal", "Nullable(Decimal(10, 0))", decimal_values, + "decimal,datetime64", id="decimal_2"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "Decimal(18, 6)", decimal_values, + "decimal,datetime64", id="decimal_3"), + pytest.param("decimal_38_6", "decimal(38, 6) NOT NULL", "Decimal(38, 6)", decimal_values, + "decimal,datetime64", id="decimal_4"), # Due to python DB driver roundtrip MySQL timestamp and datetime values # are printed with 6 digits after decimal point, so to simplify tests a bit, # we only validate precision of 0 and 6. - ("timestamp_default", "timestamp", "DateTime", timestamp_values, "decimal,datetime64"), - ("timestamp_6", "timestamp(6)", "DateTime64(6)", timestamp_values, "decimal,datetime64"), - ("datetime_default", "DATETIME NOT NULL", "DateTime64(0)", timestamp_values, - "decimal,datetime64"), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime64(6)", timestamp_values, - "decimal,datetime64"), + pytest.param("timestamp_default", "timestamp", "DateTime", timestamp_values, "decimal,datetime64", id="timestamp_default"), + pytest.param("timestamp_6", "timestamp(6)", "DateTime64(6)", timestamp_values, "decimal,datetime64", id="timestamp_6"), + pytest.param("datetime_default", "DATETIME NOT NULL", "DateTime64(0)", timestamp_values, + "decimal,datetime64", id="datetime_default"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime64(6)", timestamp_values, + "decimal,datetime64", id="datetime_6_1"), # right now precision bigger than 39 is not supported by ClickHouse's Decimal, hence fall back to String - ("decimal_40_6", "decimal(40, 6) NOT NULL", "String", decimal_values, - "decimal,datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "datetime64"), - ("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, ""), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, - "decimal"), - ("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, ""), + pytest.param("decimal_40_6", "decimal(40, 6) NOT NULL", "String", decimal_values, + "decimal,datetime64", id="decimal_40_6"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "datetime64", id="decimal_18_6_1"), + pytest.param("decimal_18_6", "decimal(18, 6) NOT NULL", "String", decimal_values, "", id="decimal_18_6_2"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, + "decimal", id="datetime_6_2"), + pytest.param("datetime_6", "DATETIME(6) NOT NULL", "DateTime", timestamp_values_no_subsecond, "", id="datetime_6_3"), ]) def test_mysql_types(started_cluster, case_name, mysql_type, expected_ch_type, mysql_values, setting_mysql_datatypes_support_level): diff --git a/tests/integration/test_send_request_to_leader_replica/test.py b/tests/integration/test_send_request_to_leader_replica/test.py index 8c58d02d104..721e446ff82 100644 --- a/tests/integration/test_send_request_to_leader_replica/test.py +++ b/tests/integration/test_send_request_to_leader_replica/test.py @@ -39,12 +39,12 @@ def started_cluster(): @pytest.mark.parametrize("table,query,expected,n1,n2", [ - ("sometable", "ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2), - ("sometable", "TRUNCATE TABLE sometable", '0', node1, node2), - ("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2), - ("someothertable", "ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4), - ("someothertable", "TRUNCATE TABLE someothertable", '0', node3, node4), - ("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4), + pytest.param("sometable", "ALTER TABLE sometable DROP PARTITION 201706", '1', node1, node2, id="case1"), + pytest.param("sometable", "TRUNCATE TABLE sometable", '0', node1, node2, id="case2"), + pytest.param("sometable", "OPTIMIZE TABLE sometable", '4', node1, node2, id="case3"), + pytest.param("someothertable", "ALTER TABLE someothertable DROP PARTITION 201706", '1', node3, node4, id="case4"), + pytest.param("someothertable", "TRUNCATE TABLE someothertable", '0', node3, node4, id="case5"), + pytest.param("someothertable", "OPTIMIZE TABLE someothertable", '4', node3, node4, id="case6"), ]) def test_alter_table_drop_partition(started_cluster, table, query, expected, n1, n2): to_insert = '''\ diff --git a/tests/integration/test_ttl_move/test.py b/tests/integration/test_ttl_move/test.py index 7e01b919589..254447478f9 100644 --- a/tests/integration/test_ttl_move/test.py +++ b/tests/integration/test_ttl_move/test.py @@ -73,12 +73,12 @@ def wait_parts_mover(node, table, *args, **kwargs): @pytest.mark.parametrize("name,engine,alter", [ - ("mt_test_rule_with_invalid_destination", "MergeTree()", 0), - ("replicated_mt_test_rule_with_invalid_destination", - "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 0), - ("mt_test_rule_with_invalid_destination", "MergeTree()", 1), - ("replicated_mt_test_rule_with_invalid_destination", - "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1), + pytest.param("mt_test_rule_with_invalid_destination", "MergeTree()", 0, id="case0"), + pytest.param("replicated_mt_test_rule_with_invalid_destination", + "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 0, id="case1"), + pytest.param("mt_test_rule_with_invalid_destination", "MergeTree()", 1, id="case2"), + pytest.param("replicated_mt_test_rule_with_invalid_destination", + "ReplicatedMergeTree('/clickhouse/replicated_test_rule_with_invalid_destination', '1')", 1, id="case3"), ]) def test_rule_with_invalid_destination(started_cluster, name, engine, alter): name = unique_table_name(name) @@ -136,12 +136,12 @@ def test_rule_with_invalid_destination(started_cluster, name, engine, alter): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_inserts_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_inserts_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_do_not_work', '1')", 0), - ("mt_test_inserts_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_inserts_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1), + pytest.param("mt_test_inserts_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_inserts_to_disk_do_not_work"), + pytest.param("replicated_mt_test_inserts_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_do_not_work', '1')", 0, id="replicated_mt_test_inserts_to_disk_do_not_work"), + pytest.param("mt_test_inserts_to_disk_work", "MergeTree()", 1, id="mt_test_inserts_to_disk_work_1"), + pytest.param("replicated_mt_test_inserts_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_disk_work', '1')", 1, id="replicated_mt_test_inserts_to_disk_work_1"), ]) def test_inserts_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -176,9 +176,9 @@ def test_inserts_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_work_after_storage_policy_change", "MergeTree()"), - ("replicated_mt_test_moves_work_after_storage_policy_change", - "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')"), + pytest.param("mt_test_moves_work_after_storage_policy_change", "MergeTree()", id="mt_test_moves_work_after_storage_policy_change"), + pytest.param("replicated_mt_test_moves_work_after_storage_policy_change", + "ReplicatedMergeTree('/clickhouse/test_moves_work_after_storage_policy_change', '1')", id="replicated_mt_test_moves_work_after_storage_policy_change"), ]) def test_moves_work_after_storage_policy_change(started_cluster, name, engine): name = unique_table_name(name) @@ -224,12 +224,12 @@ def test_moves_work_after_storage_policy_change(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_moves_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_moves_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_do_not_work', '1')", 0), - ("mt_test_moves_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_moves_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1), + pytest.param("mt_test_moves_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_moves_to_disk_do_not_work"), + pytest.param("replicated_mt_test_moves_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_do_not_work', '1')", 0, id="replicated_mt_test_moves_to_disk_do_not_work"), + pytest.param("mt_test_moves_to_disk_work", "MergeTree()", 1, id="mt_test_moves_to_disk_work"), + pytest.param("replicated_mt_test_moves_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_work', '1')", 1, id="replicated_mt_test_moves_to_disk_work"), ]) def test_moves_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -275,9 +275,9 @@ def test_moves_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_to_volume_work", "MergeTree()"), - ("replicated_mt_test_moves_to_volume_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')"), + pytest.param("mt_test_moves_to_volume_work", "MergeTree()", id="mt_test_moves_to_volume_work"), + pytest.param("replicated_mt_test_moves_to_volume_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_volume_work', '1')", id="replicated_mt_test_moves_to_volume_work"), ]) def test_moves_to_volume_work(started_cluster, name, engine): name = unique_table_name(name) @@ -322,12 +322,12 @@ def test_moves_to_volume_work(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_inserts_to_volume_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_inserts_to_volume_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_do_not_work', '1')", 0), - ("mt_test_inserts_to_volume_work", "MergeTree()", 1), - ("replicated_mt_test_inserts_to_volume_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1), + pytest.param("mt_test_inserts_to_volume_do_not_work", "MergeTree()", 0, id="mt_test_inserts_to_volume_do_not_work"), + pytest.param("replicated_mt_test_inserts_to_volume_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_do_not_work', '1')", 0, id="replicated_mt_test_inserts_to_volume_do_not_work"), + pytest.param("mt_test_inserts_to_volume_work", "MergeTree()", 1, id="mt_test_inserts_to_volume_work"), + pytest.param("replicated_mt_test_inserts_to_volume_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_inserts_to_volume_work', '1')", 1, id="replicated_mt_test_inserts_to_volume_work"), ]) def test_inserts_to_volume_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -366,9 +366,9 @@ def test_inserts_to_volume_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_moves_to_disk_eventually_work", "MergeTree()"), - ("replicated_mt_test_moves_to_disk_eventually_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')"), + pytest.param("mt_test_moves_to_disk_eventually_work", "MergeTree()", id="mt_test_moves_to_disk_eventually_work"), + pytest.param("replicated_mt_test_moves_to_disk_eventually_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_to_disk_eventually_work', '1')", id="replicated_mt_test_moves_to_disk_eventually_work"), ]) def test_moves_to_disk_eventually_work(started_cluster, name, engine): name = unique_table_name(name) @@ -460,12 +460,12 @@ def test_replicated_download_ttl_info(started_cluster): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_merges_to_disk_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_merges_to_disk_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_do_not_work', '1')", 0), - ("mt_test_merges_to_disk_work", "MergeTree()", 1), - ("replicated_mt_test_merges_to_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1), + pytest.param("mt_test_merges_to_disk_do_not_work", "MergeTree()", 0, id="mt_test_merges_to_disk_do_not_work"), + pytest.param("replicated_mt_test_merges_to_disk_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_do_not_work', '1')", 0, id="mt_test_merges_to_disk_do_not_work"), + pytest.param("mt_test_merges_to_disk_work", "MergeTree()", 1, id="mt_test_merges_to_disk_work"), + pytest.param("replicated_mt_test_merges_to_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_to_disk_work', '1')", 1, id="replicated_mt_test_merges_to_disk_work"), ]) def test_merges_to_disk_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -524,9 +524,9 @@ def test_merges_to_disk_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine", [ - ("mt_test_merges_with_full_disk_work", "MergeTree()"), - ("replicated_mt_test_merges_with_full_disk_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')"), + pytest.param("mt_test_merges_with_full_disk_work", "MergeTree()", id="mt_test_merges_with_full_disk_work"), + pytest.param("replicated_mt_test_merges_with_full_disk_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_merges_with_full_disk_work', '1')", id="replicated_mt_test_merges_with_full_disk_work"), ]) def test_merges_with_full_disk_work(started_cluster, name, engine): name = unique_table_name(name) @@ -596,12 +596,12 @@ def test_merges_with_full_disk_work(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_moves_after_merges_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_moves_after_merges_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')", 0), - ("mt_test_moves_after_merges_work", "MergeTree()", 1), - ("replicated_mt_test_moves_after_merges_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1), + pytest.param("mt_test_moves_after_merges_do_not_work", "MergeTree()", 0, id="mt_test_moves_after_merges_do_not_work"), + pytest.param("replicated_mt_test_moves_after_merges_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_do_not_work', '1')", 0, id="replicated_mt_test_moves_after_merges_do_not_work"), + pytest.param("mt_test_moves_after_merges_work", "MergeTree()", 1, id="mt_test_moves_after_merges_work"), + pytest.param("replicated_mt_test_moves_after_merges_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_merges_work', '1')", 1, id="replicated_mt_test_moves_after_merges_work"), ]) def test_moves_after_merges_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -654,18 +654,18 @@ def test_moves_after_merges_work(started_cluster, name, engine, positive): @pytest.mark.parametrize("name,engine,positive,bar", [ - ("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "DELETE"), - ("replicated_mt_test_moves_after_alter_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "DELETE"), - ("mt_test_moves_after_alter_work", "MergeTree()", 1, "DELETE"), - ("replicated_mt_test_moves_after_alter_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "DELETE"), - ("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "TO DISK 'external'"), - ("replicated_mt_test_moves_after_alter_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "TO DISK 'external'"), - ("mt_test_moves_after_alter_work", "MergeTree()", 1, "TO DISK 'external'"), - ("replicated_mt_test_moves_after_alter_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'"), + pytest.param("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "DELETE", id="mt_negative"), + pytest.param("replicated_mt_test_moves_after_alter_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "DELETE", id="repicated_negative"), + pytest.param("mt_test_moves_after_alter_work", "MergeTree()", 1, "DELETE", id="mt_positive"), + pytest.param("replicated_mt_test_moves_after_alter_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "DELETE", id="repicated_positive"), + pytest.param("mt_test_moves_after_alter_do_not_work", "MergeTree()", 0, "TO DISK 'external'", id="mt_external_negative"), + pytest.param("replicated_mt_test_moves_after_alter_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_do_not_work', '1')", 0, "TO DISK 'external'", id="replicated_external_negative"), + pytest.param("mt_test_moves_after_alter_work", "MergeTree()", 1, "TO DISK 'external'", id="mt_external_positive"), + pytest.param("replicated_mt_test_moves_after_alter_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_moves_after_alter_work', '1')", 1, "TO DISK 'external'", id="replicated_external_positive"), ]) def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, bar): name = unique_table_name(name) @@ -704,9 +704,9 @@ def test_ttls_do_not_work_after_alter(started_cluster, name, engine, positive, b @pytest.mark.parametrize("name,engine", [ - ("mt_test_materialize_ttl_in_partition", "MergeTree()"), - ("replicated_mt_test_materialize_ttl_in_partition", - "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')"), + pytest.param("mt_test_materialize_ttl_in_partition", "MergeTree()", id="mt"), + pytest.param("replicated_mt_test_materialize_ttl_in_partition", + "ReplicatedMergeTree('/clickhouse/test_materialize_ttl_in_partition', '1')", id="replicated"), ]) def test_materialize_ttl_in_partition(started_cluster, name, engine): name = unique_table_name(name) @@ -769,12 +769,12 @@ def test_materialize_ttl_in_partition(started_cluster, name, engine): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_alter_multiple_ttls_positive", "MergeTree()", True), - ("mt_replicated_test_alter_multiple_ttls_positive", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_positive', '1')", True), - ("mt_test_alter_multiple_ttls_negative", "MergeTree()", False), - ("mt_replicated_test_alter_multiple_ttls_negative", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False), + pytest.param("mt_test_alter_multiple_ttls_positive", "MergeTree()", True, id="positive"), + pytest.param("mt_replicated_test_alter_multiple_ttls_positive", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_positive', '1')", True, id="replicated_positive"), + pytest.param("mt_test_alter_multiple_ttls_negative", "MergeTree()", False, id="negative"), + pytest.param("mt_replicated_test_alter_multiple_ttls_negative", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_multiple_ttls_negative', '1')", False, id="replicated_negative"), ]) def test_alter_multiple_ttls(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -868,9 +868,9 @@ limitations under the License.""" @pytest.mark.parametrize("name,engine", [ - ("concurrently_altering_ttl_mt", "MergeTree()"), - ("concurrently_altering_ttl_replicated_mt", - "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')",), + pytest.param("concurrently_altering_ttl_mt", "MergeTree()", id="mt"), + pytest.param("concurrently_altering_ttl_replicated_mt", + "ReplicatedMergeTree('/clickhouse/concurrently_altering_ttl_replicated_mt', '1')", id="replicated_mt"), ]) def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): name = unique_table_name(name) @@ -977,8 +977,8 @@ def test_concurrent_alter_with_ttl_move(started_cluster, name, engine): @pytest.mark.skip(reason="Flacky test") @pytest.mark.parametrize("name,positive", [ - ("test_double_move_while_select_negative", 0), - ("test_double_move_while_select_positive", 1), + pytest.param("test_double_move_while_select_negative", 0, id="negative"), + pytest.param("test_double_move_while_select_positive", 1, id="positive"), ]) def test_double_move_while_select(started_cluster, name, positive): name = unique_table_name(name) @@ -1038,12 +1038,12 @@ def test_double_move_while_select(started_cluster, name, positive): @pytest.mark.parametrize("name,engine,positive", [ - ("mt_test_alter_with_merge_do_not_work", "MergeTree()", 0), - ("replicated_mt_test_alter_with_merge_do_not_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_do_not_work', '1')", 0), - ("mt_test_alter_with_merge_work", "MergeTree()", 1), - ("replicated_mt_test_alter_with_merge_work", - "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1), + pytest.param("mt_test_alter_with_merge_do_not_work", "MergeTree()", 0, id="mt"), + pytest.param("replicated_mt_test_alter_with_merge_do_not_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_do_not_work', '1')", 0, id="replicated"), + pytest.param("mt_test_alter_with_merge_work", "MergeTree()", 1, id="mt_work"), + pytest.param("replicated_mt_test_alter_with_merge_work", + "ReplicatedMergeTree('/clickhouse/replicated_test_alter_with_merge_work', '1')", 1, id="replicated_work"), ]) def test_alter_with_merge_work(started_cluster, name, engine, positive): name = unique_table_name(name) @@ -1131,10 +1131,10 @@ limitations under the License.""" @pytest.mark.parametrize("name,dest_type,engine", [ - ("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()"), - ("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()"), - ("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), - ("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')"), + pytest.param("mt_test_disabled_ttl_move_on_insert_work", "DISK", "MergeTree()", id="disk"), + pytest.param("mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "MergeTree()", id="volume"), + pytest.param("replicated_mt_test_disabled_ttl_move_on_insert_work", "DISK", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')", id="replicated_disk"), + pytest.param("replicated_mt_test_disabled_ttl_move_on_insert_work", "VOLUME", "ReplicatedMergeTree('/clickhouse/replicated_test_disabled_ttl_move_on_insert_work', '1')", id="replicated_volume"), ]) def test_disabled_ttl_move_on_insert(started_cluster, name, dest_type, engine): name = unique_table_name(name) diff --git a/tests/integration/test_ttl_replicated/test.py b/tests/integration/test_ttl_replicated/test.py index 389e249790f..a4936aee624 100644 --- a/tests/integration/test_ttl_replicated/test.py +++ b/tests/integration/test_ttl_replicated/test.py @@ -227,8 +227,8 @@ def optimize_with_retry(node, table_name, retry=20): time.sleep(0.5) @pytest.mark.parametrize("name,engine", [ - ("test_ttl_alter_delete", "MergeTree()"), - ("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')"), + pytest.param("test_ttl_alter_delete", "MergeTree()", id="test_ttl_alter_delete"), + pytest.param("test_replicated_ttl_alter_delete", "ReplicatedMergeTree('/clickhouse/test_replicated_ttl_alter_delete', '1')", id="test_ttl_alter_delete_replicated"), ]) def test_ttl_alter_delete(started_cluster, name, engine): """Copyright 2019, Altinity LTD From dbf04c2a815927a1e98a378903b45e1770b78d1a Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 14:50:17 +0300 Subject: [PATCH 0069/1060] fixes --- tests/integration/test_mysql_protocol/test.py | 17 ++++++++--------- tests/integration/test_odbc_interaction/test.py | 4 ++-- 2 files changed, 10 insertions(+), 11 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 1e0d3652a91..5b9c180b25e 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -20,8 +20,7 @@ node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "confi "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) -server_port = 9001 - +server_port = cluster.mysql_port @pytest.fixture(scope="module") def server_address(): @@ -64,31 +63,31 @@ def golang_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_golang_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_golang1_1') @pytest.fixture(scope='module') def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') + ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') @pytest.fixture(scope='module') def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') + ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') @pytest.fixture(scope='module') def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( - ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') + ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') def test_mysql_client(mysql_client, server_address): diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index 3b860ff7cee..cbd8dd827bf 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -40,10 +40,10 @@ def get_mysql_conn(): for _ in range(15): try: if conn is None: - conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=cluster.mysql_port) + conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_ip, port=cluster.mysql_port) else: conn.ping(reconnect=True) - logging.debug("MySQL Connection establised: 127.0.0.1:{}".format(cluster.mysql_port)) + logging.debug(f"MySQL Connection establised: {cluster.mysql_ip}:{cluster.mysql_port}") return conn except Exception as e: errors += [str(e)] From 0b9bdfc5c599798a5ffd4bd0acda3d48c78c9504 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 18:21:39 +0300 Subject: [PATCH 0070/1060] fix --- .../materialize_with_ddl.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py index 4740f14bcbf..c9cf93554c8 100644 --- a/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialize_mysql_database/materialize_with_ddl.py @@ -12,7 +12,7 @@ import threading from multiprocessing.dummy import Pool from helpers.test_tools import assert_eq_with_retry -def check_query(clickhouse_node, query, result_set, retry_count=1, interval_seconds=3): +def check_query(clickhouse_node, query, result_set, retry_count=10, interval_seconds=3): lastest_result = '' for i in range(retry_count): @@ -694,9 +694,10 @@ def mysql_kill_sync_thread_restore_test(clickhouse_node, mysql_node, service_nam def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): mysql_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") + clickhouse_node.query("DROP DATABASE IF EXISTS kill_mysql_while_insert") mysql_node.query("CREATE DATABASE kill_mysql_while_insert") mysql_node.query("CREATE TABLE kill_mysql_while_insert.test ( `id` int(11) NOT NULL, PRIMARY KEY (`id`) ) ENGINE=InnoDB;") - clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse')".format(service_name)) + clickhouse_node.query("CREATE DATABASE kill_mysql_while_insert ENGINE = MaterializeMySQL('{}:3306', 'kill_mysql_while_insert', 'root', 'clickhouse') SETTINGS max_wait_time_when_mysql_unavailable=-1".format(service_name)) check_query(clickhouse_node, "SHOW TABLES FROM kill_mysql_while_insert FORMAT TSV", 'test\n') try: @@ -711,7 +712,7 @@ def mysql_killed_while_insert(clickhouse_node, mysql_node, service_name): clickhouse_node.cluster.restart_service(service_name) finally: with pytest.raises(QueryRuntimeException) as exception: - time.sleep(5) + time.sleep(2) clickhouse_node.query("SELECT count() FROM kill_mysql_while_insert.test") mysql_node.alloc_connection() From f14d4be88efa37a235c08940f0c1a4d5ac7d5be6 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 19:44:08 +0300 Subject: [PATCH 0071/1060] change postgres --- tests/integration/helpers/cluster.py | 13 ++++++++++--- .../test_dictionaries_postgresql/test.py | 18 +++++++++--------- .../integration/test_odbc_interaction/test.py | 14 +++++++------- .../test_postgresql_database_engine/test.py | 18 +++++++++--------- .../test_postgresql_protocol/test.py | 4 ++-- .../test_storage_postgresql/test.py | 16 ++++++++-------- 6 files changed, 45 insertions(+), 38 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 3a4476696ed..d0e999049ea 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -302,8 +302,8 @@ class ClickHouseCluster: self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) env_variables['POSTGRES_HOST'] = self.postgres_host env_variables['POSTGRES_PORT'] = str(self.postgres_port) - env_variables['POSTGRES_LOGS'] = self.postgres_logs_dir - env_variables['POSTGRES2_LOGS'] = self.postgres2_logs_dir + env_variables['POSTGRES_DIR'] = self.postgres_logs_dir + env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir env_variables['POSTGRES_LOGS_FS'] = "bind" self.with_postgres = True @@ -684,7 +684,7 @@ class ClickHouseCluster: self.postgres_ip = self.get_instance_ip(self.postgres_host) self.postgres2_ip = self.get_instance_ip(self.postgres2_host) start = time.time() - for up in [self.postgres_ip, self.postgres2_ip]: + for ip in [self.postgres_ip, self.postgres2_ip]: while time.time() - start < timeout: try: conn = psycopg2.connect(host=ip, port=self.postgres_port, user='postgres', password='mysecretpassword') @@ -908,6 +908,13 @@ class ClickHouseCluster: if self.with_postgres and self.base_postgres_cmd: logging.debug('Setup Postgres') + if os.path.exists(self.postgres_dir): + shutil.rmtree(self.postgres_dir) + os.makedirs(self.postgres_logs_dir) + os.chmod(self.postgres_logs_dir, stat.S_IRWXO) + os.makedirs(self.postgres2_logs_dir) + os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) + subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(30) self.wait_postgres_to_start(30) diff --git a/tests/integration/test_dictionaries_postgresql/test.py b/tests/integration/test_dictionaries_postgresql/test.py index e5a4af1d91a..86763b52a6f 100644 --- a/tests/integration/test_dictionaries_postgresql/test.py +++ b/tests/integration/test_dictionaries_postgresql/test.py @@ -21,11 +21,11 @@ click_dict_table_template = """ ) ENGINE = Dictionary({}) """ -def get_postgres_conn(port, database=False): +def get_postgres_conn(ip, port, database=False): if database == True: - conn_string = "host='localhost' port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(port) + conn_string = "host={} port={} dbname='clickhouse' user='postgres' password='mysecretpassword'".format(ip, port) else: - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(port) + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(ip, port) conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) @@ -73,10 +73,10 @@ def started_cluster(): def test_load_dictionaries(started_cluster): - conn = get_postgres_conn(database=True, port=started_cluster.postgres_port) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(cursor, table_name) + create_and_fill_postgres_table(cursor, table_name, port=started_cluster.postgres_port, host=started_cluster.postgres_ip) create_dict(table_name) dict_name = 'dict0' @@ -91,10 +91,10 @@ def test_load_dictionaries(started_cluster): def test_invalidate_query(started_cluster): - conn = get_postgres_conn(database=True, port=started_cluster.postgres_port) + conn = get_postgres_conn(ip=started_cluster.postgres_ip, database=True, port=started_cluster.postgres_port) cursor = conn.cursor() table_name = 'test0' - create_and_fill_postgres_table(cursor, table_name) + create_and_fill_postgres_table(cursor, table_name, port=started_cluster.postgres_port, host=started_cluster.postgres_ip) # invalidate query: SELECT value FROM test0 WHERE id = 0 dict_name = 'dict0' @@ -128,9 +128,9 @@ def test_invalidate_query(started_cluster): def test_dictionary_with_replicas(started_cluster): - conn1 = get_postgres_conn(port=5432, database=True) + conn1 = get_postgres_conn(ip=started_cluster.postgres_ip, port=started_cluster.postgres_port, database=True) cursor1 = conn1.cursor() - conn2 = get_postgres_conn(port=5441, database=True) + conn2 = get_postgres_conn(ip=started_cluster.postgres2_ip, port=started_cluster.postgres_port, database=True) cursor2 = conn2.cursor() create_postgres_table(cursor1, 'test1') diff --git a/tests/integration/test_odbc_interaction/test.py b/tests/integration/test_odbc_interaction/test.py index cbd8dd827bf..da6abcc2751 100644 --- a/tests/integration/test_odbc_interaction/test.py +++ b/tests/integration/test_odbc_interaction/test.py @@ -64,8 +64,8 @@ def create_mysql_table(conn, table_name): cursor.execute(create_table_sql_template.format(table_name)) -def get_postgres_conn(): - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) +def get_postgres_conn(started_cluster): + conn_string = "host={} port={} user='postgres' password='mysecretpassword'".format(started_cluster.postgres_ip, started_cluster.postgres_port) errors = [] for _ in range(15): try: @@ -112,7 +112,7 @@ def started_cluster(): create_mysql_db(mysql_conn, 'clickhouse') print("mysql database created") - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(cluster) print("postgres connection received") create_postgres_db(postgres_conn, 'clickhouse') @@ -294,7 +294,7 @@ def test_sqlite_odbc_cached_dictionary(started_cluster): def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(1, 'hello'),(2, 'world')") @@ -304,7 +304,7 @@ def test_postgres_odbc_hashed_dictionary_with_schema(started_cluster): def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) cursor = conn.cursor() cursor.execute("truncate table clickhouse.test_table") cursor.execute("insert into clickhouse.test_table values(3, 'xxx')") @@ -318,7 +318,7 @@ def test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow(started_cluster): def test_postgres_insert(started_cluster): - conn = get_postgres_conn() + conn = get_postgres_conn(started_cluster) conn.cursor().execute("truncate table clickhouse.test_table") # Also test with Servername containing '.' and '-' symbols (defined in @@ -377,7 +377,7 @@ def test_bridge_dies_with_parent(started_cluster): def test_odbc_postgres_date_data_type(started_cluster): - conn = get_postgres_conn(); + conn = get_postgres_conn(started_cluster); cursor = conn.cursor() cursor.execute("CREATE TABLE IF NOT EXISTS clickhouse.test_date (column1 integer, column2 date)") diff --git a/tests/integration/test_postgresql_database_engine/test.py b/tests/integration/test_postgresql_database_engine/test.py index 2cb148db6ef..e89f1109c3a 100644 --- a/tests/integration/test_postgresql_database_engine/test.py +++ b/tests/integration/test_postgresql_database_engine/test.py @@ -14,11 +14,11 @@ postgres_table_template = """ id Integer NOT NULL, value Integer, PRIMARY KEY (id)) """ -def get_postgres_conn(database=False): +def get_postgres_conn(cluster, database=False): if database == True: - conn_string = "host='localhost' port={} dbname='test_database' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port={cluster.postgres_port} dbname='test_database' user='postgres' password='mysecretpassword'" else: - conn_string = "host='localhost' port={} user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port={cluster.postgres_port} user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -36,7 +36,7 @@ def create_postgres_table(cursor, table_name): def started_cluster(): try: cluster.start() - conn = get_postgres_conn() + conn = get_postgres_conn(cluster) cursor = conn.cursor() create_postgres_db(cursor, 'test_database') yield cluster @@ -47,7 +47,7 @@ def started_cluster(): def test_postgres_database_engine_with_postgres_ddl(started_cluster): # connect to database as well - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -68,7 +68,7 @@ def test_postgres_database_engine_with_postgres_ddl(started_cluster): def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -94,7 +94,7 @@ def test_postgresql_database_engine_with_clickhouse_ddl(started_cluster): def test_postgresql_database_engine_queries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -114,7 +114,7 @@ def test_postgresql_database_engine_queries(started_cluster): def test_get_create_table_query_with_multidim_arrays(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( @@ -147,7 +147,7 @@ def test_get_create_table_query_with_multidim_arrays(started_cluster): def test_postgresql_database_engine_table_cache(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query( diff --git a/tests/integration/test_postgresql_protocol/test.py b/tests/integration/test_postgresql_protocol/test.py index 60cf8a9239b..39ccb273299 100644 --- a/tests/integration/test_postgresql_protocol/test.py +++ b/tests/integration/test_postgresql_protocol/test.py @@ -41,7 +41,7 @@ def psql_client(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=os.environ.get("DOCKER_API_VERSION"), timeout=180).containers.get(cluster.project_name + '_psql_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_psql_1') @pytest.fixture(scope='module') @@ -65,7 +65,7 @@ def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_postgresql_java_client.yml') run_and_check( ['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--build']) - yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=self.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') + yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java_1') def test_psql_is_ready(psql_server): diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index d92a32062fc..5215d8f55ba 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -9,11 +9,11 @@ from psycopg2.extensions import ISOLATION_LEVEL_AUTOCOMMIT cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', main_configs=[], with_postgres=True) -def get_postgres_conn(database=False): +def get_postgres_conn(cluster, database=False): if database == True: - conn_string = "host='localhost' port='{}' dbname='clickhouse' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port='{cluster.postgres_port}' dbname='clickhouse' user='postgres' password='mysecretpassword'" else: - conn_string = "host='localhost' port='{}' user='postgres' password='mysecretpassword'".format(cluster.postgres_port) + conn_string = f"host={cluster.postgres_ip} port='{cluster.postgres_port}' user='postgres' password='mysecretpassword'" conn = psycopg2.connect(conn_string) conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) conn.autocommit = True @@ -29,7 +29,7 @@ def create_postgres_db(conn, name): def started_cluster(): try: cluster.start() - postgres_conn = get_postgres_conn() + postgres_conn = get_postgres_conn(cluster) print("postgres connected") create_postgres_db(postgres_conn, 'clickhouse') yield cluster @@ -39,10 +39,10 @@ def started_cluster(): def test_postgres_select_insert(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() table_name = 'test_many' - table = '''postgresql('postgres1:5432', 'clickhouse', '{}', 'postgres', 'mysecretpassword')'''.format(table_name) + table = f'''postgresql('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'clickhouse', '{table_name}', 'postgres', 'mysecretpassword')''' cursor.execute('CREATE TABLE IF NOT EXISTS {} (a integer, b text, c integer)'.format(table_name)) result = node1.query(''' @@ -59,7 +59,7 @@ def test_postgres_select_insert(started_cluster): def test_postgres_conversions(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( @@ -134,7 +134,7 @@ def test_postgres_conversions(started_cluster): def test_non_default_scema(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') cursor.execute('CREATE TABLE test_schema.test_table (a integer)') From fd32609f380ef422a54c686ff9bbd468a11d74bf Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 19 Mar 2021 19:47:59 +0300 Subject: [PATCH 0072/1060] run parallel tests --- tests/integration/ci-runner.py | 27 +++++-- tests/integration/parallel.txt | 125 +++++++++++++++++++++++++++++++++ 2 files changed, 148 insertions(+), 4 deletions(-) create mode 100644 tests/integration/parallel.txt diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index f758457ada0..fb1d85a2a3f 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -257,6 +257,17 @@ class ClickhouseIntegrationTestsRunner: all_tests.append(line.strip()) return list(sorted(all_tests)) + def _get_parallel_tests(self, repo_path): + parallel_tests_file_path = "{}/tests/integration/all_tests.txt".format(repo_path) + if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: + raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) + + parallel_tests = [] + with open(parallel_tests_file_path, "r") as parallel_tests_file: + for line in parallel_tests_file: + all_tests.append(line.strip()) + return list(sorted(parallel_tests)) + def group_test_by_file(self, tests): result = {} for test in tests: @@ -326,7 +337,7 @@ class ClickhouseIntegrationTestsRunner: test_names.add(test_name) test_cmd = ' '.join([test for test in sorted(test_names)]) - cmd = "cd {}/tests/integration && ./runner {} '-ss {} -rfEp --color=no --durations=0 {}' | tee {}".format( + cmd = "cd {}/tests/integration && ./runner {} -t {} --parallel 10 '-ss -rfEp --color=no --durations=0 {}' | tee {}".format( repo_path, image_cmd, test_cmd, _get_deselect_option(self.should_skip_tests()), output_path) with open(log_path, 'w') as log: @@ -425,7 +436,13 @@ class ClickhouseIntegrationTestsRunner: logging.info("Dump iptables before run %s", subprocess.check_output("iptables -L", shell=True)) all_tests = self._get_all_tests(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) - grouped_tests = self.group_test_by_file(all_tests) + filtered_parallel_tests = filter(lambda test: test in all_tests, parallel_tests) + filtered_unparallel_tests = filter(lambda test: test not in parallel_tests, all_tests) + not_found_tests = filter(lambda test: test not in all_tests, parallel_tests) + logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_unparallel_tests)) + logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) + + grouped_tests = self.group_test_by_file(filtered_unparallel_tests) logging.info("Found %s tests groups", len(grouped_tests)) counters = { @@ -436,7 +453,9 @@ class ClickhouseIntegrationTestsRunner: tests_times = defaultdict(float) logs = [] - items_to_run = list(grouped_tests.items()) + items_to_run = list() + items_to_run += list(("parallel", filtered_parallel_tests)) + items_to_run += list(grouped_tests.items()) logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): @@ -444,7 +463,7 @@ class ClickhouseIntegrationTestsRunner: random.shuffle(items_to_run) for group, tests in items_to_run: - logging.info("Running test group %s countaining %s tests", group, len(tests)) + logging.info("Running test group %s containing %s tests", group, len(tests)) group_counters, group_test_times, log_name, log_path = self.run_test_group(repo_path, group, tests, MAX_RETRY) total_tests = 0 for counter, value in group_counters.items(): diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt new file mode 100644 index 00000000000..fac2f589358 --- /dev/null +++ b/tests/integration/parallel.txt @@ -0,0 +1,125 @@ +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[cache] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[direct] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[flat] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_simple[hashed] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_cache] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_direct] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_complex[complex_key_hashed] +test_dictionaries_all_layouts_separate_sources/test_mysql.py::test_ranged[range_hashed] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_hashed] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed] +test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache] +test_dictionaries_mysql/test.py::test_load_mysql_dictionaries +test_disabled_mysql_server/test.py::test_disabled_mysql_server +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_8_0[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_mysql_8_0[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_ddl_with_empty_transaction_8_0[atomic] +test_materialize_mysql_database/test.py::test_select_without_columns_5_7[ordinary] +test_materialize_mysql_database/test.py::test_select_without_columns_5_7[atomic] +test_materialize_mysql_database/test.py::test_select_without_columns_8_0[ordinary] +test_materialize_mysql_database/test.py::test_select_without_columns_8_0[atomic] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[ordinary] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_5_7[atomic] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[ordinary] +test_materialize_mysql_database/test.py::test_insert_with_modify_binlog_checksum_8_0[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_5_7[atomic] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[ordinary] +test_materialize_mysql_database/test.py::test_materialize_database_err_sync_user_privs_8_0[atomic] +test_materialize_mysql_database/test.py::test_network_partition_5_7[ordinary] +test_materialize_mysql_database/test.py::test_network_partition_5_7[atomic] +test_materialize_mysql_database/test.py::test_network_partition_8_0[ordinary] +test_materialize_mysql_database/test.py::test_network_partition_8_0[atomic] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[ordinary] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_5_7[atomic] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[ordinary] +test_materialize_mysql_database/test.py::test_mysql_kill_sync_thread_restore_8_0[atomic] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[ordinary] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_5_7[atomic] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[ordinary] +test_materialize_mysql_database/test.py::test_mysql_killed_while_insert_8_0[atomic] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[ordinary] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_5_7[atomic] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[ordinary] +test_materialize_mysql_database/test.py::test_clickhouse_killed_while_insert_8_0[atomic] +test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node0] +test_materialize_mysql_database/test.py::test_utf8mb4[clickhouse_node1] +test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node0] +test_materialize_mysql_database/test.py::test_system_parts_table[clickhouse_node1] +test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node0] +test_materialize_mysql_database/test.py::test_multi_table_update[clickhouse_node1] +test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node0] +test_materialize_mysql_database/test.py::test_system_tables_table[clickhouse_node1] +test_mysql_database_engine/test.py::test_mysql_ddl_for_mysql_database +test_mysql_database_engine/test.py::test_clickhouse_ddl_for_mysql_database +test_mysql_database_engine/test.py::test_clickhouse_dml_for_mysql_database +test_mysql_database_engine/test.py::test_clickhouse_join_for_mysql_database +test_mysql_database_engine/test.py::test_bad_arguments_for_mysql_database_engine +test_mysql_database_engine/test.py::test_data_types_support_level_for_mysql_database_engine +test_mysql_database_engine/test.py::test_mysql_types[common_types_1] +test_mysql_database_engine/test.py::test_mysql_types[common_types_2] +test_mysql_database_engine/test.py::test_mysql_types[common_types_3] +test_mysql_database_engine/test.py::test_mysql_types[common_types_4] +test_mysql_database_engine/test.py::test_mysql_types[common_types_5] +test_mysql_database_engine/test.py::test_mysql_types[common_types_6] +test_mysql_database_engine/test.py::test_mysql_types[common_types_7] +test_mysql_database_engine/test.py::test_mysql_types[common_types_80] +test_mysql_database_engine/test.py::test_mysql_types[common_types_81] +test_mysql_database_engine/test.py::test_mysql_types[common_types_9] +test_mysql_database_engine/test.py::test_mysql_types[common_types_10] +test_mysql_database_engine/test.py::test_mysql_types[common_types_11] +test_mysql_database_engine/test.py::test_mysql_types[common_types_12] +test_mysql_database_engine/test.py::test_mysql_types[common_types_13] +test_mysql_database_engine/test.py::test_mysql_types[common_types_14] +test_mysql_database_engine/test.py::test_mysql_types[common_types_15] +test_mysql_database_engine/test.py::test_mysql_types[common_types_16] +test_mysql_database_engine/test.py::test_mysql_types[common_types_17] +test_mysql_database_engine/test.py::test_mysql_types[common_types_18] +test_mysql_database_engine/test.py::test_mysql_types[common_types_19] +test_mysql_database_engine/test.py::test_mysql_types[common_types_20] +test_mysql_database_engine/test.py::test_mysql_types[decimal_1] +test_mysql_database_engine/test.py::test_mysql_types[decimal_2] +test_mysql_database_engine/test.py::test_mysql_types[decimal_3] +test_mysql_database_engine/test.py::test_mysql_types[decimal_4] +test_mysql_database_engine/test.py::test_mysql_types[timestamp_default] +test_mysql_database_engine/test.py::test_mysql_types[timestamp_6] +test_mysql_database_engine/test.py::test_mysql_types[datetime_default] +test_mysql_database_engine/test.py::test_mysql_types[datetime_6_1] +test_mysql_database_engine/test.py::test_mysql_types[decimal_40_6] +test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_1] +test_mysql_database_engine/test.py::test_mysql_types[decimal_18_6_2] +test_mysql_database_engine/test.py::test_mysql_types[datetime_6_2] +test_mysql_database_engine/test.py::test_mysql_types[datetime_6_3] +test_mysql_protocol/test.py::test_mysql_client +test_mysql_protocol/test.py::test_mysql_client_exception +test_mysql_protocol/test.py::test_mysql_affected_rows +test_mysql_protocol/test.py::test_mysql_replacement_query +test_mysql_protocol/test.py::test_mysql_explain +test_mysql_protocol/test.py::test_mysql_federated +test_mysql_protocol/test.py::test_mysql_set_variables +test_mysql_protocol/test.py::test_python_client +test_mysql_protocol/test.py::test_golang_client +test_mysql_protocol/test.py::test_php_client +test_mysql_protocol/test.py::test_mysqljs_client +test_mysql_protocol/test.py::test_java_client +test_mysql_protocol/test.py::test_types +test_odbc_interaction/test.py::test_mysql_simple_select_works +test_odbc_interaction/test.py::test_mysql_insert +test_storage_mysql/test.py::test_many_connections +test_storage_mysql/test.py::test_insert_select +test_storage_mysql/test.py::test_replace_select +test_storage_mysql/test.py::test_insert_on_duplicate_select +test_storage_mysql/test.py::test_where +test_storage_mysql/test.py::test_table_function +test_storage_mysql/test.py::test_binary_type +test_storage_mysql/test.py::test_enum_type From 880a03a9dfd3494f68b69eac835d6060cd8e50e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sat, 20 Mar 2021 22:44:58 +0300 Subject: [PATCH 0073/1060] fix --- tests/integration/ci-runner.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index fb1d85a2a3f..139d73a9d34 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -435,6 +435,7 @@ class ClickhouseIntegrationTestsRunner: self._install_clickhouse(build_path) logging.info("Dump iptables before run %s", subprocess.check_output("iptables -L", shell=True)) all_tests = self._get_all_tests(repo_path) + parallel_tests = self._get_parallel_tests(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) filtered_parallel_tests = filter(lambda test: test in all_tests, parallel_tests) filtered_unparallel_tests = filter(lambda test: test not in parallel_tests, all_tests) From 360887893b48caaff1e208d235c9f25cf1fa5b8c Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Mar 2021 14:51:47 +0300 Subject: [PATCH 0074/1060] fix --- tests/integration/ci-runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index c2152cc6759..9633d4294ef 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -271,7 +271,7 @@ class ClickhouseIntegrationTestsRunner: parallel_tests = [] with open(parallel_tests_file_path, "r") as parallel_tests_file: for line in parallel_tests_file: - all_tests.append(line.strip()) + parallel_tests.append(line.strip()) return list(sorted(parallel_tests)) def group_test_by_file(self, tests): From ee84ab2a4805cb7ca1a97041db5ff8e578647ed9 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 22 Mar 2021 18:33:46 +0300 Subject: [PATCH 0075/1060] fixes --- tests/integration/ci-runner.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 9633d4294ef..93ca0435720 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -468,9 +468,9 @@ class ClickhouseIntegrationTestsRunner: all_tests = self._get_all_tests(repo_path) parallel_tests = self._get_parallel_tests(repo_path) logging.info("Found %s tests first 3 %s", len(all_tests), ' '.join(all_tests[:3])) - filtered_parallel_tests = filter(lambda test: test in all_tests, parallel_tests) - filtered_unparallel_tests = filter(lambda test: test not in parallel_tests, all_tests) - not_found_tests = filter(lambda test: test not in all_tests, parallel_tests) + filtered_parallel_tests = list(filter(lambda test: test in all_tests, parallel_tests)) + filtered_unparallel_tests = list(filter(lambda test: test not in parallel_tests, all_tests)) + not_found_tests = list(filter(lambda test: test not in all_tests, parallel_tests)) logging.info("Found %s tests first 3 %s, parallel %s, other %s", len(all_tests), ' '.join(all_tests[:3]), len(filtered_parallel_tests), len(filtered_unparallel_tests)) logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) From ecc58a76d768d45cb84b30ad5c4aacca18567108 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Mar 2021 06:35:08 +0300 Subject: [PATCH 0076/1060] fix --- tests/integration/ci-runner.py | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 93ca0435720..4196d97d2c6 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -264,7 +264,7 @@ class ClickhouseIntegrationTestsRunner: return list(sorted(all_tests)) def _get_parallel_tests(self, repo_path): - parallel_tests_file_path = "{}/tests/integration/all_tests.txt".format(repo_path) + parallel_tests_file_path = "{}/tests/integration/parallel.txt".format(repo_path) if not os.path.isfile(parallel_tests_file_path) or os.path.getsize(parallel_tests_file_path) == 0: raise Exception("There is something wrong with getting all tests list: file '{}' is empty or does not exist.".format(parallel_tests_file_path)) @@ -475,6 +475,7 @@ class ClickhouseIntegrationTestsRunner: logging.info("Not found %s tests first 3 %s", len(not_found_tests), ' '.join(not_found_tests[:3])) grouped_tests = self.group_test_by_file(filtered_unparallel_tests) + grouped_tests["parallel"] = filtered_parallel_tests logging.info("Found %s tests groups", len(grouped_tests)) counters = { @@ -487,9 +488,7 @@ class ClickhouseIntegrationTestsRunner: tests_times = defaultdict(float) logs = [] - items_to_run = list() - items_to_run += list(("parallel", filtered_parallel_tests)) - items_to_run += list(grouped_tests.items()) + items_to_run = list(grouped_tests.items()) logging.info("Total test groups %s", len(items_to_run)) if self.shuffle_test_groups(): From e79b5b57fdc2a936045fb0a1d448415435d5cba7 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Mar 2021 14:35:45 +0300 Subject: [PATCH 0077/1060] try export dockerd.log --- docker/test/integration/runner/dockerd-entrypoint.sh | 2 +- tests/integration/ci-runner.py | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index 9f04dde720d..a066e914241 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,7 +1,7 @@ #!/bin/bash set -e -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.0.0.0/8,size=24 &>/var/log/somefile & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.10.0.0/16,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index 4196d97d2c6..9a82516b756 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -15,6 +15,7 @@ MAX_RETRY = 2 SLEEP_BETWEEN_RETRIES = 5 CLICKHOUSE_BINARY_PATH = "/usr/bin/clickhouse" CLICKHOUSE_ODBC_BRIDGE_BINARY_PATH = "/usr/bin/clickhouse-odbc-bridge" +DOCKERD_LOGS_PATH = "/ClickHouse/tests/integration/dockerd.log" TRIES_COUNT = 10 MAX_TIME_SECONDS = 3600 @@ -446,6 +447,10 @@ class ClickhouseIntegrationTestsRunner: self._compress_logs("{}/tests/integration".format(repo_path), test_logs) logging.info("Compression finished") + result_path_dockerd_logs = os.path.join(str(self.path()), "dockerd.log") + if os.path.exists(result_path_dockerd_logs): + shutil.copy(DOCKERD_LOGS_PATH, result_path_dockerd_logs) + test_result = [] for state in ("ERROR", "FAILED", "PASSED", "SKIPPED", "FLAKY"): if state == "PASSED": @@ -518,6 +523,10 @@ class ClickhouseIntegrationTestsRunner: self._compress_logs("{}/tests/integration".format(repo_path), test_logs) logging.info("Compression finished") + result_path_dockerd_logs = os.path.join(str(self.path()), "dockerd.log") + if os.path.exists(result_path_dockerd_logs): + shutil.copy(DOCKERD_LOGS_PATH, result_path_dockerd_logs) + if counters["FAILED"] or counters["ERROR"]: logging.info("Overall status failure, because we have tests in FAILED or ERROR state") result_state = "failure" From af1c14b4ee6b25df2d73d4be446fae9cc1717e7d Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Tue, 23 Mar 2021 18:42:36 +0300 Subject: [PATCH 0078/1060] try other network --- docker/test/integration/runner/dockerd-entrypoint.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/integration/runner/dockerd-entrypoint.sh b/docker/test/integration/runner/dockerd-entrypoint.sh index a066e914241..037feb7ff59 100755 --- a/docker/test/integration/runner/dockerd-entrypoint.sh +++ b/docker/test/integration/runner/dockerd-entrypoint.sh @@ -1,7 +1,7 @@ #!/bin/bash set -e -dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=10.10.0.0/16,size=24 &>/ClickHouse/tests/integration/dockerd.log & +dockerd --host=unix:///var/run/docker.sock --host=tcp://0.0.0.0:2375 --default-address-pool base=172.17.0.0/12,size=24 &>/ClickHouse/tests/integration/dockerd.log & set +e reties=0 From 662d494d2051ae5caab134151ad31fd72fa214f3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Mar 2021 16:23:23 +0300 Subject: [PATCH 0079/1060] fixes --- tests/integration/test_mysql_protocol/test.py | 8 +++---- tests/integration/test_storage_mysql/test.py | 22 +++++++++---------- 2 files changed, 15 insertions(+), 15 deletions(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 5b9c180b25e..4a33dcc7e3a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -20,7 +20,7 @@ node = cluster.add_instance('node', main_configs=["configs/log_conf.xml", "confi "configs/dhparam.pem", "configs/server.crt", "configs/server.key"], user_configs=["configs/users.xml"], env_variables={'UBSAN_OPTIONS': 'print_stacktrace=1'}) -server_port = cluster.mysql_port +server_port = 9001 @pytest.fixture(scope="module") def server_address(): @@ -70,7 +70,7 @@ def golang_container(): def php_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_php_client.yml') run_and_check( - ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_php1_1') @@ -78,7 +78,7 @@ def php_container(): def nodejs_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_js_client.yml') run_and_check( - ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_mysqljs1_1') @@ -86,7 +86,7 @@ def nodejs_container(): def java_container(): docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_java_client.yml') run_and_check( - ['docker-compose', '--env-file', cluster.instances["mysql57"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) + ['docker-compose', '--env-file', cluster.instances["node"].env_file, '-p', cluster.project_name, '-f', docker_compose, 'up', '--no-recreate', '-d', '--no-build']) yield docker.DockerClient(base_url='unix:///var/run/docker.sock', version=cluster.docker_api_version, timeout=180).containers.get(cluster.project_name + '_java1_1') diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index aea6e0816d0..16d1b6048e6 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -24,7 +24,7 @@ def started_cluster(): try: cluster.start() - conn = get_mysql_conn() + conn = get_mysql_conn(cluster) ## create mysql db and table create_mysql_db(conn, 'clickhouse') yield cluster @@ -35,7 +35,7 @@ def started_cluster(): def test_many_connections(started_cluster): table_name = 'test_many_connections' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -54,7 +54,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_select(started_cluster): table_name = 'test_insert_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -70,7 +70,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_replace_select(started_cluster): table_name = 'test_replace_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -89,7 +89,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_on_duplicate_select(started_cluster): table_name = 'test_insert_on_duplicate_select' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' @@ -108,7 +108,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_where(started_cluster): table_name = 'test_where' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); @@ -128,7 +128,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, 'table_function') table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' @@ -149,7 +149,7 @@ def test_table_function(started_cluster): conn.close() def test_binary_type(started_cluster): - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') @@ -158,7 +158,7 @@ def test_binary_type(started_cluster): def test_enum_type(started_cluster): table_name = 'test_enum_type' - conn = get_mysql_conn() + conn = get_mysql_conn(started_cluster) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); @@ -168,8 +168,8 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8(' conn.close() -def get_mysql_conn(): - conn = pymysql.connect(user='root', password='clickhouse', host=cluster.mysql_host, port=cluster.mysql_port) +def get_mysql_conn(started_cluster): + conn = pymysql.connect(user='root', password='clickhouse', host=started_cluster.mysql_ip, port=started_cluster.mysql_port) return conn From 815a2ad7f2c2caa4f30b4d11c63e32dbb16fa6e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 24 Mar 2021 16:50:32 +0300 Subject: [PATCH 0080/1060] fix --- tests/integration/test_mysql_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 4a33dcc7e3a..26f44fd9491 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -217,7 +217,7 @@ def test_mysql_replacement_query(mysql_client, server_address): --password=123 -e "select database();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout.decode() == 'DATABASE()\ndefault\n' + assert stdout.decode() == 'database()\ndefault\n' code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default From 8b070495397d549e0a65da9480c24c9e002573b8 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 25 Mar 2021 07:38:24 +0300 Subject: [PATCH 0081/1060] undo --- tests/integration/test_mysql_protocol/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 26f44fd9491..4a33dcc7e3a 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -217,7 +217,7 @@ def test_mysql_replacement_query(mysql_client, server_address): --password=123 -e "select database();" '''.format(host=server_address, port=server_port), demux=True) assert code == 0 - assert stdout.decode() == 'database()\ndefault\n' + assert stdout.decode() == 'DATABASE()\ndefault\n' code, (stdout, stderr) = mysql_client.exec_run(''' mysql --protocol tcp -h {host} -P {port} default -u default From 3f9b81b0a7dc6c30786df3542465b52c74dd6278 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 26 Mar 2021 13:08:24 +0300 Subject: [PATCH 0082/1060] tune --- tests/integration/helpers/cluster.py | 4 ++-- tests/integration/test_multiple_disks/test.py | 2 +- tests/integration/test_storage_postgresql/test.py | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index c3c7710b366..0a64f8fdc7b 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -895,7 +895,7 @@ class ClickHouseCluster: os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(60) + self.wait_mysql_to_start(120) if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') @@ -904,7 +904,7 @@ class ClickHouseCluster: os.makedirs(self.mysql8_logs_dir) os.chmod(self.mysql8_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql8_cmd + common_opts) - self.wait_mysql8_to_start(120) + self.wait_mysql8_to_start(180) if self.with_postgres and self.base_postgres_cmd: logging.debug('Setup Postgres') diff --git a/tests/integration/test_multiple_disks/test.py b/tests/integration/test_multiple_disks/test.py index 1d672f8f7ce..7ad7fe92b0f 100644 --- a/tests/integration/test_multiple_disks/test.py +++ b/tests/integration/test_multiple_disks/test.py @@ -926,7 +926,7 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): tasks.append(p.apply_async(alter_drop, (100,))) for task in tasks: - task.get(timeout=60) + task.get(timeout=120) assert node1.query("SELECT 1") == "1\n" diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 8e3ec6c6e76..ceb2842697f 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -168,7 +168,7 @@ def test_non_default_scema(started_cluster): def test_concurrent_queries(started_cluster): - conn = get_postgres_conn(True) + conn = get_postgres_conn(started_cluster, True) cursor = conn.cursor() node1.query(''' From 9e02588505046431c8a0cbf72bbe389db0ba26a3 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 26 Mar 2021 15:53:53 +0300 Subject: [PATCH 0083/1060] more parallel tests --- tests/integration/parallel.txt | 20 ++++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt index fac2f589358..67daeced0f1 100644 --- a/tests/integration/parallel.txt +++ b/tests/integration/parallel.txt @@ -11,6 +11,9 @@ test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node1_cache] test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_hashed] test_dictionaries_ddl/test.py::test_create_and_select_mysql[complex_node2_cache] test_dictionaries_mysql/test.py::test_load_mysql_dictionaries +test_dictionaries_postgresql/test.py::test_load_dictionaries +test_dictionaries_postgresql/test.py::test_invalidate_query +test_dictionaries_postgresql/test.py::test_dictionary_with_replicas test_disabled_mysql_server/test.py::test_disabled_mysql_server test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[ordinary] test_materialize_mysql_database/test.py::test_materialize_database_dml_with_mysql_5_7[atomic] @@ -115,6 +118,19 @@ test_mysql_protocol/test.py::test_java_client test_mysql_protocol/test.py::test_types test_odbc_interaction/test.py::test_mysql_simple_select_works test_odbc_interaction/test.py::test_mysql_insert +test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_with_schema +test_odbc_interaction/test.py::test_postgres_odbc_hashed_dictionary_no_tty_pipe_overflow +test_odbc_interaction/test.py::test_postgres_insert +test_odbc_interaction/test.py::test_odbc_postgres_date_data_type +test_postgresql_database_engine/test.py::test_postgres_database_engine_with_postgres_ddl +test_postgresql_database_engine/test.py::test_postgresql_database_engine_with_clickhouse_ddl +test_postgresql_database_engine/test.py::test_postgresql_database_engine_queries +test_postgresql_database_engine/test.py::test_get_create_table_query_with_multidim_arrays +test_postgresql_database_engine/test.py::test_postgresql_database_engine_table_cache +test_postgresql_protocol/test.py::test_psql_is_ready +test_postgresql_protocol/test.py::test_psql_client +test_postgresql_protocol/test.py::test_python_client +test_postgresql_protocol/test.py::test_java_client test_storage_mysql/test.py::test_many_connections test_storage_mysql/test.py::test_insert_select test_storage_mysql/test.py::test_replace_select @@ -123,3 +139,7 @@ test_storage_mysql/test.py::test_where test_storage_mysql/test.py::test_table_function test_storage_mysql/test.py::test_binary_type test_storage_mysql/test.py::test_enum_type +test_storage_postgresql/test.py::test_postgres_select_insert +test_storage_postgresql/test.py::test_postgres_conversions +test_storage_postgresql/test.py::test_non_default_scema +test_storage_postgresql/test.py::test_concurrent_queries From df8013c22d34317a373f36b305ecfda5119dfc08 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Fri, 26 Mar 2021 21:46:42 +0300 Subject: [PATCH 0084/1060] fix --- tests/integration/test_storage_s3/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 60b6ef6bf0c..0d92bdf8493 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -424,7 +424,7 @@ def test_custom_auth_headers(started_cluster): instance.query( "CREATE TABLE test ({table_format}) ENGINE = S3('http://resolver:8080/{bucket}/{file}', 'CSV')".format( - bucket=cluster.minio_restricted_bucket, + bucket=started_cluster.minio_restricted_bucket, file=filename, table_format=table_format )) From 7b7580204c000890eaa16baf301040d1e07b6250 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Sun, 28 Mar 2021 00:26:53 +0300 Subject: [PATCH 0085/1060] fix --- tests/integration/test_storage_kafka/test.py | 34 +++++++++----------- 1 file changed, 16 insertions(+), 18 deletions(-) diff --git a/tests/integration/test_storage_kafka/test.py b/tests/integration/test_storage_kafka/test.py index 60ca3f5a790..5e13cdfde68 100644 --- a/tests/integration/test_storage_kafka/test.py +++ b/tests/integration/test_storage_kafka/test.py @@ -2557,26 +2557,23 @@ def test_premature_flush_on_eof(kafka_cluster): ''') -@pytest.mark.timeout(120) +@pytest.mark.timeout(180) def test_kafka_unavailable(kafka_cluster): messages = [json.dumps({'key': j + 1, 'value': j + 1}) for j in range(20000)] - kafka_produce(kafka_cluster, 'test_kafka_unavailable', messages) + kafka_produce(kafka_cluster, 'test_bad_reschedule', messages) kafka_cluster.pause_container('kafka1') instance.query(''' - DROP TABLE IF EXISTS test.destination_kafka_unavailable; - DROP TABLE IF EXISTS test.test_kafka_unavailable; - CREATE TABLE test.test_kafka_unavailable (key UInt64, value UInt64) + CREATE TABLE test.test_bad_reschedule (key UInt64, value UInt64) ENGINE = Kafka SETTINGS kafka_broker_list = 'kafka1:19092', - kafka_topic_list = 'test_kafka_unavailable', - kafka_group_name = 'test_kafka_unavailable', + kafka_topic_list = 'test_bad_reschedule', + kafka_group_name = 'test_bad_reschedule', kafka_format = 'JSONEachRow', - kafka_max_block_size = 1000, - kafka_flush_interval_ms = 1000; + kafka_max_block_size = 1000; - CREATE MATERIALIZED VIEW test.destination_kafka_unavailable Engine=Log AS + CREATE MATERIALIZED VIEW test.destination_unavailable Engine=Log AS SELECT key, now() as consume_ts, @@ -2586,19 +2583,20 @@ def test_kafka_unavailable(kafka_cluster): _offset, _partition, _timestamp - FROM test.test_kafka_unavailable; + FROM test.test_bad_reschedule; ''') - instance.query("SELECT * FROM test.test_kafka_unavailable") - - instance.wait_for_log_line('brokers are down') - instance.wait_for_log_line('stalled. Reschedule', repetitions=2) + instance.query("SELECT * FROM test.test_bad_reschedule") + instance.query("SELECT count() FROM test.destination_unavailable") + # enough to trigger issue + time.sleep(30) kafka_cluster.unpause_container('kafka1') - instance.wait_for_log_line("Committed offset 2000") - assert int(instance.query("SELECT count() FROM test.destination_kafka_unavailable")) == 5000 - time.sleep(5) # needed to give time for kafka client in python test to recovery + while int(instance.query("SELECT count() FROM test.destination_unavailable")) < 20000: + print("Waiting for consume") + time.sleep(1) + @pytest.mark.timeout(180) def test_kafka_issue14202(kafka_cluster): From 808da339136a75bf5599b235e7b18066fb5d6ab4 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Mon, 29 Mar 2021 17:52:51 +0900 Subject: [PATCH 0086/1060] Include bitset and rename test --- src/AggregateFunctions/AggregateFunctionSequenceNextNode.h | 1 + ...xt_node.reference => 01656_sequence_next_node_long.reference} | 0 ..._sequence_next_node.sql => 01656_sequence_next_node_long.sql} | 0 3 files changed, 1 insertion(+) rename tests/queries/0_stateless/{01656_sequence_next_node.reference => 01656_sequence_next_node_long.reference} (100%) rename tests/queries/0_stateless/{01656_sequence_next_node.sql => 01656_sequence_next_node_long.sql} (100%) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c0ecdadf8b1..2ace80de0a5 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -22,6 +22,7 @@ #include #include +#include namespace DB diff --git a/tests/queries/0_stateless/01656_sequence_next_node.reference b/tests/queries/0_stateless/01656_sequence_next_node_long.reference similarity index 100% rename from tests/queries/0_stateless/01656_sequence_next_node.reference rename to tests/queries/0_stateless/01656_sequence_next_node_long.reference diff --git a/tests/queries/0_stateless/01656_sequence_next_node.sql b/tests/queries/0_stateless/01656_sequence_next_node_long.sql similarity index 100% rename from tests/queries/0_stateless/01656_sequence_next_node.sql rename to tests/queries/0_stateless/01656_sequence_next_node_long.sql From 24c52414335b9cff55db5c0c9d5dd47ea0b0271c Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 1 Apr 2021 18:19:36 +0300 Subject: [PATCH 0087/1060] Minor style fixes in AggregateFunctionSequenceNextNode --- .../AggregateFunctionSequenceNextNode.cpp | 119 ++++++------- .../AggregateFunctionSequenceNextNode.h | 158 +++++++++--------- 2 files changed, 134 insertions(+), 143 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index c31337e25c6..c3a42703e3d 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -11,8 +11,9 @@ namespace DB { -constexpr size_t MAX_EVENTS_SIZE = 64; -constexpr size_t MIN_REQUIRED_ARGS = 3; +constexpr size_t max_events_size = 64; + +constexpr size_t min_required_args = 3; namespace ErrorCodes { @@ -24,84 +25,72 @@ namespace ErrorCodes namespace { -template -inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl2(const DataTypePtr data_type, const DataTypes & argument_types, SeqBase base) -{ - if (base == HEAD) - return std::make_shared, Direction, HEAD, MIN_REQUIRED_ARGS>>(data_type, argument_types); - else if (base == TAIL) - return std::make_shared, Direction, TAIL, MIN_REQUIRED_ARGS>>(data_type, argument_types); - else if (base == FIRST_MATCH) - return std::make_shared, Direction, FIRST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); - else - return std::make_shared, Direction, LAST_MATCH, MIN_REQUIRED_ARGS>>(data_type, argument_types); -} - template -inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl1(const DataTypePtr data_type, const DataTypes & argument_types, SeqDirection direction, SeqBase base) +inline AggregateFunctionPtr createAggregateFunctionSequenceNodeImpl( + const DataTypePtr data_type, const DataTypes & argument_types, SequenceDirection direction, SequenceBase base) { - if (direction == FORWARD) - return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); - else - return createAggregateFunctionSequenceNodeImpl2(data_type, argument_types, base); + return std::make_shared>>( + data_type, argument_types, base, direction, min_required_args); } AggregateFunctionPtr createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, const DataTypes & argument_types, const Array & parameters) { - assert(max_events <= MAX_EVENTS_SIZE); + assert(max_events <= max_events_size); if (parameters.size() < 2) - throw Exception("Aggregate function " + name + " requires 2 parameters (direction, head)", + throw Exception("Aggregate function '" + name + "' requires 2 parameters (direction, head)", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + auto expected_param_type = Field::Types::Which::String; + if (parameters.at(0).getType() != expected_param_type || parameters.at(1).getType() != expected_param_type) + throw Exception("Aggregate function '" + name + "' requires 'String' parameters", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); String param_dir = parameters.at(0).safeGet(); - SeqDirection direction; - if (param_dir == "forward") - direction = FORWARD; - else if (param_dir == "backward") - direction = BACKWARD; - else + std::unordered_map seq_dir_mapping{ + {"forward", SequenceDirection::Forward}, + {"backward", SequenceDirection::Backward}, + }; + if (!seq_dir_mapping.contains(param_dir)) throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_dir, ErrorCodes::BAD_ARGUMENTS}; + SequenceDirection direction = seq_dir_mapping[param_dir]; String param_base = parameters.at(1).safeGet(); - SeqBase base; - if (param_base == "head") - base = HEAD; - else if (param_base == "tail") - base = TAIL; - else if (param_base == "first_match") - base = FIRST_MATCH; - else if (param_base == "last_match") - base = LAST_MATCH; - else + std::unordered_map seq_base_mapping{ + {"head", SequenceBase::Head}, + {"tail", SequenceBase::Tail}, + {"first_match", SequenceBase::FirstMatch}, + {"last_match", SequenceBase::LastMatch}, + }; + if (!seq_base_mapping.contains(param_base)) throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; + SequenceBase base = seq_base_mapping[param_base]; - if (argument_types.size() < MIN_REQUIRED_ARGS) - throw Exception("Aggregate function " + name + " requires at least two arguments.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if ((base == FIRST_MATCH || base == LAST_MATCH) && argument_types.size() < MIN_REQUIRED_ARGS + 1) - throw Exception("Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - else if (argument_types.size() > max_events + MIN_REQUIRED_ARGS) - throw Exception("Aggregate function " + name + " requires at most " + - std::to_string(max_events + MIN_REQUIRED_ARGS) + - " (timestamp, value_column, " + std::to_string(max_events) + " events) arguments.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + if (argument_types.size() < min_required_args) + throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (const auto * cond_arg = argument_types[1].get()) - { - if (!isUInt8(cond_arg)) - throw Exception{"Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " - + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - } + bool is_base_match_type = base == SequenceBase::FirstMatch || base == SequenceBase::LastMatch; + if (is_base_match_type && argument_types.size() < min_required_args + 1) + throw Exception( + "Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - for (const auto i : ext::range(MIN_REQUIRED_ARGS, argument_types.size())) + if (argument_types.size() > max_events + min_required_args) + throw Exception(fmt::format( + "Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.", + name, max_events + min_required_args, max_events), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (const auto * cond_arg = argument_types[1].get(); cond_arg && !isUInt8(cond_arg)) + throw Exception("Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " + + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + for (const auto i : ext::range(min_required_args, argument_types.size())) { const auto * cond_arg = argument_types[i].get(); if (!isUInt8(cond_arg)) - throw Exception{"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + " of aggregate function " - + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + throw Exception(fmt::format( + "Illegal type '{}' of {} argument of aggregate function '{}', must be UInt8", cond_arg->getName(), i + 1, name), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String) @@ -113,17 +102,17 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt16) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt32) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.idx == TypeIndex::UInt64) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.isDate()) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); if (timestamp_type.isDateTime()) - return createAggregateFunctionSequenceNodeImpl1(data_type, argument_types, direction, base); + return createAggregateFunctionSequenceNodeImpl(data_type, argument_types, direction, base); throw Exception{"Illegal type " + argument_types.front().get()->getName() + " of first argument of aggregate function " + name + ", must be Unsigned Number, Date, DateTime", @@ -143,7 +132,7 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) void registerAggregateFunctionSequenceNextNode(AggregateFunctionFactory & factory) { AggregateFunctionProperties properties = { .returns_default_when_only_null = true, .is_order_dependent = false }; - factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(MAX_EVENTS_SIZE), properties }); + factory.registerFunction("sequenceNextNode", { createAggregateFunctionSequenceNodeMaxArgs(max_events_size), properties }); factory.registerFunction("sequenceFirstNode", { createAggregateFunctionSequenceNodeMaxArgs(0), properties }); } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index 2ace80de0a5..c3907a5adee 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -28,18 +28,18 @@ namespace DB { -enum SeqDirection +enum class SequenceDirection { - FORWARD = 0, - BACKWARD = 1 + Forward, + Backward, }; -enum SeqBase +enum SequenceBase { - HEAD = 0, - TAIL = 1, - FIRST_MATCH = 2, - LAST_MATCH = 3 + Head, + Tail, + FirstMatch, + LastMatch, }; /// NodeBase used to implement a linked list for storage of SequenceNextNodeImpl @@ -121,7 +121,7 @@ struct NodeString : public NodeBase, MaxEventsSize> } }; -/// TODO : Expends SequenceNextNodeGeneralData to support other types +/// TODO : Support other types than string template struct SequenceNextNodeGeneralData { @@ -150,26 +150,40 @@ struct SequenceNextNodeGeneralData }; /// Implementation of sequenceFirstNode -template +template class SequenceNextNodeImpl final - : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> + : public IAggregateFunctionDataHelper, SequenceNextNodeImpl> { + using Self = SequenceNextNodeImpl; + using Data = SequenceNextNodeGeneralData; static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - static constexpr size_t EventColumn = 2; - static constexpr size_t BaseCondition = 1; + + static constexpr size_t base_cond_column_idx = 1; + static constexpr size_t event_column_idx = 2; + + SequenceBase seq_base_kind; + SequenceDirection seq_direction; + const size_t min_required_args; DataTypePtr & data_type; UInt8 events_size; UInt64 max_elems; - public: - SequenceNextNodeImpl(const DataTypePtr & data_type_, const DataTypes & arguments, UInt64 max_elems_ = std::numeric_limits::max()) - : IAggregateFunctionDataHelper, SequenceNextNodeImpl>( - {data_type_}, {}) + SequenceNextNodeImpl( + const DataTypePtr & data_type_, + const DataTypes & arguments, + SequenceBase seq_base_kind_, + SequenceDirection seq_direction_, + size_t min_required_args_, + UInt64 max_elems_ = std::numeric_limits::max()) + : IAggregateFunctionDataHelper, Self>({data_type_}, {}) + , seq_base_kind(seq_base_kind_) + , seq_direction(seq_direction_) + , min_required_args(min_required_args_) , data_type(this->argument_types[0]) - , events_size(arguments.size() - MinRequiredArgs) + , events_size(arguments.size() - min_required_args) , max_elems(max_elems_) { } @@ -202,7 +216,7 @@ public: void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override { - Node * node = Node::allocate(*columns[EventColumn], row_num, arena); + Node * node = Node::allocate(*columns[event_column_idx], row_num, arena); const auto timestamp = assert_cast *>(columns[0])->getData()[row_num]; @@ -214,11 +228,11 @@ public: /// + 4 (bit of event3) node->events_bitset.reset(); for (UInt8 i = 0; i < events_size; ++i) - if (assert_cast *>(columns[MinRequiredArgs + i])->getData()[row_num]) + if (assert_cast *>(columns[min_required_args + i])->getData()[row_num]) node->events_bitset.set(i); node->event_time = timestamp; - node->can_be_base = assert_cast *>(columns[BaseCondition])->getData()[row_num]; + node->can_be_base = assert_cast *>(columns[base_cond_column_idx])->getData()[row_num]; data(place).value.push_back(node, arena); } @@ -264,7 +278,7 @@ public: void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf) const override { - // Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node. + /// Temporarily do a const_cast to sort the values. It helps to reduce the computational burden on the initiator node. this->data(const_cast(place)).sort(); writeBinary(data(place).sorted, buf); @@ -272,22 +286,22 @@ public: auto & value = data(place).value; size_t size = std::min(static_cast(events_size + 1), value.size()); - switch (Base) + switch (seq_base_kind) { - case HEAD: + case SequenceBase::Head: writeVarUInt(size, buf); for (size_t i = 0; i < size; ++i) value[i]->write(buf); break; - case TAIL: + case SequenceBase::Tail: writeVarUInt(size, buf); for (size_t i = 0; i < size; ++i) value[value.size() - size + i]->write(buf); break; - case FIRST_MATCH: - case LAST_MATCH: + case SequenceBase::FirstMatch: + case SequenceBase::LastMatch: writeVarUInt(value.size(), buf); for (auto & node : value) node->write(buf); @@ -312,53 +326,42 @@ public: value[i] = Node::read(buf, arena); } - inline UInt32 getBaseIndex(Data & data, bool & exist) const + inline std::optional getBaseIndex(Data & data) const { - exist = false; if (data.value.size() == 0) - return 0; + return {}; - switch (Base) + switch (seq_base_kind) { - case HEAD: + case SequenceBase::Head: if (data.value[0]->can_be_base) - { - exist = true; return 0; - } break; - case TAIL: + case SequenceBase::Tail: if (data.value[data.value.size() - 1]->can_be_base) - { - exist = true; return data.value.size() - 1; + break; + + case SequenceBase::FirstMatch: + for (size_t i = 0; i < data.value.size(); ++i) + { + if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base) + return i; } break; - case FIRST_MATCH: - for (UInt64 i = 0; i < data.value.size(); ++i) - if (data.value[i]->events_bitset.test(0) && data.value[i]->can_be_base) - { - exist = true; - return i; - } - break; - - case LAST_MATCH: - for (UInt64 i = 0; i < data.value.size(); ++i) + case SequenceBase::LastMatch: + for (size_t i = 0; i < data.value.size(); ++i) { auto reversed_i = data.value.size() - i - 1; if (data.value[reversed_i]->events_bitset.test(0) && data.value[reversed_i]->can_be_base) - { - exist = true; return reversed_i; - } } break; } - return 0; + return {}; } /// This method returns an index of next node that matched the events. @@ -366,40 +369,37 @@ public: /// The first matched event is 0x00000001, the second one is 0x00000002, the third one is 0x00000004, and so on. UInt32 getNextNodeIndex(Data & data) const { - const UInt32 unmatched = data.value.size(); + const UInt32 unmatched_idx = data.value.size(); if (data.value.size() <= events_size) - return unmatched; + return unmatched_idx; data.sort(); - bool base_existence; - UInt32 base = getBaseIndex(data, base_existence); - if (!base_existence) - return unmatched; + std::optional base_opt = getBaseIndex(data); + if (!base_opt.has_value()) + return unmatched_idx; + UInt32 base = static_cast(base_opt.value()); if (events_size == 0) - { - return data.value.size() > 0 ? base : unmatched; - } - else - { - UInt32 i = 0; - switch (Direction) - { - case FORWARD: - for (i = 0; i < events_size && base + i < data.value.size(); ++i) - if (data.value[base + i]->events_bitset.test(i) == false) - break; - return (i == events_size) ? base + i : unmatched; + return data.value.size() > 0 ? base : unmatched_idx; - case BACKWARD: - for (i = 0; i < events_size && i < base; ++i) - if (data.value[base - i]->events_bitset.test(i) == false) - break; - return (i == events_size) ? base - i : unmatched; - } + UInt32 i = 0; + switch (seq_direction) + { + case SequenceDirection::Forward: + for (i = 0; i < events_size && base + i < data.value.size(); ++i) + if (!data.value[base + i]->events_bitset.test(i)) + break; + return (i == events_size) ? base + i : unmatched_idx; + + case SequenceDirection::Backward: + for (i = 0; i < events_size && i < base; ++i) + if (!data.value[base - i]->events_bitset.test(i)) + break; + return (i == events_size) ? base - i : unmatched_idx; } + __builtin_unreachable(); } void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override @@ -414,7 +414,9 @@ public: to_concrete.getNullMapData().push_back(0); } else + { to.insertDefault(); + } } bool allocatesMemoryInArena() const override { return true; } From 63dc35780ed4b28bcf4a2dcdac16930279c4fe7d Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 2 Apr 2021 11:58:58 +0900 Subject: [PATCH 0088/1060] More parameter checking and remove a useless function --- .../AggregateFunctionNull.h | 22 ------------------- .../AggregateFunctionSequenceNextNode.cpp | 9 ++++++-- 2 files changed, 7 insertions(+), 24 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionNull.h b/src/AggregateFunctions/AggregateFunctionNull.h index 821398e715e..e1238182ab5 100644 --- a/src/AggregateFunctions/AggregateFunctionNull.h +++ b/src/AggregateFunctions/AggregateFunctionNull.h @@ -276,28 +276,6 @@ public: this->nested_function->add(this->nestedPlace(place), nested_columns, row_num, arena); } - void insertResultInto(AggregateDataPtr place, IColumn & to, Arena * arena) const override - { - if constexpr (result_is_nullable) - { - ColumnNullable & to_concrete = assert_cast(to); - if (this->getFlag(place)) - { - this->nested_function->insertResultInto(this->nestedPlace(place), to_concrete.getNestedColumn(), arena); - to_concrete.getNullMapData().push_back(0); - } - else - { - to_concrete.insertDefault(); - } - } - else - { - this->nested_function->insertResultInto(this->nestedPlace(place), to, arena); - } - } - - private: enum { MAX_ARGS = 8 }; size_t number_of_arguments = 0; diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index c3a42703e3d..6f2e2a5c3d7 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -66,13 +66,18 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, throw Exception{"Aggregate function " + name + " doesn't support a parameter: " + param_base, ErrorCodes::BAD_ARGUMENTS}; SequenceBase base = seq_base_mapping[param_base]; + if ((base == SequenceBase::Head && direction == SequenceDirection::Backward) || + (base == SequenceBase::Tail && direction == SequenceDirection::Forward)) + throw Exception(fmt::format( + "Invalid argument combination of '{}' with '{}'", param_base, param_dir), ErrorCodes::BAD_ARGUMENTS); + if (argument_types.size() < min_required_args) - throw Exception("Aggregate function " + name + " requires at least two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception("Aggregate function " + name + " requires at least " + toString(min_required_args) + " arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); bool is_base_match_type = base == SequenceBase::FirstMatch || base == SequenceBase::LastMatch; if (is_base_match_type && argument_types.size() < min_required_args + 1) throw Exception( - "Aggregate function " + name + " requires at least three arguments when base is first_match or last_match.", + "Aggregate function " + name + " requires at least " + toString(min_required_args + 1) + " arguments when base is first_match or last_match.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); if (argument_types.size() > max_events + min_required_args) From a87234f67b55e3cf6bf10cf5f86507e273754776 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 1 Apr 2021 22:50:18 +0000 Subject: [PATCH 0089/1060] Buildable version + small refactoring --- src/CMakeLists.txt | 1 + src/Disks/DiskHDFS.cpp | 713 ------------------- src/Disks/DiskType.h | 6 +- src/Disks/HDFS/DiskHDFS.cpp | 362 ++++++++++ src/Disks/{ => HDFS}/DiskHDFS.h | 82 ++- src/Disks/HDFS/HDFSHelpers.h | 234 ++++++ src/Disks/HDFS/ReadIndirectBufferFromHDFS.h | 143 ++++ src/Disks/HDFS/WriteIndirectBufferFromHDFS.h | 79 ++ src/IO/ReadBufferFromHDFS.cpp | 124 ---- src/IO/ReadBufferFromHDFS.h | 33 - src/IO/WriteBufferFromHDFS.cpp | 112 --- 11 files changed, 870 insertions(+), 1019 deletions(-) delete mode 100644 src/Disks/DiskHDFS.cpp create mode 100644 src/Disks/HDFS/DiskHDFS.cpp rename src/Disks/{ => HDFS}/DiskHDFS.h (72%) create mode 100644 src/Disks/HDFS/HDFSHelpers.h create mode 100644 src/Disks/HDFS/ReadIndirectBufferFromHDFS.h create mode 100644 src/Disks/HDFS/WriteIndirectBufferFromHDFS.h delete mode 100644 src/IO/ReadBufferFromHDFS.cpp delete mode 100644 src/IO/ReadBufferFromHDFS.h delete mode 100644 src/IO/WriteBufferFromHDFS.cpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index fe519530123..295a141fa3c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -101,6 +101,7 @@ endif() if (USE_HDFS) add_headers_and_sources(dbms Storages/HDFS) + add_headers_and_sources(dbms Disks/HDFS) endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) diff --git a/src/Disks/DiskHDFS.cpp b/src/Disks/DiskHDFS.cpp deleted file mode 100644 index 50827c80ec2..00000000000 --- a/src/Disks/DiskHDFS.cpp +++ /dev/null @@ -1,713 +0,0 @@ -#include "DiskHDFS.h" - -#include -#include -#include - -#include "DiskFactory.h" -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int EXCESSIVE_ELEMENT_IN_CONFIG; - extern const int PATH_ACCESS_DENIED; - extern const int FILE_ALREADY_EXISTS; - extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int UNKNOWN_FORMAT; - extern const int CANNOT_REMOVE_FILE; -} - -namespace -{ - String getRandomName() - { - std::uniform_int_distribution distribution('a', 'z'); - String res(32, ' '); /// The number of bits of entropy should be not less than 128. - for (auto & c : res) - c = distribution(thread_local_rng); - return res; - } - - /* - template - void throwIfError(Aws::Utils::Outcome && response) - { - if (!response.IsSuccess()) - { - const auto & err = response.GetError(); - throw Exception(err.GetMessage(), static_cast(err.GetErrorType())); - } - } - */ - - struct Metadata - { - /// Metadata file version. - static constexpr UInt32 VERSION = 1; - - using PathAndSize = std::pair; - - /// Disk path. - const String & disk_path; - /// Relative path to metadata file on local FS. - String metadata_file_path; - /// Total size of all HDFS objects. - size_t total_size; - /// HDFS objects paths and their sizes. - std::vector hdfs_objects; - /// Number of references (hardlinks) to this metadata file. - UInt32 ref_count; - - /// Load metadata by path or create empty if `create` flag is set. - explicit Metadata(const String & disk_path_, const String & metadata_file_path_, bool create = false) - : disk_path(disk_path_), metadata_file_path(metadata_file_path_), total_size(0), hdfs_objects(0), ref_count(0) - { - if (create) - return; - - ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ - - UInt32 version; - readIntText(version, buf); - - if (version != VERSION) - throw Exception( - "Unknown metadata file version. Path: " + disk_path + metadata_file_path - + " Version: " + std::to_string(version) + ", Expected version: " + std::to_string(VERSION), - ErrorCodes::UNKNOWN_FORMAT); - - assertChar('\n', buf); - - UInt32 hdfs_objects_count; - readIntText(hdfs_objects_count, buf); - assertChar('\t', buf); - readIntText(total_size, buf); - assertChar('\n', buf); - hdfs_objects.resize(hdfs_objects_count); - for (UInt32 i = 0; i < hdfs_objects_count; ++i) - { - String hdfs_object_path; - size_t hdfs_object_size; - readIntText(hdfs_object_size, buf); - assertChar('\t', buf); - readEscapedString(hdfs_object_path, buf); - assertChar('\n', buf); - hdfs_objects[i] = {hdfs_object_path, hdfs_object_size}; - } - - readIntText(ref_count, buf); - assertChar('\n', buf); - } - - void addObject(const String & path, size_t size) - { - total_size += size; - hdfs_objects.emplace_back(path, size); - } - - /// Fsync metadata file if 'sync' flag is set. - void save(bool sync = false) - { - WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); - - writeIntText(VERSION, buf); - writeChar('\n', buf); - - writeIntText(hdfs_objects.size(), buf); - writeChar('\t', buf); - writeIntText(total_size, buf); - writeChar('\n', buf); - for (const auto & [hdfs_object_path, hdfs_object_size] : hdfs_objects) - { - writeIntText(hdfs_object_size, buf); - writeChar('\t', buf); - writeEscapedString(hdfs_object_path, buf); - writeChar('\n', buf); - } - - writeIntText(ref_count, buf); - writeChar('\n', buf); - - buf.finalize(); - if (sync) - buf.sync(); - } - }; - - /// Reads data from HDFS using stored paths in metadata. - class ReadIndirectBufferFromHDFS final : public ReadBufferFromFileBase - { - public: - ReadIndirectBufferFromHDFS( - const String& hdfs_name_, const String & bucket_, Metadata metadata_, size_t buf_size_) - : hdfs_name(hdfs_name_), bucket(bucket_), metadata(std::move(metadata_)), buf_size(buf_size_) - { - } - - off_t seek(off_t offset_, int whence) override - { - if (whence == SEEK_CUR) - { - /// If position within current working buffer - shift pos. - if (working_buffer.size() && size_t(getPosition() + offset_) < absolute_position) - { - pos += offset_; - return getPosition(); - } - else - { - absolute_position += offset_; - } - } - else if (whence == SEEK_SET) - { - /// If position within current working buffer - shift pos. - if (working_buffer.size() && size_t(offset_) >= absolute_position - working_buffer.size() - && size_t(offset_) < absolute_position) - { - pos = working_buffer.end() - (absolute_position - offset_); - return getPosition(); - } - else - { - absolute_position = offset_; - } - } - else - throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - - 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 (size_t i = 0; i < metadata.hdfs_objects.size(); ++i) - { - current_buf_idx = i; - const auto & [path, size] = metadata.hdfs_objects[i]; - - if (size > offset) - { - auto buf = std::make_unique(hdfs_name + path, buf_size); - buf->seek(offset, SEEK_SET); - return buf; - } - offset -= size; - - } - return nullptr; - } - - bool nextImpl() override - { - /// Find first available buffer that fits to given offset. - if (!current_buf) - current_buf = initialize(); - - /// 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; - } - - /// If there is no available buffers - nothing to read. - if (current_buf_idx + 1 >= metadata.hdfs_objects.size()) - return false; - - ++current_buf_idx; - const auto & path = metadata.hdfs_objects[current_buf_idx].first; - current_buf = std::make_unique(hdfs_name + path, buf_size); - current_buf->next(); - working_buffer = current_buf->buffer(); - absolute_position += working_buffer.size(); - - return true; - } - - const String & hdfs_name; - const String & bucket; - Metadata metadata; - size_t buf_size; - - size_t absolute_position = 0; - size_t current_buf_idx = 0; - std::unique_ptr current_buf; - }; - - /// Stores data in HDFS and adds the object key (HDFS path) and object size to metadata file on local FS. - class WriteIndirectBufferFromHDFS final : public WriteBufferFromFileBase - { - public: - WriteIndirectBufferFromHDFS( - const String & hdfs_name_, - const String & hdfs_path_, - Metadata metadata_, - size_t buf_size_) - : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , impl(WriteBufferFromHDFS(hdfs_name_, buf_size_)) - , metadata(std::move(metadata_)) - , hdfs_path(hdfs_path_) - { - } - - ~WriteIndirectBufferFromHDFS() override - { - try - { - finalize(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - void finalize() override - { - if (finalized) - return; - - next(); - impl.finalize(); - - metadata.addObject(hdfs_path, count()); - metadata.save(); - - finalized = true; - } - - void sync() override - { - if (finalized) - metadata.save(true); - } - - std::string getFileName() const override { return metadata.metadata_file_path; } - - private: - void nextImpl() override - { - /// Transfer current working buffer to WriteBufferFromHDFS. - impl.swap(*this); - - /// Write actual data to HDFS. - impl.next(); - - /// Return back working buffer. - impl.swap(*this); - } - - WriteBufferFromHDFS impl; - bool finalized = false; - Metadata metadata; - String hdfs_path; - }; -} - - -class DiskHDFSDirectoryIterator final : public IDiskDirectoryIterator -{ -public: - DiskHDFSDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {} - - void next() override { ++iter; } - - bool isValid() const override { return iter != Poco::DirectoryIterator(); } - - String path() const override - { - if (iter->isDirectory()) - return folder_path + iter.name() + '/'; - else - return folder_path + iter.name(); - } - - String name() const override { return iter.name(); } - -private: - Poco::DirectoryIterator iter; - String folder_path; -}; - - -using DiskHDFSPtr = std::shared_ptr; - -class DiskHDFSReservation final : public IReservation -{ -public: - DiskHDFSReservation(const DiskHDFSPtr & disk_, UInt64 size_) - : disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_) - { - } - - UInt64 getSize() const override { return size; } - - DiskPtr getDisk() const override { return disk; } - - void update(UInt64 new_size) override - { - std::lock_guard lock(disk->reservation_mutex); - disk->reserved_bytes -= size; - size = new_size; - disk->reserved_bytes += size; - } - - ~DiskHDFSReservation() override; - -private: - DiskHDFSPtr disk; - UInt64 size; - CurrentMetrics::Increment metric_increment; -}; - - -DiskHDFS::DiskHDFS( - String name_, - String hdfs_name_, - String metadata_path_) - : name(std::move(name_)) - , hdfs_name(std::move(hdfs_name_)) - , metadata_path(std::move(metadata_path_)) - , builder(createHDFSBuilder(hdfs_name)) - , fs(createHDFSFS(builder.get())) -{ -} - -ReservationPtr DiskHDFS::reserve(UInt64 bytes) -{ - if (!tryReserve(bytes)) - return {}; - return std::make_unique(std::static_pointer_cast(shared_from_this()), bytes); -} - -bool DiskHDFS::exists(const String & path) const -{ - return Poco::File(metadata_path + path).exists(); -} - -bool DiskHDFS::isFile(const String & path) const -{ - return Poco::File(metadata_path + path).isFile(); -} - -bool DiskHDFS::isDirectory(const String & path) const -{ - return Poco::File(metadata_path + path).isDirectory(); -} - -size_t DiskHDFS::getFileSize(const String & path) const -{ - Metadata metadata(metadata_path, path); - return metadata.total_size; -} - -void DiskHDFS::createDirectory(const String & path) -{ - Poco::File(metadata_path + path).createDirectory(); -} - -void DiskHDFS::createDirectories(const String & path) -{ - Poco::File(metadata_path + path).createDirectories(); -} - -DiskDirectoryIteratorPtr DiskHDFS::iterateDirectory(const String & path) -{ - return std::make_unique(metadata_path + path, path); -} - -void DiskHDFS::clearDirectory(const String & path) -{ - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) - if (isFile(it->path())) - remove(it->path()); -} - -void DiskHDFS::moveFile(const String & from_path, const String & to_path) -{ - if (exists(to_path)) - throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS); - Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); -} - -void DiskHDFS::replaceFile(const String & from_path, const String & to_path) -{ - Poco::File from_file(metadata_path + from_path); - Poco::File to_file(metadata_path + to_path); - if (to_file.exists()) - { - Poco::File tmp_file(metadata_path + to_path + ".old"); - to_file.renameTo(tmp_file.path()); - from_file.renameTo(metadata_path + to_path); - remove(to_path + ".old"); - } - else - from_file.renameTo(to_file.path()); -} - -void DiskHDFS::copyFile(const String & from_path, const String & to_path) -{ - if (exists(to_path)) - remove(to_path); - - Metadata from(metadata_path, from_path); - Metadata to(metadata_path, to_path, true); - - for (const auto & [path, size] : from.hdfs_objects) - { - auto new_path = hdfs_name + getRandomName(); - /// TODO:: hdfs copy semantics - /* - Aws::HDFS::Model::CopyObjectRequest req; - req.SetCopySource(bucket + "/" + path); - req.SetBucket(bucket); - req.SetKey(new_path); - throwIfError(client->CopyObject(req)); - */ - throw Exception("is not implemented yet", 1); - to.addObject(new_path, size); - } - - to.save(); -} - -std::unique_ptr DiskHDFS::readFile(const String & path, size_t buf_size, size_t, size_t, size_t) const -{ - Metadata metadata(metadata_path, path); - - LOG_DEBUG( - &Logger::get("DiskHDFS"), - "Read from file by path: " << backQuote(metadata_path + path) << " Existing HDFS objects: " << metadata.hdfs_objects.size()); - - return std::make_unique(hdfs_name, "", metadata, buf_size); -} - -std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode, size_t, size_t) -{ - bool exist = exists(path); - /// Path to store new HDFS object. - auto file_name = getRandomName(); - auto HDFS_path = hdfs_name + file_name; - if (!exist || mode == WriteMode::Rewrite) - { - /// If metadata file exists - remove and new. - if (exist) - remove(path); - Metadata metadata(metadata_path, path, true); - /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. - metadata.save(); - - LOG_DEBUG(&Logger::get("DiskHDFS"), "Write to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path); - - return std::make_unique(HDFS_path, file_name, metadata, buf_size); - } - else - { - Metadata metadata(metadata_path, path); - - LOG_DEBUG( - &Logger::get("DiskHDFS"), - "Append to file by path: " << backQuote(metadata_path + path) << " New HDFS path: " << HDFS_path - << " Existing HDFS objects: " << metadata.hdfs_objects.size()); - - return std::make_unique(HDFS_path, file_name, metadata, buf_size); - } -} - -void DiskHDFS::remove(const String & path) -{ - LOG_DEBUG(&Logger::get("DiskHDFS"), "Remove file by path: " << backQuote(metadata_path + path)); - - Poco::File file(metadata_path + path); - if (file.isFile()) - { - Metadata metadata(metadata_path, path); - - /// If there is no references - delete content from HDFS. - if (metadata.ref_count == 0) - { - file.remove(); - for (const auto & [hdfs_object_path, _] : metadata.hdfs_objects) - { - const size_t begin_of_path = hdfs_name.find('/', hdfs_name.find("//") + 2); - const std::string hdfs_path = hdfs_name.substr(begin_of_path) + hdfs_object_path; - int res = hdfsDelete(fs.get(), hdfs_path.c_str(), 0); - if (res == -1) - throw Exception("fuck " + hdfs_path, 1); - } - } - else /// In other case decrement number of references, save metadata and delete file. - { - --metadata.ref_count; - metadata.save(); - file.remove(); - } - } - else - file.remove(); -} - -void DiskHDFS::removeRecursive(const String & path) -{ - checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. - - Poco::File file(metadata_path + path); - if (file.isFile()) - { - remove(path); - } - else - { - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) - removeRecursive(it->path()); - file.remove(); - } -} - - -bool DiskHDFS::tryReserve(UInt64 bytes) -{ - std::lock_guard lock(reservation_mutex); - if (bytes == 0) - { - LOG_DEBUG(&Logger::get("DiskHDFS"), "Reserving 0 bytes on HDFS disk " << backQuote(name)); - ++reservation_count; - return true; - } - - auto available_space = getAvailableSpace(); - UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); - if (unreserved_space >= bytes) - { - LOG_DEBUG( - &Logger::get("DiskHDFS"), - "Reserving " << formatReadableSizeWithBinarySuffix(bytes) << " on disk " << backQuote(name) << ", having unreserved " - << formatReadableSizeWithBinarySuffix(unreserved_space) << "."); - ++reservation_count; - reserved_bytes += bytes; - return true; - } - return false; -} - -void DiskHDFS::listFiles(const String & path, std::vector & file_names) -{ - for (auto it = iterateDirectory(path); it->isValid(); it->next()) - file_names.push_back(it->name()); -} - -void DiskHDFS::setLastModified(const String & path, const Poco::Timestamp & timestamp) -{ - Poco::File(metadata_path + path).setLastModified(timestamp); -} - -Poco::Timestamp DiskHDFS::getLastModified(const String & path) -{ - return Poco::File(metadata_path + path).getLastModified(); -} - -void DiskHDFS::createHardLink(const String & src_path, const String & dst_path) -{ - /// Increment number of references. - Metadata src(metadata_path, src_path); - ++src.ref_count; - src.save(); - - /// Create FS hardlink to metadata file. - DB::createHardLink(metadata_path + src_path, metadata_path + dst_path); -} - -void DiskHDFS::createFile(const String & path) -{ - /// Create empty metadata file. - Metadata metadata(metadata_path, path, true); - metadata.save(); -} - -void DiskHDFS::setReadOnly(const String & path) -{ - Poco::File(metadata_path + path).setReadOnly(true); -} - -DiskHDFSReservation::~DiskHDFSReservation() -{ - try - { - std::lock_guard lock(disk->reservation_mutex); - if (disk->reserved_bytes < size) - { - disk->reserved_bytes = 0; - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservations size for disk '" + disk->getName() + "'."); - } - else - { - disk->reserved_bytes -= size; - } - - if (disk->reservation_count == 0) - LOG_ERROR(&Logger::get("DiskLocal"), "Unbalanced reservation count for disk '" + disk->getName() + "'."); - else - --disk->reservation_count; - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - -void registerDiskHDFS(DiskFactory & factory) -{ - auto creator = [](const String & name, - const Poco::Util::AbstractConfiguration & config, - const String & config_prefix, - const Context & context) -> DiskPtr { - const auto * disk_config = config.createView(config_prefix); - - Poco::File disk{context.getPath() + "disks/" + name}; - disk.createDirectories(); - - DB::String uri{disk_config->getString("endpoint")}; - if (uri.back() != '/') - throw Exception("HDFS path must ends with '/', but '" + uri + "' doesn't.", ErrorCodes::BAD_ARGUMENTS); - - String metadata_path = context.getPath() + "disks/" + name + "/"; - - return std::make_shared( - name, - uri, - metadata_path - ); - }; - factory.registerDiskType("hdfs", creator); -} - -} diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index 4e0ae226af4..f0c22eb6495 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -11,7 +11,8 @@ struct DiskType { Local, RAM, - S3 + S3, + HDFS }; static String toString(Type disk_type) { @@ -23,10 +24,11 @@ struct DiskType return "memory"; case Type::S3: return "s3"; + case Type::HDFS: + return "HDFS"; } __builtin_unreachable(); } }; } - diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp new file mode 100644 index 00000000000..103a5c93c94 --- /dev/null +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -0,0 +1,362 @@ +#include + +#include +#include +#include "ReadIndirectBufferFromHDFS.h" +#include "WriteIndirectBufferFromHDFS.h" + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include "HDFSHelpers.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; + extern const int PATH_ACCESS_DENIED; + extern const int FILE_ALREADY_EXISTS; + extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int CANNOT_REMOVE_FILE; +} + + +DiskHDFS::DiskHDFS( + const Poco::Util::AbstractConfiguration & config_, + const String & name_, + const String & hdfs_name_, + const String & metadata_path_) + : config(config_) + , name(name_) + , hdfs_name(hdfs_name_) + , metadata_path(std::move(metadata_path_)) + , builder(createHDFSBuilder(hdfs_name, config)) + , fs(createHDFSFS(builder.get())) +{ +} + + +ReservationPtr DiskHDFS::reserve(UInt64 bytes) +{ + if (!tryReserve(bytes)) + return {}; + return std::make_unique(std::static_pointer_cast(shared_from_this()), bytes); +} + + +bool DiskHDFS::exists(const String & path) const +{ + return Poco::File(metadata_path + path).exists(); +} + + +bool DiskHDFS::isFile(const String & path) const +{ + return Poco::File(metadata_path + path).isFile(); +} + + +bool DiskHDFS::isDirectory(const String & path) const +{ + return Poco::File(metadata_path + path).isDirectory(); +} + + +size_t DiskHDFS::getFileSize(const String & path) const +{ + Metadata metadata(metadata_path, path); + return metadata.total_size; +} + + +void DiskHDFS::createDirectory(const String & path) +{ + Poco::File(metadata_path + path).createDirectory(); +} + + +void DiskHDFS::createDirectories(const String & path) +{ + Poco::File(metadata_path + path).createDirectories(); +} + + +DiskDirectoryIteratorPtr DiskHDFS::iterateDirectory(const String & path) +{ + return std::make_unique(metadata_path + path, path); +} + + +void DiskHDFS::clearDirectory(const String & path) +{ + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + if (isFile(it->path())) + removeFile(it->path()); +} + + +void DiskHDFS::moveFile(const String & from_path, const String & to_path) +{ + if (exists(to_path)) + throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS); + Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); +} + + +void DiskHDFS::replaceFile(const String & from_path, const String & to_path) +{ + Poco::File from_file(metadata_path + from_path); + Poco::File to_file(metadata_path + to_path); + if (to_file.exists()) + { + Poco::File tmp_file(metadata_path + to_path + ".old"); + to_file.renameTo(tmp_file.path()); + from_file.renameTo(metadata_path + to_path); + removeFile(to_path + ".old"); + } + else + from_file.renameTo(to_file.path()); +} + + +//void DiskHDFS::copyFile(const String & from_path, const String & to_path) +//{ +// if (exists(to_path)) +// remove(to_path); +// +// Metadata from(metadata_path, from_path); +// Metadata to(metadata_path, to_path, true); +// +// for (const auto & [path, size] : from.hdfs_objects) +// { +// auto new_path = hdfs_name + getRandomName(); +// /// TODO:: hdfs copy semantics +// /* +// Aws::HDFS::Model::CopyObjectRequest req; +// req.SetCopySource(bucket + "/" + path); +// req.SetBucket(bucket); +// req.SetKey(new_path); +// throwIfError(client->CopyObject(req)); +// */ +// throw Exception("is not implemented yet", 1); +// to.addObject(new_path, size); +// } +// +// to.save(); +//} + + +std::unique_ptr DiskHDFS::readFile(const String & path, size_t buf_size, size_t, size_t, size_t, MMappedFileCache *) const +{ + Metadata metadata(metadata_path, path); + + LOG_DEBUG( + &Poco::Logger::get("DiskHDFS"), + "Read from file by path: {}. Existing HDFS objects: {}", + backQuote(metadata_path + path), metadata.hdfs_objects.size()); + + return std::make_unique(config, hdfs_name, "", metadata, buf_size); +} + + +std::unique_ptr DiskHDFS::writeFile(const String & path, size_t buf_size, WriteMode mode) +{ + bool exist = exists(path); + /// Path to store new HDFS object. + auto file_name = getRandomName(); + auto HDFS_path = hdfs_name + file_name; + if (!exist || mode == WriteMode::Rewrite) + { + /// If metadata file exists - remove and new. + if (exist) + removeFile(path); + Metadata metadata(metadata_path, path, true); + /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. + metadata.save(); + + LOG_DEBUG( + &Poco::Logger::get("DiskHDFS"), + "Write to file by path: {}. New HDFS path: {}", backQuote(metadata_path + path), HDFS_path); + + return std::make_unique(config, HDFS_path, file_name, metadata, buf_size); + } + else + { + Metadata metadata(metadata_path, path); + + LOG_DEBUG( + &Poco::Logger::get("DiskHDFS"), + "Append to file by path: {}. New HDFS path: {}. Existing HDFS objects: {}", + backQuote(metadata_path + path), HDFS_path, metadata.hdfs_objects.size()); + + return std::make_unique(config, HDFS_path, file_name, metadata, buf_size); + } +} + + +void DiskHDFS::remove(const String & path) +{ + LOG_DEBUG(&Poco::Logger::get("DiskHDFS"), "Remove file by path: {}", backQuote(metadata_path + path)); + + Poco::File file(metadata_path + path); + if (file.isFile()) + { + Metadata metadata(metadata_path, path); + + /// If there is no references - delete content from HDFS. + if (metadata.ref_count == 0) + { + file.remove(); + for (const auto & [hdfs_object_path, _] : metadata.hdfs_objects) + { + const size_t begin_of_path = hdfs_name.find('/', hdfs_name.find("//") + 2); + const std::string hdfs_path = hdfs_name.substr(begin_of_path) + hdfs_object_path; + int res = hdfsDelete(fs.get(), hdfs_path.c_str(), 0); + if (res == -1) + throw Exception("HDFSDelete failed with path: " + hdfs_path, 1); + } + } + else /// In other case decrement number of references, save metadata and delete file. + { + --metadata.ref_count; + metadata.save(); + file.remove(); + } + } + else + file.remove(); +} + + +void DiskHDFS::removeRecursive(const String & path) +{ + checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. + + Poco::File file(metadata_path + path); + if (file.isFile()) + { + remove(path); + } + else + { + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + removeRecursive(it->path()); + file.remove(); + } +} + + +bool DiskHDFS::tryReserve(UInt64 bytes) +{ + std::lock_guard lock(reservation_mutex); + if (bytes == 0) + { + LOG_DEBUG(&Poco::Logger::get("DiskHDFS"), "Reserving 0 bytes on HDFS disk {}", backQuote(name)); + ++reservation_count; + return true; + } + + auto available_space = getAvailableSpace(); + UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); + if (unreserved_space >= bytes) + { + LOG_DEBUG( + &Poco::Logger::get("DiskHDFS"), + "Reserving {} on disk {}, having unreserved ", + formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); + + ++reservation_count; + reserved_bytes += bytes; + return true; + } + return false; +} + + +void DiskHDFS::listFiles(const String & path, std::vector & file_names) +{ + for (auto it = iterateDirectory(path); it->isValid(); it->next()) + file_names.push_back(it->name()); +} + + +void DiskHDFS::setLastModified(const String & path, const Poco::Timestamp & timestamp) +{ + Poco::File(metadata_path + path).setLastModified(timestamp); +} + + +Poco::Timestamp DiskHDFS::getLastModified(const String & path) +{ + return Poco::File(metadata_path + path).getLastModified(); +} + + +void DiskHDFS::createHardLink(const String & src_path, const String & dst_path) +{ + /// Increment number of references. + Metadata src(metadata_path, src_path); + ++src.ref_count; + src.save(); + + /// Create FS hardlink to metadata file. + DB::createHardLink(metadata_path + src_path, metadata_path + dst_path); +} + + +void DiskHDFS::createFile(const String & path) +{ + /// Create empty metadata file. + Metadata metadata(metadata_path, path, true); + metadata.save(); +} + + +void DiskHDFS::setReadOnly(const String & path) +{ + Poco::File(metadata_path + path).setReadOnly(true); +} + + +void registerDiskHDFS(DiskFactory & factory) +{ + auto creator = [](const String & name, + const Poco::Util::AbstractConfiguration & config, + const String & config_prefix, + const Context & context) -> DiskPtr + { + const auto * disk_config = config.createView(config_prefix); + + Poco::File disk{context.getPath() + "disks/" + name}; + disk.createDirectories(); + DB::String uri{disk_config->getString("endpoint")}; + + if (uri.back() != '/') + throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri); + + String metadata_path = context.getPath() + "disks/" + name + "/"; + + return std::make_shared(config, name, uri, metadata_path); + }; + + factory.registerDiskType("hdfs", creator); +} + +} diff --git a/src/Disks/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h similarity index 72% rename from src/Disks/DiskHDFS.h rename to src/Disks/HDFS/DiskHDFS.h index 5b35a90286c..781b7a961ec 100644 --- a/src/Disks/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -1,14 +1,15 @@ #pragma once -#include "Disks/DiskFactory.h" +#include #include -#include +#include #include namespace DB { + /** * Storage for persisting data in HDFS and metadata on the local disk. * Files are represented by file in local filesystem (clickhouse_root/disks/disk_name/path/to/file) @@ -16,20 +17,33 @@ namespace DB */ class DiskHDFS : public IDisk { + +friend class DiskHDFSReservation; + public: - friend class DiskHDFSReservation; DiskHDFS( - String name_, - String hdfs_name_, - String metadata_path_); + const Poco::Util::AbstractConfiguration & config, + const String & name_, + const String & hdfs_name_, + const String & metadata_path_); const String & getName() const override { return name; } - const String & getPath() const override { return metadata_path; } - ReservationPtr reserve(UInt64 bytes) override; + void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; + + Poco::Timestamp getLastModified(const String & path) override; + + void setReadOnly(const String & path) override; + + void createHardLink(const String & src_path, const String & dst_path) override; + + DiskType::Type getType() const override { return DiskType::Type::HDFS; } + + const String & getPath() const override { return metadata_path; } + UInt64 getTotalSpace() const override { return std::numeric_limits::max(); } UInt64 getAvailableSpace() const override { return std::numeric_limits::max(); } @@ -56,11 +70,13 @@ public: DiskDirectoryIteratorPtr iterateDirectory(const String & path) override; + void createFile(const String & path) override; + void moveFile(const String & from_path, const String & to_path) override; void replaceFile(const String & from_path, const String & to_path) override; - void copyFile(const String & from_path, const String & to_path) override; + void copy(const String & /* from_path */, const std::shared_ptr & /* to_disk */, const String & /* to_path */) override {} void listFiles(const String & path, std::vector & file_names) override; @@ -69,44 +85,40 @@ public: size_t buf_size, size_t estimated_size, size_t aio_threshold, - size_t mmap_threshold) const override; + size_t mmap_threshold, + MMappedFileCache * mmap_cache = nullptr) const override; std::unique_ptr writeFile( const String & path, size_t buf_size, - WriteMode mode, - size_t estimated_size, - size_t aio_threshold) override; + WriteMode mode) override; - void remove(const String & path) override; + void removeFile(const String & /* path */) override {} + + void removeFileIfExists(const String & /* path */) override {} + + void removeDirectory(const String & /* path */) override {} void removeRecursive(const String & path) override; - void createHardLink(const String & src_path, const String & dst_path) override; - - void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; - - Poco::Timestamp getLastModified(const String & path) override; - - void createFile(const String & path) override; - - void setReadOnly(const String & path) override; - -private: - bool tryReserve(UInt64 bytes); - -private: - const String name; - const String hdfs_name; - const String metadata_path; - - HDFSBuilderPtr builder; - HDFSFSPtr fs; - UInt64 reserved_bytes = 0; UInt64 reservation_count = 0; std::mutex reservation_mutex; std::mutex copying_mutex; + +private: + bool tryReserve(UInt64 bytes); + void remove(const String & path); + + const Poco::Util::AbstractConfiguration & config; + const String name; + const String hdfs_name; + String metadata_path; + + HDFSBuilderWrapper builder; + HDFSFSPtr fs; }; +using DiskHDFSPtr = std::shared_ptr; + } diff --git a/src/Disks/HDFS/HDFSHelpers.h b/src/Disks/HDFS/HDFSHelpers.h new file mode 100644 index 00000000000..e3465cfaa64 --- /dev/null +++ b/src/Disks/HDFS/HDFSHelpers.h @@ -0,0 +1,234 @@ +#pragma once +#include "DiskHDFS.h" + +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} +} + +namespace +{ + String getRandomName() + { + std::uniform_int_distribution distribution('a', 'z'); + String res(32, ' '); /// The number of bits of entropy should be not less than 128. + for (auto & c : res) + c = distribution(thread_local_rng); + return res; + } + + /* + template + void throwIfError(Aws::Utils::Outcome && response) + { + if (!response.IsSuccess()) + { + const auto & err = response.GetError(); + throw Exception(err.GetMessage(), static_cast(err.GetErrorType())); + } + } + */ + + struct Metadata + { + /// Metadata file version. + static constexpr UInt32 VERSION = 1; + + using PathAndSize = std::pair; + + /// Disk path. + const String & disk_path; + /// Relative path to metadata file on local FS. + String metadata_file_path; + /// Total size of all HDFS objects. + size_t total_size; + /// HDFS objects paths and their sizes. + std::vector hdfs_objects; + /// Number of references (hardlinks) to this metadata file. + UInt32 ref_count; + + /// Load metadata by path or create empty if `create` flag is set. + explicit Metadata(const String & disk_path_, const String & metadata_file_path_, bool create = false) + : disk_path(disk_path_), metadata_file_path(metadata_file_path_), total_size(0), hdfs_objects(0), ref_count(0) + { + if (create) + return; + + DB::ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ + + UInt32 version; + readIntText(version, buf); + + if (version != VERSION) + throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, + "Unknown metadata file version. Path: {}. Version: {}. Expected version: {}", + disk_path + metadata_file_path, std::to_string(version), std::to_string(VERSION)); + + assertChar('\n', buf); + + UInt32 hdfs_objects_count; + readIntText(hdfs_objects_count, buf); + assertChar('\t', buf); + readIntText(total_size, buf); + assertChar('\n', buf); + hdfs_objects.resize(hdfs_objects_count); + for (UInt32 i = 0; i < hdfs_objects_count; ++i) + { + String hdfs_object_path; + size_t hdfs_object_size; + readIntText(hdfs_object_size, buf); + assertChar('\t', buf); + readEscapedString(hdfs_object_path, buf); + assertChar('\n', buf); + hdfs_objects[i] = {hdfs_object_path, hdfs_object_size}; + } + + readIntText(ref_count, buf); + assertChar('\n', buf); + } + + void addObject(const String & path, size_t size) + { + total_size += size; + hdfs_objects.emplace_back(path, size); + } + + /// Fsync metadata file if 'sync' flag is set. + void save(bool sync = false) + { + DB::WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); + + writeIntText(VERSION, buf); + writeChar('\n', buf); + + writeIntText(hdfs_objects.size(), buf); + writeChar('\t', buf); + writeIntText(total_size, buf); + writeChar('\n', buf); + for (const auto & [hdfs_object_path, hdfs_object_size] : hdfs_objects) + { + writeIntText(hdfs_object_size, buf); + writeChar('\t', buf); + writeEscapedString(hdfs_object_path, buf); + writeChar('\n', buf); + } + + writeIntText(ref_count, buf); + writeChar('\n', buf); + + buf.finalize(); + if (sync) + buf.sync(); + } + }; + + +} + +namespace DB +{ + +class DiskHDFSDirectoryIterator final : public IDiskDirectoryIterator +{ +public: + DiskHDFSDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {} + + void next() override { ++iter; } + + bool isValid() const override { return iter != Poco::DirectoryIterator(); } + + String path() const override + { + if (iter->isDirectory()) + return folder_path + iter.name() + '/'; + else + return folder_path + iter.name(); + } + + String name() const override { return iter.name(); } + +private: + Poco::DirectoryIterator iter; + String folder_path; +}; + + +class DiskHDFSReservation : public IReservation +{ +public: + DiskHDFSReservation(const DiskHDFSPtr & disk_, UInt64 size_) + : disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_) + { + } + + ~DiskHDFSReservation() override + { + try + { + std::lock_guard lock(disk->reservation_mutex); + if (disk->reserved_bytes < size) + { + disk->reserved_bytes = 0; + //LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'", disk->getName()); + } + else + { + disk->reserved_bytes -= size; + } + + if (disk->reservation_count == 0) + { + //LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'", disk->getName()); + } + else + --disk->reservation_count; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + UInt64 getSize() const override { return size; } + + DiskPtr getDisk(size_t /* i */) const override { return disk; } + + void update(UInt64 new_size) override + { + std::lock_guard lock(disk->reservation_mutex); + disk->reserved_bytes -= size; + size = new_size; + disk->reserved_bytes += size; + } + + Disks getDisks() const override { return {}; } + +private: + DiskHDFSPtr disk; + UInt64 size; + CurrentMetrics::Increment metric_increment; +}; + +} diff --git a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h new file mode 100644 index 00000000000..702f45291ec --- /dev/null +++ b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h @@ -0,0 +1,143 @@ +#pragma once + +#include +#include "HDFSHelpers.h" + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int UNKNOWN_ELEMENT_IN_CONFIG; + extern const int EXCESSIVE_ELEMENT_IN_CONFIG; + extern const int PATH_ACCESS_DENIED; + extern const int FILE_ALREADY_EXISTS; + extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int UNKNOWN_FORMAT; + extern const int CANNOT_REMOVE_FILE; +} + + +/// Reads data from HDFS using stored paths in metadata. +class ReadIndirectBufferFromHDFS final : public ReadBufferFromFileBase +{ +public: + ReadIndirectBufferFromHDFS( + const Poco::Util::AbstractConfiguration & config_, + const String & hdfs_name_, + const String & /* bucket */, + Metadata metadata_, + size_t buf_size_) + : config(config_) + , hdfs_name(hdfs_name_) + , metadata(std::move(metadata_)) + , buf_size(buf_size_) + { + } + + off_t seek(off_t offset_, int whence) override + { + if (whence == SEEK_CUR) + { + /// If position within current working buffer - shift pos. + if (working_buffer.size() && size_t(getPosition() + offset_) < absolute_position) + { + pos += offset_; + return getPosition(); + } + else + { + absolute_position += offset_; + } + } + else if (whence == SEEK_SET) + { + /// If position within current working buffer - shift pos. + if (working_buffer.size() && size_t(offset_) >= absolute_position - working_buffer.size() + && size_t(offset_) < absolute_position) + { + pos = working_buffer.end() - (absolute_position - offset_); + return getPosition(); + } + else + { + absolute_position = offset_; + } + } + else + throw Exception("Only SEEK_SET or SEEK_CUR modes are allowed.", ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + + 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 (size_t i = 0; i < metadata.hdfs_objects.size(); ++i) + { + current_buf_idx = i; + const auto & [path, size] = metadata.hdfs_objects[i]; + + if (size > offset) + { + auto buf = std::make_unique(hdfs_name + path, config, buf_size); + //buf->seek(offset, SEEK_SET); + return buf; + } + offset -= size; + + } + + return nullptr; + } + + bool nextImpl() override + { + /// Find first available buffer that fits to given offset. + if (!current_buf) + current_buf = initialize(); + + /// 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; + } + + /// If there is no available buffers - nothing to read. + if (current_buf_idx + 1 >= metadata.hdfs_objects.size()) + return false; + + ++current_buf_idx; + const auto & path = metadata.hdfs_objects[current_buf_idx].first; + current_buf = std::make_unique(hdfs_name + path, config, buf_size); + current_buf->next(); + working_buffer = current_buf->buffer(); + absolute_position += working_buffer.size(); + + return true; + } + + const Poco::Util::AbstractConfiguration & config; + const String & hdfs_name; + Metadata metadata; + size_t buf_size; + + size_t absolute_position = 0; + size_t current_buf_idx = 0; + std::unique_ptr current_buf; +}; + +} diff --git a/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h b/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h new file mode 100644 index 00000000000..5f6bc353360 --- /dev/null +++ b/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h @@ -0,0 +1,79 @@ +#pragma once +#include +#include "HDFSHelpers.h" + + +namespace DB +{ + +/// Stores data in HDFS and adds the object key (HDFS path) and object size to metadata file on local FS. +class WriteIndirectBufferFromHDFS final : public WriteBufferFromFileBase +{ +public: + WriteIndirectBufferFromHDFS( + const Poco::Util::AbstractConfiguration & config, + const String & hdfs_name_, + const String & hdfs_path_, + Metadata metadata_, + size_t buf_size_) + : WriteBufferFromFileBase(buf_size_, nullptr, 0) + , impl(WriteBufferFromHDFS(hdfs_name_, config, buf_size_)) + , metadata(std::move(metadata_)) + , hdfs_path(hdfs_path_) + { + } + + ~WriteIndirectBufferFromHDFS() override + { + try + { + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + void finalize() override + { + if (finalized) + return; + + next(); + impl.finalize(); + + metadata.addObject(hdfs_path, count()); + metadata.save(); + + finalized = true; + } + + void sync() override + { + if (finalized) + metadata.save(true); + } + + std::string getFileName() const override { return metadata.metadata_file_path; } + +private: + void nextImpl() override + { + /// Transfer current working buffer to WriteBufferFromHDFS. + impl.swap(*this); + + /// Write actual data to HDFS. + impl.next(); + + /// Return back working buffer. + impl.swap(*this); + } + + WriteBufferFromHDFS impl; + bool finalized = false; + Metadata metadata; + String hdfs_path; +}; + +} diff --git a/src/IO/ReadBufferFromHDFS.cpp b/src/IO/ReadBufferFromHDFS.cpp deleted file mode 100644 index f7c5b0fe2b3..00000000000 --- a/src/IO/ReadBufferFromHDFS.cpp +++ /dev/null @@ -1,124 +0,0 @@ -#include "ReadBufferFromHDFS.h" - -#if USE_HDFS -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int NETWORK_ERROR; - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int CANNOT_TELL_THROUGH_FILE; -} - -ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; - -struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl -{ - std::string hdfs_uri; - hdfsFile fin; - HDFSBuilderPtr builder; - HDFSFSPtr fs; - off_t offset = 0; - bool initialized = false; - - explicit ReadBufferFromHDFSImpl(const std::string & hdfs_name_) - : hdfs_uri(hdfs_name_) - , builder(createHDFSBuilder(hdfs_uri)) - , fs(createHDFSFS(builder.get())) - { - const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); - const std::string path = hdfs_uri.substr(begin_of_path); - fin = hdfsOpenFile(fs.get(), path.c_str(), O_RDONLY, 0, 0, 0); - - if (fin == nullptr) - throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), - ErrorCodes::CANNOT_OPEN_FILE); - - } - - void initialize() - { - if (offset) - { - int seek_status = hdfsSeek(fs.get(), fin, offset); - if (seek_status != 0) - throw Exception("Fail to seek HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - } - } - - int read(char * start, size_t size) - { - if (!initialized) - { - initialize(); - initialized = true; - } - - int bytes_read = hdfsRead(fs.get(), fin, start, size); - if (bytes_read < 0) - throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - return bytes_read; - } - - int seek(off_t off, int whence) - { - 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); - - offset = off; - return offset; - } - - int tell() const - { - return offset; - } - - ~ReadBufferFromHDFSImpl() - { - hdfsCloseFile(fs.get(), fin); - } -}; - -ReadBufferFromHDFS::ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size) - : BufferWithOwnMemory(buf_size) - , impl(std::make_unique(hdfs_name_)) -{ -} - - -bool ReadBufferFromHDFS::nextImpl() -{ - int bytes_read = impl->read(internal_buffer.begin(), internal_buffer.size()); - - if (bytes_read) - working_buffer.resize(bytes_read); - else - return false; - return true; -} - -off_t ReadBufferFromHDFS::seek(off_t off, int whence) -{ - return impl->seek(off, whence); -} - -off_t ReadBufferFromHDFS::getPosition() -{ - return impl->tell() + count(); -} - -} - -#endif diff --git a/src/IO/ReadBufferFromHDFS.h b/src/IO/ReadBufferFromHDFS.h deleted file mode 100644 index e9e0dc5520c..00000000000 --- a/src/IO/ReadBufferFromHDFS.h +++ /dev/null @@ -1,33 +0,0 @@ -#pragma once - -#include - -#if USE_HDFS -#include -#include "SeekableReadBuffer.h" -#include -#include -#include - -namespace DB -{ -/** Accepts HDFS path to file and opens it. - * Closes file by himself (thus "owns" a file descriptor). - */ -class ReadBufferFromHDFS : public BufferWithOwnMemory -{ - struct ReadBufferFromHDFSImpl; - std::unique_ptr impl; -public: - ReadBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE); - ReadBufferFromHDFS(ReadBufferFromHDFS &&) = default; - ~ReadBufferFromHDFS() override; - - bool nextImpl() override; - - off_t seek(off_t off, int whence) override; - off_t getPosition() override; - -}; -} -#endif diff --git a/src/IO/WriteBufferFromHDFS.cpp b/src/IO/WriteBufferFromHDFS.cpp deleted file mode 100644 index 1fe3b19f865..00000000000 --- a/src/IO/WriteBufferFromHDFS.cpp +++ /dev/null @@ -1,112 +0,0 @@ -#include - -#if USE_HDFS - -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ -extern const int NETWORK_ERROR; -extern const int CANNOT_OPEN_FILE; -extern const int CANNOT_FSYNC; -extern const int BAD_ARGUMENTS; -} - - -struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl -{ - std::string hdfs_uri; - hdfsFile fout; - HDFSBuilderPtr builder; - HDFSFSPtr fs; - - explicit WriteBufferFromHDFSImpl(const std::string & hdfs_name_) - : hdfs_uri(hdfs_name_) - , builder(createHDFSBuilder(hdfs_uri)) - , fs(createHDFSFS(builder.get())) - { - const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); - const std::string path = hdfs_uri.substr(begin_of_path); - if (path.find_first_of("*?{") != std::string::npos) - throw Exception("URI '" + hdfs_uri + "' contains globs, so the table is in readonly mode", ErrorCodes::CANNOT_OPEN_FILE); - - if (!hdfsExists(fs.get(), path.c_str())) - throw Exception("File: " + path + " is already exists", ErrorCodes::BAD_ARGUMENTS); - fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here - - if (fout == nullptr) - { - throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), - ErrorCodes::CANNOT_OPEN_FILE); - } - - } - - ~WriteBufferFromHDFSImpl() - { - hdfsCloseFile(fs.get(), fout); - } - - - int write(const char * start, size_t size) const - { - int bytes_written = hdfsWrite(fs.get(), fout, start, size); - if (bytes_written < 0) - throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - return bytes_written; - } - - void sync() const - { - int result = hdfsSync(fs.get(), fout); - if (result < 0) - throwFromErrno("Cannot HDFS sync" + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::CANNOT_FSYNC); - } -}; - -WriteBufferFromHDFS::WriteBufferFromHDFS(const std::string & hdfs_name_, size_t buf_size) - : BufferWithOwnMemory(buf_size) - , impl(std::make_unique(hdfs_name_)) -{ -} - - -void WriteBufferFromHDFS::nextImpl() -{ - if (!offset()) - return; - - size_t bytes_written = 0; - - while (bytes_written != offset()) - bytes_written += impl->write(working_buffer.begin() + bytes_written, offset() - bytes_written); -} - - -void WriteBufferFromHDFS::sync() -{ - impl->sync(); -} - -WriteBufferFromHDFS::~WriteBufferFromHDFS() -{ - try - { - next(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } -} - -} -#endif From a75aa5d10133419c161f14b2431c324a7743bb7d Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Apr 2021 21:02:48 +0000 Subject: [PATCH 0090/1060] Some refactoring --- src/Disks/HDFS/DiskHDFS.cpp | 8 +- src/Disks/HDFS/DiskHDFS.h | 3 + src/Disks/HDFS/DiskHDFSDirectoryIterator.h | 34 +++ src/Disks/HDFS/DiskHDFSMetadata.h | 112 +++++++++ src/Disks/HDFS/DiskHDFSReservation.h | 68 ++++++ src/Disks/HDFS/HDFSHelpers.h | 234 ------------------- src/Disks/HDFS/ReadIndirectBufferFromHDFS.h | 2 +- src/Disks/HDFS/WriteIndirectBufferFromHDFS.h | 3 +- 8 files changed, 222 insertions(+), 242 deletions(-) create mode 100644 src/Disks/HDFS/DiskHDFSDirectoryIterator.h create mode 100644 src/Disks/HDFS/DiskHDFSMetadata.h create mode 100644 src/Disks/HDFS/DiskHDFSReservation.h delete mode 100644 src/Disks/HDFS/HDFSHelpers.h diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 103a5c93c94..e58212fabda 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -4,14 +4,12 @@ #include #include "ReadIndirectBufferFromHDFS.h" #include "WriteIndirectBufferFromHDFS.h" +#include "DiskHDFSReservation.h" +#include "DiskHDFSDirectoryIterator.h" #include #include #include -#include -#include -#include -#include #include #include #include @@ -19,9 +17,7 @@ #include #include #include - #include -#include "HDFSHelpers.h" namespace DB diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 781b7a961ec..65a35f3e2bd 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -107,6 +108,8 @@ public: std::mutex copying_mutex; private: + String getRandomName() { return toString(UUIDHelpers::generateV4()); } + bool tryReserve(UInt64 bytes); void remove(const String & path); diff --git a/src/Disks/HDFS/DiskHDFSDirectoryIterator.h b/src/Disks/HDFS/DiskHDFSDirectoryIterator.h new file mode 100644 index 00000000000..621949b0a82 --- /dev/null +++ b/src/Disks/HDFS/DiskHDFSDirectoryIterator.h @@ -0,0 +1,34 @@ +#pragma once + +#include + + +namespace DB +{ + +class DiskHDFSDirectoryIterator final : public IDiskDirectoryIterator +{ + +public: + DiskHDFSDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {} + + void next() override { ++iter; } + + bool isValid() const override { return iter != Poco::DirectoryIterator(); } + + String path() const override + { + if (iter->isDirectory()) + return folder_path + iter.name() + '/'; + else + return folder_path + iter.name(); + } + + String name() const override { return iter.name(); } + +private: + Poco::DirectoryIterator iter; + String folder_path; +}; + +} diff --git a/src/Disks/HDFS/DiskHDFSMetadata.h b/src/Disks/HDFS/DiskHDFSMetadata.h new file mode 100644 index 00000000000..16e3fe1c46a --- /dev/null +++ b/src/Disks/HDFS/DiskHDFSMetadata.h @@ -0,0 +1,112 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +struct Metadata +{ + /// Metadata file version. + static constexpr UInt32 VERSION = 1; + + using PathAndSize = std::pair; + + /// Disk path. + const String & disk_path; + /// Relative path to metadata file on local FS. + String metadata_file_path; + /// Total size of all HDFS objects. + size_t total_size; + /// HDFS objects paths and their sizes. + std::vector hdfs_objects; + /// Number of references (hardlinks) to this metadata file. + UInt32 ref_count; + + /// Load metadata by path or create empty if `create` flag is set. + explicit Metadata( + const String & disk_path_, + const String & metadata_file_path_, + bool create = false) + : disk_path(disk_path_) + , metadata_file_path(metadata_file_path_) + , total_size(0) + , hdfs_objects(0) + , ref_count(0) + { + if (create) + return; + + ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ + + UInt32 version; + readIntText(version, buf); + + if (version != VERSION) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Unknown metadata file version. Path: {}. Version: {}. Expected version: {}", + disk_path + metadata_file_path, std::to_string(version), std::to_string(VERSION)); + + assertChar('\n', buf); + + UInt32 hdfs_objects_count; + readIntText(hdfs_objects_count, buf); + assertChar('\t', buf); + readIntText(total_size, buf); + assertChar('\n', buf); + hdfs_objects.resize(hdfs_objects_count); + for (UInt32 i = 0; i < hdfs_objects_count; ++i) + { + String hdfs_object_path; + size_t hdfs_object_size; + readIntText(hdfs_object_size, buf); + assertChar('\t', buf); + readEscapedString(hdfs_object_path, buf); + assertChar('\n', buf); + hdfs_objects[i] = {hdfs_object_path, hdfs_object_size}; + } + + readIntText(ref_count, buf); + assertChar('\n', buf); + } + + void addObject(const String & path, size_t size) + { + total_size += size; + hdfs_objects.emplace_back(path, size); + } + + /// Fsync metadata file if 'sync' flag is set. + void save(bool sync = false) + { + DB::WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); + + writeIntText(VERSION, buf); + writeChar('\n', buf); + + writeIntText(hdfs_objects.size(), buf); + writeChar('\t', buf); + writeIntText(total_size, buf); + writeChar('\n', buf); + for (const auto & [hdfs_object_path, hdfs_object_size] : hdfs_objects) + { + writeIntText(hdfs_object_size, buf); + writeChar('\t', buf); + writeEscapedString(hdfs_object_path, buf); + writeChar('\n', buf); + } + + writeIntText(ref_count, buf); + writeChar('\n', buf); + + buf.finalize(); + if (sync) + buf.sync(); + } +}; + +} diff --git a/src/Disks/HDFS/DiskHDFSReservation.h b/src/Disks/HDFS/DiskHDFSReservation.h new file mode 100644 index 00000000000..c90614e599b --- /dev/null +++ b/src/Disks/HDFS/DiskHDFSReservation.h @@ -0,0 +1,68 @@ +#pragma once + +#include + + +namespace DB +{ + +class DiskHDFSReservation : public IReservation +{ + +public: + DiskHDFSReservation(const DiskHDFSPtr & disk_, UInt64 size_) + : disk(disk_) + , size(size_) + , metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_) + { + } + + ~DiskHDFSReservation() override + { + try + { + std::lock_guard lock(disk->reservation_mutex); + if (disk->reserved_bytes < size) + { + disk->reserved_bytes = 0; + //LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'", disk->getName()); + } + else + { + disk->reserved_bytes -= size; + } + + if (disk->reservation_count == 0) + { + //LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'", disk->getName()); + } + else + --disk->reservation_count; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } + + UInt64 getSize() const override { return size; } + + DiskPtr getDisk(size_t /* i */) const override { return disk; } + + void update(UInt64 new_size) override + { + std::lock_guard lock(disk->reservation_mutex); + disk->reserved_bytes -= size; + size = new_size; + disk->reserved_bytes += size; + } + + Disks getDisks() const override { return {}; } + +private: + DiskHDFSPtr disk; + UInt64 size; + CurrentMetrics::Increment metric_increment; +}; + +} diff --git a/src/Disks/HDFS/HDFSHelpers.h b/src/Disks/HDFS/HDFSHelpers.h deleted file mode 100644 index e3465cfaa64..00000000000 --- a/src/Disks/HDFS/HDFSHelpers.h +++ /dev/null @@ -1,234 +0,0 @@ -#pragma once -#include "DiskHDFS.h" - -#include -#include - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} -} - -namespace -{ - String getRandomName() - { - std::uniform_int_distribution distribution('a', 'z'); - String res(32, ' '); /// The number of bits of entropy should be not less than 128. - for (auto & c : res) - c = distribution(thread_local_rng); - return res; - } - - /* - template - void throwIfError(Aws::Utils::Outcome && response) - { - if (!response.IsSuccess()) - { - const auto & err = response.GetError(); - throw Exception(err.GetMessage(), static_cast(err.GetErrorType())); - } - } - */ - - struct Metadata - { - /// Metadata file version. - static constexpr UInt32 VERSION = 1; - - using PathAndSize = std::pair; - - /// Disk path. - const String & disk_path; - /// Relative path to metadata file on local FS. - String metadata_file_path; - /// Total size of all HDFS objects. - size_t total_size; - /// HDFS objects paths and their sizes. - std::vector hdfs_objects; - /// Number of references (hardlinks) to this metadata file. - UInt32 ref_count; - - /// Load metadata by path or create empty if `create` flag is set. - explicit Metadata(const String & disk_path_, const String & metadata_file_path_, bool create = false) - : disk_path(disk_path_), metadata_file_path(metadata_file_path_), total_size(0), hdfs_objects(0), ref_count(0) - { - if (create) - return; - - DB::ReadBufferFromFile buf(disk_path + metadata_file_path, 1024); /* reasonable buffer size for small file */ - - UInt32 version; - readIntText(version, buf); - - if (version != VERSION) - throw DB::Exception(DB::ErrorCodes::LOGICAL_ERROR, - "Unknown metadata file version. Path: {}. Version: {}. Expected version: {}", - disk_path + metadata_file_path, std::to_string(version), std::to_string(VERSION)); - - assertChar('\n', buf); - - UInt32 hdfs_objects_count; - readIntText(hdfs_objects_count, buf); - assertChar('\t', buf); - readIntText(total_size, buf); - assertChar('\n', buf); - hdfs_objects.resize(hdfs_objects_count); - for (UInt32 i = 0; i < hdfs_objects_count; ++i) - { - String hdfs_object_path; - size_t hdfs_object_size; - readIntText(hdfs_object_size, buf); - assertChar('\t', buf); - readEscapedString(hdfs_object_path, buf); - assertChar('\n', buf); - hdfs_objects[i] = {hdfs_object_path, hdfs_object_size}; - } - - readIntText(ref_count, buf); - assertChar('\n', buf); - } - - void addObject(const String & path, size_t size) - { - total_size += size; - hdfs_objects.emplace_back(path, size); - } - - /// Fsync metadata file if 'sync' flag is set. - void save(bool sync = false) - { - DB::WriteBufferFromFile buf(disk_path + metadata_file_path, 1024); - - writeIntText(VERSION, buf); - writeChar('\n', buf); - - writeIntText(hdfs_objects.size(), buf); - writeChar('\t', buf); - writeIntText(total_size, buf); - writeChar('\n', buf); - for (const auto & [hdfs_object_path, hdfs_object_size] : hdfs_objects) - { - writeIntText(hdfs_object_size, buf); - writeChar('\t', buf); - writeEscapedString(hdfs_object_path, buf); - writeChar('\n', buf); - } - - writeIntText(ref_count, buf); - writeChar('\n', buf); - - buf.finalize(); - if (sync) - buf.sync(); - } - }; - - -} - -namespace DB -{ - -class DiskHDFSDirectoryIterator final : public IDiskDirectoryIterator -{ -public: - DiskHDFSDirectoryIterator(const String & full_path, const String & folder_path_) : iter(full_path), folder_path(folder_path_) {} - - void next() override { ++iter; } - - bool isValid() const override { return iter != Poco::DirectoryIterator(); } - - String path() const override - { - if (iter->isDirectory()) - return folder_path + iter.name() + '/'; - else - return folder_path + iter.name(); - } - - String name() const override { return iter.name(); } - -private: - Poco::DirectoryIterator iter; - String folder_path; -}; - - -class DiskHDFSReservation : public IReservation -{ -public: - DiskHDFSReservation(const DiskHDFSPtr & disk_, UInt64 size_) - : disk(disk_), size(size_), metric_increment(CurrentMetrics::DiskSpaceReservedForMerge, size_) - { - } - - ~DiskHDFSReservation() override - { - try - { - std::lock_guard lock(disk->reservation_mutex); - if (disk->reserved_bytes < size) - { - disk->reserved_bytes = 0; - //LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservations size for disk '{}'", disk->getName()); - } - else - { - disk->reserved_bytes -= size; - } - - if (disk->reservation_count == 0) - { - //LOG_ERROR(&Poco::Logger::get("DiskLocal"), "Unbalanced reservation count for disk '{}'", disk->getName()); - } - else - --disk->reservation_count; - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } - } - - UInt64 getSize() const override { return size; } - - DiskPtr getDisk(size_t /* i */) const override { return disk; } - - void update(UInt64 new_size) override - { - std::lock_guard lock(disk->reservation_mutex); - disk->reserved_bytes -= size; - size = new_size; - disk->reserved_bytes += size; - } - - Disks getDisks() const override { return {}; } - -private: - DiskHDFSPtr disk; - UInt64 size; - CurrentMetrics::Increment metric_increment; -}; - -} diff --git a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h index 702f45291ec..509e183d56b 100644 --- a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h +++ b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h @@ -1,7 +1,7 @@ #pragma once #include -#include "HDFSHelpers.h" +#include "DiskHDFSMetadata.h" namespace DB diff --git a/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h b/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h index 5f6bc353360..348aaef5c69 100644 --- a/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h +++ b/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h @@ -1,6 +1,7 @@ #pragma once + #include -#include "HDFSHelpers.h" +#include "DiskHDFSMetadata.h" namespace DB From e5ebdb0da56ea941a1b9f82635068f429deb4e57 Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Apr 2021 21:47:29 +0000 Subject: [PATCH 0091/1060] Fix fast test --- src/Disks/registerDisks.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 963b3bd8f87..5caea936045 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -13,7 +13,9 @@ void registerDiskMemory(DiskFactory & factory); #if USE_AWS_S3 void registerDiskS3(DiskFactory & factory); #endif +#if USE_HDFS void registerDiskHDFS(DiskFactory & factory); +#endif void registerDisks() From 8d2709b212817a27c03de8a3180e31b61d9263ca Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 2 Apr 2021 22:09:12 +0000 Subject: [PATCH 0092/1060] Fix fast test --- src/Disks/registerDisks.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Disks/registerDisks.cpp b/src/Disks/registerDisks.cpp index 5caea936045..8f4901e49e5 100644 --- a/src/Disks/registerDisks.cpp +++ b/src/Disks/registerDisks.cpp @@ -8,11 +8,14 @@ namespace DB { + void registerDiskLocal(DiskFactory & factory); void registerDiskMemory(DiskFactory & factory); + #if USE_AWS_S3 void registerDiskS3(DiskFactory & factory); #endif + #if USE_HDFS void registerDiskHDFS(DiskFactory & factory); #endif @@ -24,11 +27,14 @@ void registerDisks() registerDiskLocal(factory); registerDiskMemory(factory); - registerDiskHDFS(factory); #if USE_AWS_S3 registerDiskS3(factory); #endif + +#if USE_HDFS + registerDiskHDFS(factory); +#endif } } From a2b6dbe18e3c29d112d6a5d5a8e696b7f4772ab6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 3 Apr 2021 09:05:05 +0000 Subject: [PATCH 0093/1060] Fix build, style, ya-make --- src/Disks/HDFS/DiskHDFS.cpp | 18 +++++++----------- src/Disks/HDFS/DiskHDFSMetadata.h | 5 +++++ src/Disks/HDFS/ReadIndirectBufferFromHDFS.h | 6 ------ src/Disks/tests/gtest_disk.cpp | 6 +++--- src/Disks/tests/gtest_disk.h | 3 ++- src/Disks/ya.make | 2 +- 6 files changed, 18 insertions(+), 22 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index e58212fabda..4051af644c3 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -25,12 +25,8 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int EXCESSIVE_ELEMENT_IN_CONFIG; - extern const int PATH_ACCESS_DENIED; extern const int FILE_ALREADY_EXISTS; - extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int CANNOT_REMOVE_FILE; + extern const int BAD_ARGUMENTS; } @@ -177,7 +173,7 @@ std::unique_ptr DiskHDFS::writeFile(const String & path bool exist = exists(path); /// Path to store new HDFS object. auto file_name = getRandomName(); - auto HDFS_path = hdfs_name + file_name; + auto hdfs_path = hdfs_name + file_name; if (!exist || mode == WriteMode::Rewrite) { /// If metadata file exists - remove and new. @@ -189,9 +185,9 @@ std::unique_ptr DiskHDFS::writeFile(const String & path LOG_DEBUG( &Poco::Logger::get("DiskHDFS"), - "Write to file by path: {}. New HDFS path: {}", backQuote(metadata_path + path), HDFS_path); + "Write to file by path: {}. New hdfs path: {}", backQuote(metadata_path + path), hdfs_path); - return std::make_unique(config, HDFS_path, file_name, metadata, buf_size); + return std::make_unique(config, hdfs_path, file_name, metadata, buf_size); } else { @@ -199,10 +195,10 @@ std::unique_ptr DiskHDFS::writeFile(const String & path LOG_DEBUG( &Poco::Logger::get("DiskHDFS"), - "Append to file by path: {}. New HDFS path: {}. Existing HDFS objects: {}", - backQuote(metadata_path + path), HDFS_path, metadata.hdfs_objects.size()); + "Append to file by path: {}. New hdfs path: {}. Existing HDFS objects: {}", + backQuote(metadata_path + path), hdfs_path, metadata.hdfs_objects.size()); - return std::make_unique(config, HDFS_path, file_name, metadata, buf_size); + return std::make_unique(config, hdfs_path, file_name, metadata, buf_size); } } diff --git a/src/Disks/HDFS/DiskHDFSMetadata.h b/src/Disks/HDFS/DiskHDFSMetadata.h index 16e3fe1c46a..50c6cd23373 100644 --- a/src/Disks/HDFS/DiskHDFSMetadata.h +++ b/src/Disks/HDFS/DiskHDFSMetadata.h @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + struct Metadata { /// Metadata file version. diff --git a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h index 509e183d56b..cabda3a6e11 100644 --- a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h +++ b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h @@ -9,13 +9,7 @@ namespace DB namespace ErrorCodes { - extern const int UNKNOWN_ELEMENT_IN_CONFIG; - extern const int EXCESSIVE_ELEMENT_IN_CONFIG; - extern const int PATH_ACCESS_DENIED; - extern const int FILE_ALREADY_EXISTS; extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int UNKNOWN_FORMAT; - extern const int CANNOT_REMOVE_FILE; } diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index 5875877a14b..d42b3180c92 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -1,5 +1,4 @@ #include - #include #include #include "gtest_disk.h" @@ -9,6 +8,7 @@ # pragma GCC diagnostic ignored "-Wsuggest-override" #endif + template DB::DiskPtr createDisk(); @@ -158,7 +158,7 @@ TEST(DiskHdfsTest, testHdfsCreation) readString(d, *in); EXPECT_EQ("test data", d); } - + { std::unique_ptr out = disk.writeFile("test_file", 1, DB::WriteMode::Rewrite, 1024, 1); writeString("test data", *out); @@ -189,4 +189,4 @@ TEST(DiskHdfsTest, testHdfsCreation) in->readStrict(buf.data(), 4); EXPECT_EQ("data", buf); } -} \ No newline at end of file +} diff --git a/src/Disks/tests/gtest_disk.h b/src/Disks/tests/gtest_disk.h index 8fb685d0a59..a757246e1e4 100644 --- a/src/Disks/tests/gtest_disk.h +++ b/src/Disks/tests/gtest_disk.h @@ -1,7 +1,8 @@ #pragma once + #include #include -#include +#include #include diff --git a/src/Disks/ya.make b/src/Disks/ya.make index 6797c16ee4f..d919f4970c5 100644 --- a/src/Disks/ya.make +++ b/src/Disks/ya.make @@ -13,9 +13,9 @@ SRCS( DiskDecorator.cpp DiskFactory.cpp DiskLocal.cpp - DiskHDFS.cpp DiskMemory.cpp DiskSelector.cpp + HDFS/DiskHDFS.cpp IDisk.cpp IVolume.cpp LocalDirectorySyncGuard.cpp From 8cbfbb5de78d4738449f9986f6c81e60ed774ae6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 3 Apr 2021 12:02:29 +0000 Subject: [PATCH 0094/1060] Small refactoring, add hdfs to test_disk_types --- src/Disks/DiskType.h | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 155 +++++++++--------- src/Disks/HDFS/DiskHDFS.h | 69 ++++---- src/Disks/HDFS/ReadIndirectBufferFromHDFS.h | 4 +- src/Disks/HDFS/WriteIndirectBufferFromHDFS.h | 4 +- src/Storages/HDFS/HDFSCommon.h | 2 +- src/Storages/HDFS/ReadBufferFromHDFS.h | 2 +- .../test_disk_types/configs/storage.xml | 4 + tests/integration/test_disk_types/test.py | 4 +- 9 files changed, 126 insertions(+), 120 deletions(-) diff --git a/src/Disks/DiskType.h b/src/Disks/DiskType.h index f0c22eb6495..a5c23fe2c2c 100644 --- a/src/Disks/DiskType.h +++ b/src/Disks/DiskType.h @@ -25,7 +25,7 @@ struct DiskType case Type::S3: return "s3"; case Type::HDFS: - return "HDFS"; + return "hdfs"; } __builtin_unreachable(); } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 4051af644c3..3556b95558d 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -31,28 +31,22 @@ namespace ErrorCodes DiskHDFS::DiskHDFS( - const Poco::Util::AbstractConfiguration & config_, const String & name_, const String & hdfs_name_, - const String & metadata_path_) - : config(config_) + const String & metadata_path_, + const Context & context_) + : log(&Poco::Logger::get("DiskHDFS")) , name(name_) , hdfs_name(hdfs_name_) , metadata_path(std::move(metadata_path_)) + , context(context_) + , config(context.getGlobalContext().getConfigRef()) , builder(createHDFSBuilder(hdfs_name, config)) , fs(createHDFSFS(builder.get())) { } -ReservationPtr DiskHDFS::reserve(UInt64 bytes) -{ - if (!tryReserve(bytes)) - return {}; - return std::make_unique(std::static_pointer_cast(shared_from_this()), bytes); -} - - bool DiskHDFS::exists(const String & path) const { return Poco::File(metadata_path + path).exists(); @@ -108,6 +102,7 @@ void DiskHDFS::moveFile(const String & from_path, const String & to_path) { if (exists(to_path)) throw Exception("File already exists: " + to_path, ErrorCodes::FILE_ALREADY_EXISTS); + Poco::File(metadata_path + from_path).renameTo(metadata_path + to_path); } @@ -128,43 +123,15 @@ void DiskHDFS::replaceFile(const String & from_path, const String & to_path) } -//void DiskHDFS::copyFile(const String & from_path, const String & to_path) -//{ -// if (exists(to_path)) -// remove(to_path); -// -// Metadata from(metadata_path, from_path); -// Metadata to(metadata_path, to_path, true); -// -// for (const auto & [path, size] : from.hdfs_objects) -// { -// auto new_path = hdfs_name + getRandomName(); -// /// TODO:: hdfs copy semantics -// /* -// Aws::HDFS::Model::CopyObjectRequest req; -// req.SetCopySource(bucket + "/" + path); -// req.SetBucket(bucket); -// req.SetKey(new_path); -// throwIfError(client->CopyObject(req)); -// */ -// throw Exception("is not implemented yet", 1); -// to.addObject(new_path, size); -// } -// -// to.save(); -//} - - std::unique_ptr DiskHDFS::readFile(const String & path, size_t buf_size, size_t, size_t, size_t, MMappedFileCache *) const { Metadata metadata(metadata_path, path); - LOG_DEBUG( - &Poco::Logger::get("DiskHDFS"), + LOG_DEBUG(log, "Read from file by path: {}. Existing HDFS objects: {}", backQuote(metadata_path + path), metadata.hdfs_objects.size()); - return std::make_unique(config, hdfs_name, "", metadata, buf_size); + return std::make_unique(context, hdfs_name, "", metadata, buf_size); } @@ -183,22 +150,20 @@ std::unique_ptr DiskHDFS::writeFile(const String & path /// Save empty metadata to disk to have ability to get file size while buffer is not finalized. metadata.save(); - LOG_DEBUG( - &Poco::Logger::get("DiskHDFS"), + LOG_DEBUG(log, "Write to file by path: {}. New hdfs path: {}", backQuote(metadata_path + path), hdfs_path); - return std::make_unique(config, hdfs_path, file_name, metadata, buf_size); + return std::make_unique(context, hdfs_path, file_name, metadata, buf_size); } else { Metadata metadata(metadata_path, path); - LOG_DEBUG( - &Poco::Logger::get("DiskHDFS"), + LOG_DEBUG(log, "Append to file by path: {}. New hdfs path: {}. Existing HDFS objects: {}", backQuote(metadata_path + path), hdfs_path, metadata.hdfs_objects.size()); - return std::make_unique(config, hdfs_path, file_name, metadata, buf_size); + return std::make_unique(context, hdfs_path, file_name, metadata, buf_size); } } @@ -255,33 +220,6 @@ void DiskHDFS::removeRecursive(const String & path) } -bool DiskHDFS::tryReserve(UInt64 bytes) -{ - std::lock_guard lock(reservation_mutex); - if (bytes == 0) - { - LOG_DEBUG(&Poco::Logger::get("DiskHDFS"), "Reserving 0 bytes on HDFS disk {}", backQuote(name)); - ++reservation_count; - return true; - } - - auto available_space = getAvailableSpace(); - UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); - if (unreserved_space >= bytes) - { - LOG_DEBUG( - &Poco::Logger::get("DiskHDFS"), - "Reserving {} on disk {}, having unreserved ", - formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); - - ++reservation_count; - reserved_bytes += bytes; - return true; - } - return false; -} - - void DiskHDFS::listFiles(const String & path, std::vector & file_names) { for (auto it = iterateDirectory(path); it->isValid(); it->next()) @@ -327,6 +265,41 @@ void DiskHDFS::setReadOnly(const String & path) } +ReservationPtr DiskHDFS::reserve(UInt64 bytes) +{ + if (!tryReserve(bytes)) + return {}; + return std::make_unique(std::static_pointer_cast(shared_from_this()), bytes); +} + + +bool DiskHDFS::tryReserve(UInt64 bytes) +{ + std::lock_guard lock(reservation_mutex); + + if (bytes == 0) + { + LOG_DEBUG(log, "Reserving 0 bytes on HDFS disk {}", backQuote(name)); + ++reservation_count; + return true; + } + + auto available_space = getAvailableSpace(); + UInt64 unreserved_space = available_space - std::min(available_space, reserved_bytes); + if (unreserved_space >= bytes) + { + LOG_DEBUG(log, + "Reserving {} on disk {}, having unreserved {}", + formatReadableSizeWithBinarySuffix(bytes), backQuote(name), formatReadableSizeWithBinarySuffix(unreserved_space)); + ++reservation_count; + reserved_bytes += bytes; + return true; + } + + return false; +} + + void registerDiskHDFS(DiskFactory & factory) { auto creator = [](const String & name, @@ -334,21 +307,45 @@ void registerDiskHDFS(DiskFactory & factory) const String & config_prefix, const Context & context) -> DiskPtr { - const auto * disk_config = config.createView(config_prefix); - Poco::File disk{context.getPath() + "disks/" + name}; disk.createDirectories(); - DB::String uri{disk_config->getString("endpoint")}; + + DB::String uri{config.getString(config_prefix + ".endpoint")}; if (uri.back() != '/') throw Exception(ErrorCodes::BAD_ARGUMENTS, "HDFS path must ends with '/', but '{}' doesn't.", uri); String metadata_path = context.getPath() + "disks/" + name + "/"; - - return std::make_shared(config, name, uri, metadata_path); + return std::make_shared(name, uri, metadata_path, context); }; factory.registerDiskType("hdfs", creator); } +//void DiskHDFS::copyFile(const String & from_path, const String & to_path) +//{ +// if (exists(to_path)) +// remove(to_path); +// +// Metadata from(metadata_path, from_path); +// Metadata to(metadata_path, to_path, true); +// +// for (const auto & [path, size] : from.hdfs_objects) +// { +// auto new_path = hdfs_name + getRandomName(); +// /// TODO:: hdfs copy semantics +// /* +// Aws::HDFS::Model::CopyObjectRequest req; +// req.SetCopySource(bucket + "/" + path); +// req.SetBucket(bucket); +// req.SetKey(new_path); +// throwIfError(client->CopyObject(req)); +// */ +// throw Exception("is not implemented yet", 1); +// to.addObject(new_path, size); +// } +// +// to.save(); +//} + } diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 65a35f3e2bd..26dc6bf85fe 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -4,7 +4,7 @@ #include #include -#include +#include // Y_IGNORE #include @@ -24,25 +24,15 @@ friend class DiskHDFSReservation; public: DiskHDFS( - const Poco::Util::AbstractConfiguration & config, const String & name_, const String & hdfs_name_, - const String & metadata_path_); - - const String & getName() const override { return name; } - - ReservationPtr reserve(UInt64 bytes) override; - - void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; - - Poco::Timestamp getLastModified(const String & path) override; - - void setReadOnly(const String & path) override; - - void createHardLink(const String & src_path, const String & dst_path) override; + const String & metadata_path_, + const Context & context_); DiskType::Type getType() const override { return DiskType::Type::HDFS; } + const String & getName() const override { return name; } + const String & getPath() const override { return metadata_path; } UInt64 getTotalSpace() const override { return std::numeric_limits::max(); } @@ -81,6 +71,24 @@ public: void listFiles(const String & path, std::vector & file_names) override; + void removeFile(const String & /* path */) override {} + + void removeFileIfExists(const String & /* path */) override {} + + void removeDirectory(const String & /* path */) override {} + + void removeRecursive(const String & path) override; + + ReservationPtr reserve(UInt64 bytes) override; + + void setLastModified(const String & path, const Poco::Timestamp & timestamp) override; + + Poco::Timestamp getLastModified(const String & path) override; + + void setReadOnly(const String & path) override; + + void createHardLink(const String & src_path, const String & dst_path) override; + std::unique_ptr readFile( const String & path, size_t buf_size, @@ -94,32 +102,27 @@ public: size_t buf_size, WriteMode mode) override; - void removeFile(const String & /* path */) override {} +private: + String getRandomName() { return toString(UUIDHelpers::generateV4()); } - void removeFileIfExists(const String & /* path */) override {} + bool tryReserve(UInt64 bytes); - void removeDirectory(const String & /* path */) override {} + void remove(const String & path); - void removeRecursive(const String & path) override; + Poco::Logger * log; + const String name; + const String hdfs_name; + String metadata_path; + Context context; + const Poco::Util::AbstractConfiguration & config; + + HDFSBuilderWrapper builder; + HDFSFSPtr fs; UInt64 reserved_bytes = 0; UInt64 reservation_count = 0; std::mutex reservation_mutex; std::mutex copying_mutex; - -private: - String getRandomName() { return toString(UUIDHelpers::generateV4()); } - - bool tryReserve(UInt64 bytes); - void remove(const String & path); - - const Poco::Util::AbstractConfiguration & config; - const String name; - const String hdfs_name; - String metadata_path; - - HDFSBuilderWrapper builder; - HDFSFSPtr fs; }; using DiskHDFSPtr = std::shared_ptr; diff --git a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h index cabda3a6e11..188ef22c524 100644 --- a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h +++ b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h @@ -18,12 +18,12 @@ class ReadIndirectBufferFromHDFS final : public ReadBufferFromFileBase { public: ReadIndirectBufferFromHDFS( - const Poco::Util::AbstractConfiguration & config_, + const Context & context, const String & hdfs_name_, const String & /* bucket */, Metadata metadata_, size_t buf_size_) - : config(config_) + : config(context.getGlobalContext().getConfigRef()) , hdfs_name(hdfs_name_) , metadata(std::move(metadata_)) , buf_size(buf_size_) diff --git a/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h b/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h index 348aaef5c69..9c28ac0665f 100644 --- a/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h +++ b/src/Disks/HDFS/WriteIndirectBufferFromHDFS.h @@ -12,13 +12,13 @@ class WriteIndirectBufferFromHDFS final : public WriteBufferFromFileBase { public: WriteIndirectBufferFromHDFS( - const Poco::Util::AbstractConfiguration & config, + const Context & context, const String & hdfs_name_, const String & hdfs_path_, Metadata metadata_, size_t buf_size_) : WriteBufferFromFileBase(buf_size_, nullptr, 0) - , impl(WriteBufferFromHDFS(hdfs_name_, config, buf_size_)) + , impl(WriteBufferFromHDFS(hdfs_name_, context.getGlobalContext().getConfigRef(), buf_size_)) , metadata(std::move(metadata_)) , hdfs_path(hdfs_path_) { diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/HDFS/HDFSCommon.h index fa1ca88464e..5f227df7fd2 100644 --- a/src/Storages/HDFS/HDFSCommon.h +++ b/src/Storages/HDFS/HDFSCommon.h @@ -7,7 +7,7 @@ #include #include -#include +#include // Y_IGNORE #include #include diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index 8d26c001b2e..b7985dd885e 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -8,7 +8,7 @@ #include #include -#include +#include // Y_IGNORE #include diff --git a/tests/integration/test_disk_types/configs/storage.xml b/tests/integration/test_disk_types/configs/storage.xml index 2bf9a2e363a..1167a4f7382 100644 --- a/tests/integration/test_disk_types/configs/storage.xml +++ b/tests/integration/test_disk_types/configs/storage.xml @@ -11,6 +11,10 @@ memory + + hdfs + http://hdfs1:9000/data/ + diff --git a/tests/integration/test_disk_types/test.py b/tests/integration/test_disk_types/test.py index ad09519a484..3f1a656d98f 100644 --- a/tests/integration/test_disk_types/test.py +++ b/tests/integration/test_disk_types/test.py @@ -5,6 +5,7 @@ disk_types = { "default": "local", "disk_s3": "s3", "disk_memory": "memory", + "disk_hdfs": "hdfs", } @@ -12,7 +13,7 @@ disk_types = { def cluster(): try: cluster = ClickHouseCluster(__file__) - cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True) + cluster.add_instance("node", main_configs=["configs/storage.xml"], with_minio=True, with_hdfs=True) cluster.start() yield cluster finally: @@ -35,3 +36,4 @@ def test_select_by_type(cluster): node = cluster.instances["node"] for name, disk_type in list(disk_types.items()): assert node.query("SELECT name FROM system.disks WHERE type='" + disk_type + "'") == name + "\n" + From cbd678d031652c7ff777e363acfbab9302a72de3 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sat, 3 Apr 2021 12:10:49 +0000 Subject: [PATCH 0095/1060] Remove test --- src/Disks/tests/gtest_disk.cpp | 46 ---------------------------------- 1 file changed, 46 deletions(-) diff --git a/src/Disks/tests/gtest_disk.cpp b/src/Disks/tests/gtest_disk.cpp index d42b3180c92..907bf5dd294 100644 --- a/src/Disks/tests/gtest_disk.cpp +++ b/src/Disks/tests/gtest_disk.cpp @@ -144,49 +144,3 @@ TYPED_TEST(DiskTest, iterateDirectory) EXPECT_FALSE(iter->isValid()); } } - -TEST(DiskHdfsTest, testHdfsCreation) -{ - auto disk = DB::DiskHDFS("gtesthdfs", "hdfs://localhost:9010/gtest/", "/home/ershov-ov/metadata/"); - { - auto out = disk.writeFile("keek", 1024, DB::WriteMode::Rewrite, 1024, 1024); - writeString("test data", *out); - } - { - DB::String d; - auto in = disk.readFile("keek", 1024, 1024, 1024, 1024); - readString(d, *in); - EXPECT_EQ("test data", d); - } - - { - std::unique_ptr out = disk.writeFile("test_file", 1, DB::WriteMode::Rewrite, 1024, 1); - writeString("test data", *out); - } - - // Test SEEK_SET - { - String buf(4, '0'); - std::unique_ptr in = disk.readFile("test_file", 1, 1024, 1024, 1); - - in->seek(5, SEEK_SET); - - in->readStrict(buf.data(), 4); - EXPECT_EQ("data", buf); - } - - // Test SEEK_CUR - { - std::unique_ptr in = disk.readFile("test_file", 1, 1024, 1024, 1); - String buf(4, '0'); - - in->readStrict(buf.data(), 4); - EXPECT_EQ("test", buf); - - // Skip whitespace - in->seek(1, SEEK_CUR); - - in->readStrict(buf.data(), 4); - EXPECT_EQ("data", buf); - } -} From 388a71e6b151410e34951841f293aefda295cb5e Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 4 Apr 2021 07:33:07 +0000 Subject: [PATCH 0096/1060] Fix build, ya check --- src/Disks/HDFS/DiskHDFS.h | 2 +- src/Disks/tests/gtest_disk.h | 1 - src/Storages/HDFS/HDFSCommon.h | 2 ++ 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/HDFS/DiskHDFS.h b/src/Disks/HDFS/DiskHDFS.h index 26dc6bf85fe..02729734842 100644 --- a/src/Disks/HDFS/DiskHDFS.h +++ b/src/Disks/HDFS/DiskHDFS.h @@ -95,7 +95,7 @@ public: size_t estimated_size, size_t aio_threshold, size_t mmap_threshold, - MMappedFileCache * mmap_cache = nullptr) const override; + MMappedFileCache * mmap_cache) const override; std::unique_ptr writeFile( const String & path, diff --git a/src/Disks/tests/gtest_disk.h b/src/Disks/tests/gtest_disk.h index a757246e1e4..314caf288a2 100644 --- a/src/Disks/tests/gtest_disk.h +++ b/src/Disks/tests/gtest_disk.h @@ -2,7 +2,6 @@ #include #include -#include #include diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/HDFS/HDFSCommon.h index 5f227df7fd2..427d4d8f1c0 100644 --- a/src/Storages/HDFS/HDFSCommon.h +++ b/src/Storages/HDFS/HDFSCommon.h @@ -1,6 +1,8 @@ #pragma once +#if !defined(ARCADIA_BUILD) #include +#endif #if USE_HDFS #include From 908d872b76e1ad446ffe5acd62cf84443da39374 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 4 Apr 2021 09:08:09 +0000 Subject: [PATCH 0097/1060] Add seek method --- src/Disks/HDFS/ReadIndirectBufferFromHDFS.h | 4 +- src/Disks/ya.make | 1 - src/Disks/ya.make.in | 2 +- src/Storages/HDFS/HDFSCommon.h | 46 ++++++------ src/Storages/HDFS/ReadBufferFromHDFS.cpp | 80 +++++++++++++++++---- src/Storages/HDFS/ReadBufferFromHDFS.h | 19 +++-- src/Storages/HDFS/WriteBufferFromHDFS.h | 10 ++- 7 files changed, 113 insertions(+), 49 deletions(-) diff --git a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h index 188ef22c524..ef0b9175c6e 100644 --- a/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h +++ b/src/Disks/HDFS/ReadIndirectBufferFromHDFS.h @@ -86,11 +86,11 @@ private: if (size > offset) { auto buf = std::make_unique(hdfs_name + path, config, buf_size); - //buf->seek(offset, SEEK_SET); + buf->seek(offset, SEEK_SET); return buf; } - offset -= size; + offset -= size; } return nullptr; diff --git a/src/Disks/ya.make b/src/Disks/ya.make index d919f4970c5..53dc9fd75c4 100644 --- a/src/Disks/ya.make +++ b/src/Disks/ya.make @@ -15,7 +15,6 @@ SRCS( DiskLocal.cpp DiskMemory.cpp DiskSelector.cpp - HDFS/DiskHDFS.cpp IDisk.cpp IVolume.cpp LocalDirectorySyncGuard.cpp diff --git a/src/Disks/ya.make.in b/src/Disks/ya.make.in index ce205bd25ac..13569d8a19d 100644 --- a/src/Disks/ya.make.in +++ b/src/Disks/ya.make.in @@ -8,7 +8,7 @@ PEERDIR( SRCS( - + ) END() diff --git a/src/Storages/HDFS/HDFSCommon.h b/src/Storages/HDFS/HDFSCommon.h index 427d4d8f1c0..09b806956bc 100644 --- a/src/Storages/HDFS/HDFSCommon.h +++ b/src/Storages/HDFS/HDFSCommon.h @@ -53,6 +53,27 @@ struct HDFSFileInfo class HDFSBuilderWrapper { + +friend HDFSBuilderWrapper createHDFSBuilder(const String & uri_str, const Poco::Util::AbstractConfiguration &); + +public: + HDFSBuilderWrapper() : hdfs_builder(hdfsNewBuilder()) {} + + ~HDFSBuilderWrapper() + { + hdfsFreeBuilder(hdfs_builder); + + } + + HDFSBuilderWrapper(const HDFSBuilderWrapper &) = delete; + HDFSBuilderWrapper(HDFSBuilderWrapper &&) = default; + + hdfsBuilder * get() + { + return hdfs_builder; + } + +private: hdfsBuilder * hdfs_builder; String hadoop_kerberos_keytab; String hadoop_kerberos_principal; @@ -73,37 +94,12 @@ class HDFSBuilderWrapper static const String CONFIG_PREFIX; -private: - void loadFromConfig(const Poco::Util::AbstractConfiguration & config, const String & config_path, bool isUser = false); String getKinitCmd(); void runKinit(); -public: - - hdfsBuilder * - get() - { - return hdfs_builder; - } - - HDFSBuilderWrapper() - : hdfs_builder(hdfsNewBuilder()) - { - } - - ~HDFSBuilderWrapper() - { - hdfsFreeBuilder(hdfs_builder); - - } - - HDFSBuilderWrapper(const HDFSBuilderWrapper &) = delete; - HDFSBuilderWrapper(HDFSBuilderWrapper &&) = default; - - friend HDFSBuilderWrapper createHDFSBuilder(const String & uri_str, const Poco::Util::AbstractConfiguration &); }; using HDFSFSPtr = std::unique_ptr, detail::HDFSFsDeleter>; diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.cpp b/src/Storages/HDFS/ReadBufferFromHDFS.cpp index affb76314b1..5774e7784b0 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.cpp +++ b/src/Storages/HDFS/ReadBufferFromHDFS.cpp @@ -12,6 +12,9 @@ namespace ErrorCodes { extern const int NETWORK_ERROR; extern const int CANNOT_OPEN_FILE; + extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int SEEK_POSITION_OUT_OF_BOUND; + extern const int CANNOT_TELL_THROUGH_FILE; } ReadBufferFromHDFS::~ReadBufferFromHDFS() = default; @@ -26,6 +29,9 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl HDFSBuilderWrapper builder; HDFSFSPtr fs; + off_t offset = 0; + bool initialized = false; + ReadBufferFromHDFSImpl(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_) : hdfs_uri(hdfs_name_), @@ -39,17 +45,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl fin = hdfsOpenFile(fs.get(), path.c_str(), O_RDONLY, 0, 0, 0); if (fin == nullptr) - throw Exception("Unable to open HDFS file: " + path + " error: " + std::string(hdfsGetLastError()), - ErrorCodes::CANNOT_OPEN_FILE); - } - - int read(char * start, size_t size) const - { - int bytes_read = hdfsRead(fs.get(), fin, start, size); - if (bytes_read < 0) - throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), - ErrorCodes::NETWORK_ERROR); - return bytes_read; + throw Exception(ErrorCodes::CANNOT_OPEN_FILE, + "Unable to open HDFS file: {}, error: {}", path, std::string(hdfsGetLastError())); } ~ReadBufferFromHDFSImpl() @@ -57,6 +54,53 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl std::lock_guard lock(hdfs_init_mutex); hdfsCloseFile(fs.get(), fin); } + + void initialize() const + { + if (!offset) + return; + + int seek_status = hdfsSeek(fs.get(), fin, offset); + if (seek_status != 0) + throw Exception(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Fail to seek HDFS file: {}, error: {}", hdfs_uri, std::string(hdfsGetLastError())); + } + + int read(char * start, size_t size) + { + if (!initialized) + { + initialize(); + initialized = true; + } + + int bytes_read = hdfsRead(fs.get(), fin, start, size); + if (bytes_read < 0) + throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), + ErrorCodes::NETWORK_ERROR); + + return bytes_read; + } + + int seek(off_t offset_, int whence) + { + 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); + + if (offset_ < 0) + throw Exception(ErrorCodes::SEEK_POSITION_OUT_OF_BOUND, "Seek position is out of bounds. Offset: {}", std::to_string(offset_)); + + offset = offset_; + + return offset; + } + + int tell() const + { + return offset; + } }; @@ -65,7 +109,7 @@ std::mutex ReadBufferFromHDFS::ReadBufferFromHDFSImpl::hdfs_init_mutex; ReadBufferFromHDFS::ReadBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration & config_, size_t buf_size_) - : BufferWithOwnMemory(buf_size_) + : BufferWithOwnMemory(buf_size_) , impl(std::make_unique(hdfs_name_, config_)) { } @@ -82,6 +126,18 @@ bool ReadBufferFromHDFS::nextImpl() return true; } + +off_t ReadBufferFromHDFS::seek(off_t off, int whence) +{ + return impl->seek(off, whence); +} + + +off_t ReadBufferFromHDFS::getPosition() +{ + return impl->tell() + count(); +} + } #endif diff --git a/src/Storages/HDFS/ReadBufferFromHDFS.h b/src/Storages/HDFS/ReadBufferFromHDFS.h index b7985dd885e..61436f9deb9 100644 --- a/src/Storages/HDFS/ReadBufferFromHDFS.h +++ b/src/Storages/HDFS/ReadBufferFromHDFS.h @@ -7,28 +7,35 @@ #include #include #include - #include // Y_IGNORE - #include - #include +#include namespace DB { + /** Accepts HDFS path to file and opens it. * Closes file by himself (thus "owns" a file descriptor). */ -class ReadBufferFromHDFS : public BufferWithOwnMemory +class ReadBufferFromHDFS : public BufferWithOwnMemory { - struct ReadBufferFromHDFSImpl; - std::unique_ptr impl; + public: ReadBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration &, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + ~ReadBufferFromHDFS() override; bool nextImpl() override; + + off_t seek(off_t offset_, int whence) override; + + off_t getPosition() override; + +private: + struct ReadBufferFromHDFSImpl; + std::unique_ptr impl; }; } #endif diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.h b/src/Storages/HDFS/WriteBufferFromHDFS.h index 339ec0c2a7c..20b7b3c8348 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.h +++ b/src/Storages/HDFS/WriteBufferFromHDFS.h @@ -8,6 +8,7 @@ #include #include + namespace DB { /** Accepts HDFS path to file and opens it. @@ -15,10 +16,10 @@ namespace DB */ class WriteBufferFromHDFS : public BufferWithOwnMemory { - struct WriteBufferFromHDFSImpl; - std::unique_ptr impl; + public: WriteBufferFromHDFS(const std::string & hdfs_name_, const Poco::Util::AbstractConfiguration &, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE); + WriteBufferFromHDFS(WriteBufferFromHDFS &&) = default; void nextImpl() override; @@ -26,6 +27,11 @@ public: ~WriteBufferFromHDFS() override; void sync() override; + +private: + struct WriteBufferFromHDFSImpl; + std::unique_ptr impl; }; + } #endif From 2f73dc254c087c77c908777ff469ef559622b966 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 5 Apr 2021 09:33:08 +0000 Subject: [PATCH 0098/1060] Add integration test --- tests/integration/test_disk_hdfs/__init__.py | 0 .../test_disk_hdfs/configs/log_conf.xml | 11 ++++++++ .../test_disk_hdfs/configs/storage.xml | 21 +++++++++++++++ tests/integration/test_disk_hdfs/test.py | 27 +++++++++++++++++++ 4 files changed, 59 insertions(+) create mode 100644 tests/integration/test_disk_hdfs/__init__.py create mode 100644 tests/integration/test_disk_hdfs/configs/log_conf.xml create mode 100644 tests/integration/test_disk_hdfs/configs/storage.xml create mode 100644 tests/integration/test_disk_hdfs/test.py diff --git a/tests/integration/test_disk_hdfs/__init__.py b/tests/integration/test_disk_hdfs/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_disk_hdfs/configs/log_conf.xml b/tests/integration/test_disk_hdfs/configs/log_conf.xml new file mode 100644 index 00000000000..f9d15e572aa --- /dev/null +++ b/tests/integration/test_disk_hdfs/configs/log_conf.xml @@ -0,0 +1,11 @@ + + + trace + /var/log/clickhouse-server/log.log + /var/log/clickhouse-server/log.err.log + 1000M + 10 + /var/log/clickhouse-server/stderr.log + /var/log/clickhouse-server/stdout.log + + diff --git a/tests/integration/test_disk_hdfs/configs/storage.xml b/tests/integration/test_disk_hdfs/configs/storage.xml new file mode 100644 index 00000000000..92afa92cb14 --- /dev/null +++ b/tests/integration/test_disk_hdfs/configs/storage.xml @@ -0,0 +1,21 @@ + + + + + + hdfs + hdfs://hdfs1:9000/ + + + + + + +
+ hdfs +
+
+
+
+
+
diff --git a/tests/integration/test_disk_hdfs/test.py b/tests/integration/test_disk_hdfs/test.py new file mode 100644 index 00000000000..1336bc539d4 --- /dev/null +++ b/tests/integration/test_disk_hdfs/test.py @@ -0,0 +1,27 @@ +import os +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.hdfs_api import HDFSApi + +cluster = ClickHouseCluster(__file__) +node1 = cluster.add_instance('node1', main_configs=[ + 'configs/storage.xml', + 'configs/log_conf.xml'], with_hdfs=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_read_write(started_cluster): + node1.query("DROP TABLE IF EXISTS simple_test") + node1.query("CREATE TABLE simple_test (id UInt64) Engine=TinyLog SETTINGS disk = 'hdfs'") + node1.query("INSERT INTO simple_test SELECT number FROM numbers(3)") + node1.query("INSERT INTO simple_test SELECT number FROM numbers(3, 3)") + assert node1.query("SELECT * FROM simple_test") == "0\n1\n2\n3\n4\n5\n" + From 9115fd8cecd2ce95e59eae1495b50fd252913e9b Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Wed, 7 Apr 2021 15:22:53 +0300 Subject: [PATCH 0099/1060] wip --- .../compose/docker_compose_cassandra.yml | 4 +- .../compose/docker_compose_postgres.yml | 4 +- tests/integration/helpers/cluster.py | 53 ++++++++++++++----- tests/integration/helpers/external_sources.py | 3 ++ tests/integration/parallel.txt | 42 +++++++++++++++ .../test_alter_on_mixed_type_cluster/test.py | 12 ++--- .../test_concurrent_ttl_merges/test.py | 4 +- .../test_cross_replication/test.py | 2 +- .../test_ddl_worker_non_leader/test.py | 6 +-- .../test_cassandra.py | 2 +- 10 files changed, 101 insertions(+), 31 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_cassandra.yml b/docker/test/integration/runner/compose/docker_compose_cassandra.yml index e8aed06c08a..b6190a11d73 100644 --- a/docker/test/integration/runner/compose/docker_compose_cassandra.yml +++ b/docker/test/integration/runner/compose/docker_compose_cassandra.yml @@ -1,7 +1,5 @@ version: '2.3' services: cassandra1: - image: cassandra + image: cassandra:4.0 restart: always - ports: - - ${CASSANDRA_EXTERNAL_PORT}:${CASSANDRA_INTERNAL_PORT} diff --git a/docker/test/integration/runner/compose/docker_compose_postgres.yml b/docker/test/integration/runner/compose/docker_compose_postgres.yml index 3b760e2f84b..a2f205c7afd 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres.yml @@ -2,6 +2,7 @@ version: '2.3' services: postgres1: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always expose: - ${POSTGRES_PORT} @@ -24,6 +25,7 @@ services: target: /postgres/ postgres2: image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] restart: always environment: POSTGRES_HOST_AUTH_METHOD: "trust" @@ -34,4 +36,4 @@ services: volumes: - type: ${POSTGRES_LOGS_FS:-tmpfs} source: ${POSTGRES2_DIR:-} - target: /postgres/ \ No newline at end of file + target: /postgres/ diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 0a64f8fdc7b..3f26135dccd 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -16,6 +16,7 @@ import traceback import urllib.parse import shlex +from cassandra.policies import RoundRobinPolicy import cassandra.cluster import psycopg2 import pymongo @@ -99,7 +100,6 @@ def get_docker_compose_path(): logging.debug(f"Fallback docker_compose_path to LOCAL_DOCKER_COMPOSE_DIR: {LOCAL_DOCKER_COMPOSE_DIR}") return LOCAL_DOCKER_COMPOSE_DIR - def check_kafka_is_available(kafka_id, kafka_port): p = subprocess.Popen(('docker', 'exec', @@ -225,7 +225,9 @@ class ClickHouseCluster: # available when with_cassandra == True self.cassandra_host = "cassandra1" - self.cassandra_port = get_open_port() + self.cassandra_port = 9042 + self.cassandra_ip = None + self.cassandra_id = self.get_instance_docker_id(self.cassandra_host) # available when with_rabbitmq == True self.rabbitmq_host = "rabbitmq1" @@ -551,8 +553,7 @@ class ClickHouseCluster: if with_cassandra and not self.with_cassandra: self.with_cassandra = True - env_variables['CASSANDRA_EXTERNAL_PORT'] = str(self.cassandra_port) - env_variables['CASSANDRA_INTERNAL_PORT'] = "9042" + env_variables['CASSANDRA_PORT'] = str(self.cassandra_port) self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')]) self.base_cassandra_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, '--file', p.join(docker_compose_yml_dir, 'docker_compose_cassandra.yml')] @@ -778,7 +779,7 @@ class ClickHouseCluster: logging.debug("Can't connect to Mongo " + str(ex)) time.sleep(1) - def wait_minio_to_start(self, timeout=30, secure=False): + def wait_minio_to_start(self, timeout=120, secure=False): os.environ['SSL_CERT_FILE'] = p.join(self.base_dir, self.minio_dir, 'certs', 'public.crt') minio_client = Minio('localhost:{}'.format(self.minio_port), access_key='minio', @@ -819,18 +820,27 @@ class ClickHouseCluster: logging.debug(("Can't connect to SchemaRegistry: %s", str(ex))) time.sleep(1) - def wait_cassandra_to_start(self, timeout=30): - cass_client = cassandra.cluster.Cluster(["localhost"], self.cassandra_port) + raise Exception("Can't wait Schema Registry to start") + + + def wait_cassandra_to_start(self, timeout=120): + self.cassandra_ip = self.get_instance_ip(self.cassandra_host) + cass_client = cassandra.cluster.Cluster([self.cassandra_ip], port=self.cassandra_port, load_balancing_policy=RoundRobinPolicy()) start = time.time() while time.time() - start < timeout: try: + logging.info(f"Check Cassandra Online {self.cassandra_id} {self.cassandra_ip} {self.cassandra_port}") + check = self.exec_in_container(self.cassandra_id, ["bash", "-c", f"/opt/cassandra/bin/cqlsh -u cassandra -p cassandra -e 'describe keyspaces' {self.cassandra_ip} {self.cassandra_port}"], user='root') + logging.info("Cassandra Online") cass_client.connect() - logging.info("Connected to Cassandra") + logging.info("Connected Clients to Cassandra") return except Exception as ex: logging.warning("Can't connect to Cassandra: %s", str(ex)) time.sleep(1) + raise Exception("Can't wait Cassandra to start") + def start(self, destroy_dirs=True): logging.debug("Cluster start called. is_up={}, destroy_dirs={}".format(self.is_up, destroy_dirs)) if self.is_up: @@ -844,6 +854,8 @@ class ClickHouseCluster: if not subprocess_call(['docker-compose', 'kill']): subprocess_call(['docker-compose', 'down', '--volumes']) logging.debug("Unstopped containers killed") + subprocess_call(['docker-compose', 'ps', '--services', '--all']) + except: pass @@ -895,7 +907,7 @@ class ClickHouseCluster: os.makedirs(self.mysql_logs_dir) os.chmod(self.mysql_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_mysql_cmd + common_opts) - self.wait_mysql_to_start(120) + self.wait_mysql_to_start(180) if self.with_mysql8 and self.base_mysql8_cmd: logging.debug('Setup MySQL 8') @@ -916,8 +928,7 @@ class ClickHouseCluster: os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) - self.wait_postgres_to_start(30) - self.wait_postgres_to_start(30) + self.wait_postgres_to_start(120) if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') @@ -987,7 +998,7 @@ class ClickHouseCluster: subprocess_check_call(clickhouse_start_cmd) logging.debug("ClickHouse instance created") - start_deadline = time.time() + 120.0 # seconds + start_deadline = time.time() + 180.0 # seconds for instance in self.instances.values(): instance.docker_client = self.docker_client instance.ip_address = self.get_instance_ip(instance.name) @@ -1334,8 +1345,20 @@ class ClickHouseInstance: if not self.stay_alive: raise Exception("clickhouse can be stopped only with stay_alive=True instance") - self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root') - time.sleep(start_wait_sec) + try: + ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], user='root') + if ps_clickhouse == " PID TTY STAT TIME COMMAND" : + logging.warning("ClickHouse process already stopped") + return + + self.exec_in_container(["bash", "-c", "pkill {} clickhouse".format("-9" if kill else "")], user='root') + time.sleep(start_wait_sec) + ps_clickhouse = self.exec_in_container(["bash", "-c", "ps -C clickhouse"], user='root') + if ps_clickhouse != " PID TTY STAT TIME COMMAND" : + logging.warning(f"Force kill clickhouse in stop_clickhouse. ps:{ps_clickhouse}") + self.stop_clickhouse(kill=True) + except Exception as e: + logging.warning(f"Stop ClickHouse raised an error {e}") def start_clickhouse(self, stop_wait_sec=5): if not self.stay_alive: @@ -1360,8 +1383,10 @@ class ClickHouseInstance: return len(result) > 0 def grep_in_log(self, substring): + logging.debug(f"grep in log called {substring}") result = self.exec_in_container( ["bash", "-c", 'grep "{}" /var/log/clickhouse-server/clickhouse-server.log || true'.format(substring)]) + logging.debug(f"grep result {result}") return result def count_in_log(self, substring): diff --git a/tests/integration/helpers/external_sources.py b/tests/integration/helpers/external_sources.py index 66cd45583a7..32ebdfa58c6 100644 --- a/tests/integration/helpers/external_sources.py +++ b/tests/integration/helpers/external_sources.py @@ -462,6 +462,9 @@ class SourceCassandra(ExternalSource): ) def prepare(self, structure, table_name, cluster): + if self.internal_hostname is None: + self.internal_hostname = cluster.cassandra_ip + self.client = cassandra.cluster.Cluster([self.internal_hostname], port=self.internal_port) self.session = self.client.connect() self.session.execute( diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt index 67daeced0f1..8fd6c6999cc 100644 --- a/tests/integration/parallel.txt +++ b/tests/integration/parallel.txt @@ -131,6 +131,48 @@ test_postgresql_protocol/test.py::test_psql_is_ready test_postgresql_protocol/test.py::test_psql_client test_postgresql_protocol/test.py::test_python_client test_postgresql_protocol/test.py::test_java_client +test_storage_kafka/test.py::test_kafka_json_as_string +test_storage_kafka/test.py::test_kafka_formats +test_storage_kafka/test.py::test_kafka_settings_old_syntax +test_storage_kafka/test.py::test_kafka_settings_new_syntax +test_storage_kafka/test.py::test_kafka_issue11308 +test_storage_kafka/test.py::test_kafka_issue4116 +test_storage_kafka/test.py::test_kafka_consumer_hang +test_storage_kafka/test.py::test_kafka_consumer_hang2 +test_storage_kafka/test.py::test_kafka_csv_with_delimiter +test_storage_kafka/test.py::test_kafka_tsv_with_delimiter +test_storage_kafka/test.py::test_kafka_select_empty +test_storage_kafka/test.py::test_kafka_json_without_delimiter +test_storage_kafka/test.py::test_kafka_protobuf +test_storage_kafka/test.py::test_kafka_string_field_on_first_position_in_protobuf +test_storage_kafka/test.py::test_kafka_protobuf_no_delimiter +test_storage_kafka/test.py::test_kafka_materialized_view +test_storage_kafka/test.py::test_librdkafka_compression +test_storage_kafka/test.py::test_kafka_materialized_view_with_subquery +test_storage_kafka/test.py::test_kafka_many_materialized_views +test_storage_kafka/test.py::test_kafka_flush_on_big_message +test_storage_kafka/test.py::test_kafka_virtual_columns +test_storage_kafka/test.py::test_kafka_virtual_columns_with_materialized_view +test_storage_kafka/test.py::test_kafka_insert +test_storage_kafka/test.py::test_kafka_produce_consume +test_storage_kafka/test.py::test_kafka_commit_on_block_write +test_storage_kafka/test.py::test_kafka_virtual_columns2 +test_storage_kafka/test.py::test_kafka_produce_key_timestamp +test_storage_kafka/test.py::test_kafka_flush_by_time +test_storage_kafka/test.py::test_kafka_flush_by_block_size +test_storage_kafka/test.py::test_kafka_lot_of_partitions_partial_commit_of_bulk +test_storage_kafka/test.py::test_kafka_rebalance +test_storage_kafka/test.py::test_kafka_no_holes_when_write_suffix_failed +test_storage_kafka/test.py::test_exception_from_destructor +test_storage_kafka/test.py::test_commits_of_unprocessed_messages_on_drop +test_storage_kafka/test.py::test_bad_reschedule +test_storage_kafka/test.py::test_kafka_duplicates_when_commit_failed +test_storage_kafka/test.py::test_premature_flush_on_eof +test_storage_kafka/test.py::test_kafka_unavailable +test_storage_kafka/test.py::test_kafka_issue14202 +test_storage_kafka/test.py::test_kafka_csv_with_thread_per_consumer +test_storage_kerberized_kafka/test.py::test_kafka_json_as_string +test_storage_kerberized_kafka/test.py::test_kafka_json_as_string_no_kdc test_storage_mysql/test.py::test_many_connections test_storage_mysql/test.py::test_insert_select test_storage_mysql/test.py::test_replace_select diff --git a/tests/integration/test_alter_on_mixed_type_cluster/test.py b/tests/integration/test_alter_on_mixed_type_cluster/test.py index 852554f009d..c22626cb379 100644 --- a/tests/integration/test_alter_on_mixed_type_cluster/test.py +++ b/tests/integration/test_alter_on_mixed_type_cluster/test.py @@ -17,19 +17,19 @@ def started_cluster(): cluster.start() for node in [node1, node2]: - node.query(''' - CREATE TABLE test_table_replicated(date Date, id UInt32, value Int32) + node.query_with_retry(''' + CREATE TABLE IF NOT EXISTS test_table_replicated(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '{replica}') ORDER BY id; '''.format(replica=node.name)) - node.query('''CREATE TABLE test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') + node.query_with_retry('''CREATE TABLE IF NOT EXISTS test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') for node in [node3, node4]: - node.query(''' - CREATE TABLE test_table_replicated(date Date, id UInt32, value Int32) + node.query_with_retry(''' + CREATE TABLE IF NOT EXISTS test_table_replicated(date Date, id UInt32, value Int32) ENGINE = ReplicatedMergeTree('/clickhouse/tables/1/someotable', '{replica}') ORDER BY id; '''.format(replica=node.name)) - node.query('''CREATE TABLE test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') + node.query_with_retry('''CREATE TABLE IF NOT EXISTS test_table(date Date, id UInt32, value Int32) ENGINE=MergeTree ORDER BY id''') yield cluster diff --git a/tests/integration/test_concurrent_ttl_merges/test.py b/tests/integration/test_concurrent_ttl_merges/test.py index ba5ed9f0758..18206a4dfb9 100644 --- a/tests/integration/test_concurrent_ttl_merges/test.py +++ b/tests/integration/test_concurrent_ttl_merges/test.py @@ -116,7 +116,7 @@ def test_limited_ttl_merges_in_empty_pool_replicated(started_cluster): node1.query("SYSTEM STOP TTL MERGES") for i in range(100): - node1.query("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) + node1.query_with_retry("INSERT INTO replicated_ttl SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(1)".format(i)) assert node1.query("SELECT COUNT() FROM replicated_ttl") == "100\n" @@ -147,7 +147,7 @@ def test_limited_ttl_merges_two_replicas(started_cluster): node2.query("SYSTEM STOP TTL MERGES") for i in range(100): - node1.query( + node1.query_with_retry( "INSERT INTO replicated_ttl_2 SELECT now() - INTERVAL 1 MONTH, {}, number FROM numbers(10000)".format(i)) node2.query("SYSTEM SYNC REPLICA replicated_ttl_2", timeout=10) diff --git a/tests/integration/test_cross_replication/test.py b/tests/integration/test_cross_replication/test.py index 8a118934c93..cc5618e04e6 100644 --- a/tests/integration/test_cross_replication/test.py +++ b/tests/integration/test_cross_replication/test.py @@ -45,7 +45,7 @@ CREATE TABLE distributed(date Date, id UInt32, shard_id UInt32) 2017-06-16 333 2 ''' node1.query("INSERT INTO distributed FORMAT TSV", stdin=to_insert) - time.sleep(0.5) + time.sleep(5) yield cluster diff --git a/tests/integration/test_ddl_worker_non_leader/test.py b/tests/integration/test_ddl_worker_non_leader/test.py index b64f99d5345..172fc03c005 100644 --- a/tests/integration/test_ddl_worker_non_leader/test.py +++ b/tests/integration/test_ddl_worker_non_leader/test.py @@ -20,14 +20,14 @@ def started_cluster(): def test_non_leader_replica(started_cluster): - node1.query('''CREATE TABLE sometable(id UInt32, value String) + node1.query_with_retry('''CREATE TABLE IF NOT EXISTS sometable(id UInt32, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '1') ORDER BY tuple()''') - node2.query('''CREATE TABLE sometable(id UInt32, value String) + node2.query_with_retry('''CREATE TABLE IF NOT EXISTS sometable(id UInt32, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/0/sometable', '2') ORDER BY tuple() SETTINGS replicated_can_become_leader = 0''') node1.query("INSERT INTO sometable SELECT number, toString(number) FROM numbers(100)") - node2.query("SYSTEM SYNC REPLICA sometable", timeout=10) + node2.query_with_retry("SYSTEM SYNC REPLICA sometable", timeout=10) assert node1.query("SELECT COUNT() FROM sometable") == "100\n" assert node2.query("SELECT COUNT() FROM sometable") == "100\n" diff --git a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py index 1271619f1f7..65080ab40dd 100644 --- a/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py +++ b/tests/integration/test_dictionaries_all_layouts_separate_sources/test_cassandra.py @@ -25,7 +25,7 @@ def setup_module(module): cluster = ClickHouseCluster(__file__, name=test_name) - SOURCE = SourceCassandra("Cassandra", "localhost", cluster.cassandra_port, cluster.cassandra_host, "9042", "", "") + SOURCE = SourceCassandra("Cassandra", None, cluster.cassandra_port, cluster.cassandra_host, cluster.cassandra_port, "", "") simple_tester = SimpleLayoutTester(test_name) simple_tester.cleanup() From ebc846b9f83daf0c5ef129a5aba4f544e700b2a1 Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 7 Apr 2021 16:46:57 +0300 Subject: [PATCH 0100/1060] Pass Settings to aggregate function creator --- .../AggregateFunctionFactory.cpp | 24 +++++++++++++++---- .../AggregateFunctionFactory.h | 9 ++++++- .../AggregateFunctionUniqCombined.cpp | 10 ++++++-- 3 files changed, 36 insertions(+), 7 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index c0011b6ebec..cf6ba6e2fca 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -35,9 +35,25 @@ const String & getAggregateFunctionCanonicalNameIfAny(const String & name) return AggregateFunctionFactory::instance().getCanonicalNameIfAny(name); } + +bool AggregateFunctionWithProperties::hasCreator() const +{ + return std::visit([](auto func) { return func != nullptr; }, creator); +} + +AggregateFunctionPtr +AggregateFunctionWithProperties::create(String name, const DataTypes & argument_types, const Array & params, const Settings & settings) const +{ + if (std::holds_alternative(creator)) + return std::get(creator)(name, argument_types, params); + if (std::holds_alternative(creator)) + return std::get(creator)(name, argument_types, params, settings); + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Unhandled aggregate function creator type"); +} + void AggregateFunctionFactory::registerFunction(const String & name, Value creator_with_properties, CaseSensitiveness case_sensitiveness) { - if (creator_with_properties.creator == nullptr) + if (!creator_with_properties.hasCreator()) throw Exception("AggregateFunctionFactory: the aggregate function " + name + " has been provided " " a null constructor", ErrorCodes::LOGICAL_ERROR); @@ -125,7 +141,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( if (CurrentThread::isInitialized()) query_context = CurrentThread::get().getQueryContext(); - if (found.creator) + if (found.hasCreator()) { out_properties = found.properties; @@ -137,7 +153,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl( if (!out_properties.returns_default_when_only_null && has_null_arguments) return nullptr; - return found.creator(name, argument_types, parameters); + return found.create(name, argument_types, parameters, query_context->getSettingsRef()); } /// Combinators of aggregate functions. @@ -197,7 +213,7 @@ std::optional AggregateFunctionFactory::tryGetPrope if (auto jt = case_insensitive_aggregate_functions.find(Poco::toLower(name)); jt != case_insensitive_aggregate_functions.end()) found = jt->second; - if (found.creator) + if (found.hasCreator()) return found.properties; /// Combinators of aggregate functions. diff --git a/src/AggregateFunctions/AggregateFunctionFactory.h b/src/AggregateFunctions/AggregateFunctionFactory.h index d308a5f559b..d4a713245dd 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.h +++ b/src/AggregateFunctions/AggregateFunctionFactory.h @@ -10,12 +10,14 @@ #include #include #include +#include namespace DB { class Context; +struct Settings; class IDataType; using DataTypePtr = std::shared_ptr; @@ -27,10 +29,12 @@ using DataTypes = std::vector; * For example, in quantileWeighted(0.9)(x, weight), 0.9 is "parameter" and x, weight are "arguments". */ using AggregateFunctionCreator = std::function; +using AggregateFunctionCreatorWithSettings + = std::function; struct AggregateFunctionWithProperties { - AggregateFunctionCreator creator; + std::variant creator; AggregateFunctionProperties properties; AggregateFunctionWithProperties() = default; @@ -42,6 +46,9 @@ struct AggregateFunctionWithProperties : creator(std::forward(creator_)), properties(std::move(properties_)) { } + + bool hasCreator() const; + AggregateFunctionPtr create(String name, const DataTypes & argument_types, const Array & params, const Settings & settings) const; }; diff --git a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp index c0064044f95..5fefe494d61 100644 --- a/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp +++ b/src/AggregateFunctions/AggregateFunctionUniqCombined.cpp @@ -131,8 +131,14 @@ namespace void registerAggregateFunctionUniqCombined(AggregateFunctionFactory & factory) { using namespace std::placeholders; - factory.registerFunction("uniqCombined", std::bind(createAggregateFunctionUniqCombined, false, _1, _2, _3)); // NOLINT - factory.registerFunction("uniqCombined64", std::bind(createAggregateFunctionUniqCombined, true, _1, _2, _3)); // NOLINT + factory.registerFunction("uniqCombined", [](const std::string & name, const DataTypes & argument_types, const Array & params) + { + return createAggregateFunctionUniqCombined(false, name, argument_types, params); + }); + factory.registerFunction("uniqCombined64", [](const std::string & name, const DataTypes & argument_types, const Array & params) + { + return createAggregateFunctionUniqCombined(true, name, argument_types, params); + }); } } From d13d69e246ec5d2fe26f80cb67fd8afdd27c77d2 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Thu, 8 Apr 2021 00:26:51 +0900 Subject: [PATCH 0101/1060] Add allow_experimental_funnel_functions and fix argument order of sequenceNextNode --- .../AggregateFunctionSequenceNextNode.cpp | 19 +- .../AggregateFunctionSequenceNextNode.h | 4 +- src/Core/Settings.h | 1 + .../01656_sequence_next_node_long.sql | 210 +++++++++--------- 4 files changed, 124 insertions(+), 110 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp index 6f2e2a5c3d7..2870f233b65 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include @@ -20,6 +22,7 @@ namespace ErrorCodes extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int BAD_ARGUMENTS; + extern const int FUNCTION_NOT_ALLOWED; } namespace @@ -85,8 +88,8 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, "Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.", name, max_events + min_required_args, max_events), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (const auto * cond_arg = argument_types[1].get(); cond_arg && !isUInt8(cond_arg)) - throw Exception("Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function " + if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg)) + throw Exception("Illegal type " + cond_arg->getName() + " of third argument of aggregate function " + name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); for (const auto i : ext::range(min_required_args, argument_types.size())) @@ -98,12 +101,12 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } - if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String) + if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String) throw Exception{"Illegal type " + argument_types[1].get()->getName() + " of second argument of aggregate function " + name + ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - DataTypePtr data_type = makeNullable(argument_types[2]); + DataTypePtr data_type = makeNullable(argument_types[1]); WhichDataType timestamp_type(argument_types[0].get()); if (timestamp_type.idx == TypeIndex::UInt8) @@ -128,6 +131,14 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events) { return [max_events](const std::string & name, const DataTypes & argument_types, const Array & parameters) { + if (CurrentThread::isInitialized()) + { + const Context * query_context = CurrentThread::get().getQueryContext(); + if (query_context && !query_context->getSettingsRef().allow_experimental_funnel_functions) + throw Exception{"Cannot call 'sequenceNextNode' aggregate function because experimental_funnel_functions is not allowed. " + "Set 'allow_experimental_funnel_functions = 1' setting to enable", ErrorCodes::FUNCTION_NOT_ALLOWED}; + } + return createAggregateFunctionSequenceNode(name, max_events, argument_types, parameters); }; } diff --git a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h index c3907a5adee..629a130a0e3 100644 --- a/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h +++ b/src/AggregateFunctions/AggregateFunctionSequenceNextNode.h @@ -160,8 +160,8 @@ class SequenceNextNodeImpl final static Data & data(AggregateDataPtr place) { return *reinterpret_cast(place); } static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast(place); } - static constexpr size_t base_cond_column_idx = 1; - static constexpr size_t event_column_idx = 2; + static constexpr size_t base_cond_column_idx = 2; + static constexpr size_t event_column_idx = 1; SequenceBase seq_base_kind; SequenceDirection seq_direction; diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2986564b398..cf51508c82c 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -428,6 +428,7 @@ class IColumn; M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \ M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \ + M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate funcion", 0) \ M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated experimental parser", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ \ diff --git a/tests/queries/0_stateless/01656_sequence_next_node_long.sql b/tests/queries/0_stateless/01656_sequence_next_node_long.sql index c03abed8a54..492f8b58862 100644 --- a/tests/queries/0_stateless/01656_sequence_next_node_long.sql +++ b/tests/queries/0_stateless/01656_sequence_next_node_long.sql @@ -1,3 +1,5 @@ +SET allow_experimental_funnel_functions = 1; + DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id; @@ -27,26 +29,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B'); @@ -54,41 +56,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_sequenceNextNode_Nullable; @@ -123,74 +125,74 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C'); -SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D'); -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); -SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); -SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); -SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); -SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D'); +SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL); +SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B'); +SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A'); +SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null); -SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; +SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id; -SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; +SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id; INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A'); -SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; +SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12; DROP TABLE IF EXISTS test_sequenceNextNode; @@ -211,18 +213,18 @@ INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2'); INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3'); INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4'); -SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; -SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; +SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id; DROP TABLE IF EXISTS test_base_condition; From e176b52c1383350f2c861df4aa9ef5c7f30f663d Mon Sep 17 00:00:00 2001 From: achimbab Date: Thu, 8 Apr 2021 13:02:41 +0900 Subject: [PATCH 0102/1060] Fix typo in Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index cf51508c82c..ad495c6bee9 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -428,7 +428,7 @@ class IColumn; M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \ M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \ M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \ - M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate funcion", 0) \ + M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate function", 0) \ M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated experimental parser", 0) \ M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \ \ From 49e6b3e8c762d5678559f203156aa859164c1abe Mon Sep 17 00:00:00 2001 From: Vladimir Date: Thu, 8 Apr 2021 12:10:41 +0300 Subject: [PATCH 0103/1060] Apply suggestions from code review, doc for sequenceNextNode --- .../parametric-functions.md | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index a2531303c7a..2cc1ebd40bf 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -511,10 +511,12 @@ Same behavior as [sumMap](../../sql-reference/aggregate-functions/reference/summ Returns a value of next event that matched an event chain. +_Experimental function, `SET allow_experimental_funnel_functions = 1` to enable it._ + **Syntax** ``` sql -sequenceNextNode(direction, base)(timestamp, base_condition, event_column, event1, event2, event3, ...) +sequenceNextNode(direction, base)(timestamp, event_column, base_condition, event1, event2, event3, ...) ``` **Parameters** @@ -530,8 +532,8 @@ sequenceNextNode(direction, base)(timestamp, base_condition, event_column, event **Arguments** - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime` and other unsigned integer types. -- `base_condition` — Condition that the base point must fulfill. - `event_column` — Name of the column containing the value of the next event to be returned. Data types supported: `String` and `Nullable(String)` +- `base_condition` — Condition that the base point must fulfill. - `cond` — Conditions describing the chain of events. `UInt8` **Returned value** @@ -557,7 +559,7 @@ ORDER BY id; INSERT INTO test_flow VALUES (1, 1, 'A') (2, 1, 'B') (3, 1, 'C') (4, 1, 'E') (5, 1, 'F'); -SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'A', page, page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'A', page = 'A', page = 'B') as next_flow FROM test_flow GROUP BY id; ``` Result: @@ -579,7 +581,7 @@ INSERT INTO test_flow VALUES (1, 3, 'Gift') (2, 3, 'Home') (3, 3, 'Gift') (4, 3, ``` ```SQL -SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'Home', page, page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, page = 'Home', page = 'Home', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Base point, Matched with Home @@ -600,7 +602,7 @@ SELECT id, sequenceNextNode('forward', 'head')(dt, page = 'Home', page, page = ' **Behavior for `backward` and `tail`** ```SQL -SELECT id, sequenceNextNode('backward', 'tail')(dt, page = 'Basket', page, page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, page = 'Basket', page = 'Basket', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home @@ -622,7 +624,7 @@ SELECT id, sequenceNextNode('backward', 'tail')(dt, page = 'Basket', page, page **Behavior for `forward` and `first_match`** ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home @@ -641,7 +643,7 @@ SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, p ``` ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home @@ -663,7 +665,7 @@ SELECT id, sequenceNextNode('forward', 'first_match')(dt, page = 'Gift', page, p **Behavior for `backward` and `last_match`** ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // The result From bd5b29d3a5b103fbd857a2a99d27cdcbd5b7bffd Mon Sep 17 00:00:00 2001 From: Vladimir Date: Thu, 8 Apr 2021 12:13:10 +0300 Subject: [PATCH 0104/1060] Apply suggestions from code review, doc for sequenceNextNode --- .../aggregate-functions/parametric-functions.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/parametric-functions.md b/docs/en/sql-reference/aggregate-functions/parametric-functions.md index 2cc1ebd40bf..27f76ebc5b6 100644 --- a/docs/en/sql-reference/aggregate-functions/parametric-functions.md +++ b/docs/en/sql-reference/aggregate-functions/parametric-functions.md @@ -684,7 +684,7 @@ SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', p ``` ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, page = 'Gift', page, page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, page = 'Gift', page = 'Gift', page = 'Home') FROM test_flow GROUP BY id; dt id page 1970-01-01 09:00:01 1 Home // Matched with Home, the result is null @@ -721,7 +721,7 @@ INSERT INTO test_flow_basecond VALUES (1, 1, 'A', 'ref4') (2, 1, 'A', 'ref3') (3 ``` ```SQL -SELECT id, sequenceNextNode('forward', 'head')(dt, ref = 'ref1', page, page = 'A') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('forward', 'head')(dt, page, ref = 'ref1', page = 'A') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 // The head can't be base point becasue the ref column of the head unmatched with 'ref1'. @@ -731,7 +731,7 @@ SELECT id, sequenceNextNode('forward', 'head')(dt, ref = 'ref1', page, page = 'A ``` ```SQL -SELECT id, sequenceNextNode('backward', 'tail')(dt, ref = 'ref4', page, page = 'B') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('backward', 'tail')(dt, page, ref = 'ref4', page = 'B') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 @@ -741,7 +741,7 @@ SELECT id, sequenceNextNode('backward', 'tail')(dt, ref = 'ref4', page, page = ' ``` ```SQL -SELECT id, sequenceNextNode('forward', 'first_match')(dt, ref = 'ref3', page, page = 'A') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('forward', 'first_match')(dt, page, ref = 'ref3', page = 'A') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 // This row can't be base point becasue the ref column unmatched with 'ref3'. @@ -751,7 +751,7 @@ SELECT id, sequenceNextNode('forward', 'first_match')(dt, ref = 'ref3', page, pa ``` ```SQL -SELECT id, sequenceNextNode('backward', 'last_match')(dt, ref = 'ref2', page, page = 'B') FROM test_flow_basecond GROUP BY id; +SELECT id, sequenceNextNode('backward', 'last_match')(dt, page, ref = 'ref2', page = 'B') FROM test_flow_basecond GROUP BY id; dt id page ref 1970-01-01 09:00:01 1 A ref4 From 2cc9d314be53d0f8ca16c1050a71a88439d67563 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 8 Apr 2021 17:43:57 +0300 Subject: [PATCH 0105/1060] fix --- .../docker_compose_postgres_cluster.yml | 26 ++++----- tests/integration/helpers/cluster.py | 55 ++++++++++++------- .../test_storage_postgresql/test.py | 8 +-- 3 files changed, 52 insertions(+), 37 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml index aa9301b1060..94b941b74da 100644 --- a/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_postgres_cluster.yml @@ -29,16 +29,16 @@ services: source: ${POSTGRES3_DIR:-} target: /postgres/ postgres4: - image: postgres - command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] - restart: always - environment: - POSTGRES_HOST_AUTH_METHOD: "trust" - POSTGRES_PASSWORD: mysecretpassword - PGDATA: /postgres/data - expose: - - ${POSTGRES_PORT} - volumes: - - type: ${POSTGRES_LOGS_FS:-tmpfs} - source: ${POSTGRES4_DIR:-} - target: /postgres/ \ No newline at end of file + image: postgres + command: ["postgres", "-c", "logging_collector=on", "-c", "log_directory=/postgres/logs", "-c", "log_filename=postgresql.log", "-c", "log_statement=all"] + restart: always + environment: + POSTGRES_HOST_AUTH_METHOD: "trust" + POSTGRES_PASSWORD: mysecretpassword + PGDATA: /postgres/data + expose: + - ${POSTGRES_PORT} + volumes: + - type: ${POSTGRES_LOGS_FS:-tmpfs} + source: ${POSTGRES4_DIR:-} + target: /postgres/ \ No newline at end of file diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 703882fcd49..7fb8d7dc25a 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -344,12 +344,8 @@ class ClickHouseCluster: def setup_postgres_cmd(self, instance, env_variables, docker_compose_yml_dir): self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - env_variables['POSTGRES_HOST'] = self.postgres_host env_variables['POSTGRES_PORT'] = str(self.postgres_port) env_variables['POSTGRES_DIR'] = self.postgres_logs_dir - env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir - env_variables['POSTGRES3_DIR'] = self.postgres3_logs_dir - env_variables['POSTGRES4_DIR'] = self.postgres4_logs_dir env_variables['POSTGRES_LOGS_FS'] = "bind" self.with_postgres = True @@ -357,6 +353,17 @@ class ClickHouseCluster: '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')] return self.base_postgres_cmd + def setup_postgres_cluster_cmd(self, instance, env_variables, docker_compose_yml_dir): + self.with_postgres_cluster = True + env_variables['POSTGRES_PORT'] = str(self.postgres_port) + env_variables['POSTGRES2_DIR'] = self.postgres2_logs_dir + env_variables['POSTGRES3_DIR'] = self.postgres3_logs_dir + env_variables['POSTGRES4_DIR'] = self.postgres4_logs_dir + env_variables['POSTGRES_LOGS_FS'] = "bind" + self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) + self.base_postgres_cluster_cmd = ['docker-compose', '--env-file', instance.env_file, '--project-name', self.project_name, + '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')] + def setup_hdfs_cmd(self, instance, env_variables, docker_compose_yml_dir): self.with_hdfs = True env_variables['HDFS_HOST'] = self.hdfs_host @@ -559,11 +566,7 @@ class ClickHouseCluster: cmds.append(self.setup_postgres_cmd(instance, env_variables, docker_compose_yml_dir)) if with_postgres_cluster and not self.with_postgres_cluster: - self.with_postgres_cluster = True - self.base_cmd.extend(['--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres.yml')]) - self.base_postgres_cluster_cmd = ['docker-compose', '--project-name', self.project_name, - '--file', p.join(docker_compose_yml_dir, 'docker_compose_postgres_cluster.yml')] - cmds.append(self.base_postgres_cluster_cmd) + cmds.append(self.setup_postgres_cluster_cmd(instance, env_variables, docker_compose_yml_dir)) if with_odbc_drivers and not self.with_odbc_drivers: self.with_odbc_drivers = True @@ -760,16 +763,30 @@ class ClickHouseCluster: def wait_postgres_to_start(self, timeout=60): self.postgres_ip = self.get_instance_ip(self.postgres_host) + start = time.time() + while time.time() - start < timeout: + try: + conn = psycopg2.connect(host=self.postgres_ip, port=self.postgres_port, user='postgres', password='mysecretpassword') + conn.close() + logging.debug("Postgres Started") + return + except Exception as ex: + logging.debug("Can't connect to Postgres " + str(ex)) + time.sleep(0.5) + + raise Exception("Cannot wait Postgres container") + + def wait_postgres_cluster_to_start(self, timeout=60): self.postgres2_ip = self.get_instance_ip(self.postgres2_host) self.postgres3_ip = self.get_instance_ip(self.postgres3_host) self.postgres4_ip = self.get_instance_ip(self.postgres4_host) start = time.time() - for ip in [self.postgres_ip, self.postgres2_ip, self.postgres3_ip, self.postgres4_ip]: + for ip in [self.postgres2_ip, self.postgres3_ip, self.postgres4_ip]: while time.time() - start < timeout: try: conn = psycopg2.connect(host=ip, port=self.postgres_port, user='postgres', password='mysecretpassword') conn.close() - logging.debug("Postgres Started") + logging.debug("Postgres Cluster Started") return except Exception as ex: logging.debug("Can't connect to Postgres " + str(ex)) @@ -1013,22 +1030,20 @@ class ClickHouseCluster: shutil.rmtree(self.postgres_dir) os.makedirs(self.postgres_logs_dir) os.chmod(self.postgres_logs_dir, stat.S_IRWXO) - os.makedirs(self.postgres2_logs_dir) - os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) - os.makedirs(self.postgres3_logs_dir) - os.chmod(self.postgres3_logs_dir, stat.S_IRWXO) - os.makedirs(self.postgres4_logs_dir) - os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cmd + common_opts) self.wait_postgres_to_start(120) if self.with_postgres_cluster and self.base_postgres_cluster_cmd: print('Setup Postgres') + os.makedirs(self.postgres2_logs_dir) + os.chmod(self.postgres2_logs_dir, stat.S_IRWXO) + os.makedirs(self.postgres3_logs_dir) + os.chmod(self.postgres3_logs_dir, stat.S_IRWXO) + os.makedirs(self.postgres4_logs_dir) + os.chmod(self.postgres4_logs_dir, stat.S_IRWXO) subprocess_check_call(self.base_postgres_cluster_cmd + common_opts) - self.wait_postgres_to_start(120, port=5421) - self.wait_postgres_to_start(120, port=5441) - self.wait_postgres_to_start(120, port=5461) + self.wait_postgres_cluster_to_start(120) if self.with_kafka and self.base_kafka_cmd: logging.debug('Setup Kafka') diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index 1ab090cb89f..e5fffe389d2 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -57,7 +57,7 @@ def started_cluster(): def test_postgres_select_insert(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() table_name = 'test_many' table = f'''postgresql('{started_cluster.postgres_ip}:{started_cluster.postgres_port}', 'clickhouse', '{table_name}', 'postgres', 'mysecretpassword')''' @@ -77,7 +77,7 @@ def test_postgres_select_insert(started_cluster): def test_postgres_conversions(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() cursor.execute( '''CREATE TABLE IF NOT EXISTS test_types ( @@ -158,7 +158,7 @@ def test_postgres_conversions(started_cluster): def test_non_default_scema(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() cursor.execute('CREATE SCHEMA test_schema') cursor.execute('CREATE TABLE test_schema.test_table (a integer)') @@ -190,7 +190,7 @@ def test_non_default_scema(started_cluster): def test_concurrent_queries(started_cluster): - conn = get_postgres_conn(started_cluster, True) + conn = get_postgres_conn(started_cluster, started_cluster.postgres_ip, True) cursor = conn.cursor() node1.query(''' From c3212f1c4677a905b41a9228cb675f5c172c803a Mon Sep 17 00:00:00 2001 From: Anton Ivashkin Date: Thu, 8 Apr 2021 21:27:56 +0300 Subject: [PATCH 0106/1060] Fix Zero-Copy replication with several S3 volumes (issue 22679) --- src/Storages/MergeTree/DataPartsExchange.cpp | 27 ++++++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 14 +++- src/Storages/MergeTree/IMergeTreeDataPart.h | 28 ++++---- src/Storages/MergeTree/MergeTreeData.cpp | 17 ++++- .../configs/config.d/s3.xml | 32 +++++++++ .../test_s3_zero_copy_replication/test.py | 72 +++++++++++++++++-- 6 files changed, 167 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 862a3088f89..98f0423d334 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -410,9 +410,34 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchPart( size_t sum_files_size = 0; readBinary(sum_files_size, in); IMergeTreeDataPart::TTLInfos ttl_infos; - /// Skip ttl infos, not required for S3 metadata String ttl_infos_string; readBinary(ttl_infos_string, in); + ReadBufferFromString ttl_infos_buffer(ttl_infos_string); + assertString("ttl format version: 1\n", ttl_infos_buffer); + ttl_infos.read(ttl_infos_buffer); + + ReservationPtr reservation + = data.balancedReservation(metadata_snapshot, sum_files_size, 0, part_name, part_info, {}, tagger_ptr, &ttl_infos, true); + if (!reservation) + reservation + = data.reserveSpacePreferringTTLRules(metadata_snapshot, sum_files_size, ttl_infos, std::time(nullptr), 0, true); + if (reservation) + { + DiskPtr disk = reservation->getDisk(); + if (disk && disk->getType() == DiskType::Type::S3) + { + for (const auto & d : disks_s3) + { + if (d->getPath() == disk->getPath()) + { + Disks disks_tmp = { disk }; + disks_s3.swap(disks_tmp); + break; + } + } + } + } + String part_type = "Wide"; readStringBinary(part_type, in); if (part_type == "InMemory") diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 72ce05e7aab..2683357af3c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -189,6 +189,9 @@ static void incrementStateMetric(IMergeTreeDataPart::State state) case IMergeTreeDataPart::State::DeleteOnDestroy: CurrentMetrics::add(CurrentMetrics::PartsDeleteOnDestroy); return; + case IMergeTreeDataPart::State::DeleteOnDestroyKeepS3: + CurrentMetrics::add(CurrentMetrics::PartsDeleteOnDestroy); + return; } } @@ -214,6 +217,9 @@ static void decrementStateMetric(IMergeTreeDataPart::State state) case IMergeTreeDataPart::State::DeleteOnDestroy: CurrentMetrics::sub(CurrentMetrics::PartsDeleteOnDestroy); return; + case IMergeTreeDataPart::State::DeleteOnDestroyKeepS3: + CurrentMetrics::sub(CurrentMetrics::PartsDeleteOnDestroy); + return; } } @@ -393,7 +399,7 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) void IMergeTreeDataPart::removeIfNeeded() { - if (state == State::DeleteOnDestroy || is_temp) + if (state == State::DeleteOnDestroy || state == State::DeleteOnDestroyKeepS3 || is_temp) { try { @@ -416,9 +422,9 @@ void IMergeTreeDataPart::removeIfNeeded() } } - remove(false); + remove(state == State::DeleteOnDestroyKeepS3); - if (state == State::DeleteOnDestroy) + if (state == State::DeleteOnDestroy || state == State::DeleteOnDestroyKeepS3) { LOG_TRACE(storage.log, "Removed part from old location {}", path); } @@ -463,6 +469,8 @@ String IMergeTreeDataPart::stateToString(IMergeTreeDataPart::State state) return "Deleting"; case State::DeleteOnDestroy: return "DeleteOnDestroy"; + case State::DeleteOnDestroyKeepS3: + return "DeleteOnDestroyKeepS3"; } __builtin_unreachable(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 03f6564788a..57738f8ea2d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -202,22 +202,24 @@ public: * Part state should be modified under data_parts mutex. * * Possible state transitions: - * Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set - * Precommitted -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part) - * Precommitted -> Committed: we successfully committed a part to active dataset - * Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION - * Outdated -> Deleting: a cleaner selected this part for deletion - * Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion - * Committed -> DeleteOnDestroy if part was moved to another disk + * Temporary -> Precommitted: we are trying to commit a fetched, inserted or merged part to active set + * Precommitted -> Outdated: we could not add a part to active set and are doing a rollback (for example it is duplicated part) + * Precommitted -> Committed: we successfully committed a part to active dataset + * Precommitted -> Outdated: a part was replaced by a covering part or DROP PARTITION + * Outdated -> Deleting: a cleaner selected this part for deletion + * Deleting -> Outdated: if an ZooKeeper error occurred during the deletion, we will retry deletion + * Committed -> DeleteOnDestroy if part was moved to another disk + * Committed -> DeleteOnDestroyKeepS3 if part was moved to another disk but shared data on S3 */ enum class State { - Temporary, /// the part is generating now, it is not in data_parts list - PreCommitted, /// the part is in data_parts, but not used for SELECTs - Committed, /// active data part, used by current and upcoming SELECTs - Outdated, /// not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes - Deleting, /// not active data part with identity refcounter, it is deleting right now by a cleaner - DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor + Temporary, /// the part is generating now, it is not in data_parts list + PreCommitted, /// the part is in data_parts, but not used for SELECTs + Committed, /// active data part, used by current and upcoming SELECTs + Outdated, /// not active data part, but could be used by only current SELECTs, could be deleted after SELECTs finishes + Deleting, /// not active data part with identity refcounter, it is deleting right now by a cleaner + DeleteOnDestroy, /// part was moved to another disk and should be deleted in own destructor + DeleteOnDestroyKeepS3, /// same as DeleteOnDestroy but shared S3 data should be keeped }; using TTLInfo = MergeTreeDataPartTTLInfo; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d61de13b604..0f142e07f7d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2611,7 +2611,22 @@ void MergeTreeData::swapActivePart(MergeTreeData::DataPartPtr part_copy) if (active_part_it == data_parts_by_info.end()) throw Exception("Cannot swap part '" + part_copy->name + "', no such active part.", ErrorCodes::NO_SUCH_DATA_PART); - modifyPartState(original_active_part, DataPartState::DeleteOnDestroy); + /// We do not check allow_s3_zero_copy_replication here because data may be shared + /// when allow_s3_zero_copy_replication turned on and off again + bool keep_s3 = false; + + if (original_active_part->volume->getDisk()->getType() == DiskType::Type::S3) + { + if (part_copy->volume->getDisk()->getType() == DiskType::Type::S3 + && original_active_part->getUniqueId() == part_copy->getUniqueId()) + { /// May be when several volumes use the same S3 storage + keep_s3 = true; + } + else + keep_s3 = !unlockSharedData(*original_active_part); + } + + modifyPartState(original_active_part, keep_s3 ? DataPartState::DeleteOnDestroyKeepS3 : DataPartState::DeleteOnDestroy); data_parts_indexes.erase(active_part_it); auto part_it = data_parts_indexes.insert(part_copy).first; diff --git a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml index 88eb49d9f17..ee990ee9012 100644 --- a/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml +++ b/tests/integration/test_s3_zero_copy_replication/configs/config.d/s3.xml @@ -8,6 +8,18 @@ minio minio123 + + s3 + http://minio1:9001/root/data/ + minio + minio123 + + + s3 + http://minio1:9001/root/data2/ + minio + minio123 +
@@ -27,6 +39,26 @@ + + +
+ s31 +
+ + s32 + +
+
+ + +
+ s31 +
+ + s31_again + +
+
diff --git a/tests/integration/test_s3_zero_copy_replication/test.py b/tests/integration/test_s3_zero_copy_replication/test.py index 5bc30ab1d6b..423bf3f8c03 100644 --- a/tests/integration/test_s3_zero_copy_replication/test.py +++ b/tests/integration/test_s3_zero_copy_replication/test.py @@ -27,10 +27,10 @@ def cluster(): cluster.shutdown() -def get_large_objects_count(cluster, size=100): +def get_large_objects_count(cluster, folder='data', size=100): minio = cluster.minio_client counter = 0 - for obj in minio.list_objects(cluster.minio_bucket, 'data/'): + for obj in minio.list_objects(cluster.minio_bucket, '{}/'.format(folder)): if obj.size >= size: counter = counter + 1 return counter @@ -69,7 +69,7 @@ def test_s3_zero_copy_replication(cluster, policy): # Based on version 20.x - two parts assert get_large_objects_count(cluster) == 2 - node1.query("OPTIMIZE TABLE s3_test") + node1.query("OPTIMIZE TABLE s3_test FINAL") time.sleep(1) @@ -119,7 +119,7 @@ def test_s3_zero_copy_on_hybrid_storage(cluster): assert node2.query("SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values") == "('all','default')" # Total objects in S3 - s3_objects = get_large_objects_count(cluster, 0) + s3_objects = get_large_objects_count(cluster, size=0) node2.query("ALTER TABLE hybrid_test MOVE PARTITION ID 'all' TO DISK 's31'") @@ -127,10 +127,72 @@ def test_s3_zero_copy_on_hybrid_storage(cluster): assert node2.query("SELECT partition_id,disk_name FROM system.parts WHERE table='hybrid_test' FORMAT Values") == "('all','s31')" # Check that after moving partition on node2 no new obects on s3 - assert get_large_objects_count(cluster, 0) == s3_objects + assert get_large_objects_count(cluster, size=0) == s3_objects assert node1.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") == "(0,'data'),(1,'data')" assert node2.query("SELECT * FROM hybrid_test ORDER BY id FORMAT Values") == "(0,'data'),(1,'data')" node1.query("DROP TABLE IF EXISTS hybrid_test NO DELAY") node2.query("DROP TABLE IF EXISTS hybrid_test NO DELAY") + + +@pytest.mark.parametrize( + "storage_policy", ["tiered", "tiered_copy"] +) +def test_s3_zero_copy_with_ttl_move(cluster, storage_policy): + node1 = cluster.instances["node1"] + node2 = cluster.instances["node2"] + + for i in range(10): + node1.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY") + node2.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY") + + node1.query( + """ + CREATE TABLE ttl_move_test ON CLUSTER test_cluster (d UInt64, d1 DateTime) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/ttl_move_test', '{}') + ORDER BY d + TTL d1 + INTERVAL 2 DAY TO VOLUME 'external' + SETTINGS storage_policy='{}' + """ + .format('{replica}', storage_policy) + ) + + node1.query("INSERT INTO ttl_move_test VALUES (10, now() - INTERVAL 3 DAY)") + node1.query("INSERT INTO ttl_move_test VALUES (11, now() - INTERVAL 1 DAY)") + + assert node1.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(2)" + assert node2.query("SELECT count() FROM ttl_move_test FORMAT Values") == "(2)" + assert node1.query("SELECT d FROM ttl_move_test ORDER BY d1 FORMAT Values") == "(10),(11)" + assert node2.query("SELECT d FROM ttl_move_test ORDER BY d1 FORMAT Values") == "(10),(11)" + + node1.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY") + node2.query("DROP TABLE IF EXISTS ttl_move_test NO DELAY") + + +def test_s3_zero_copy_with_ttl_delete(cluster): + node1 = cluster.instances["node1"] + node2 = cluster.instances["node2"] + + for i in range(10): + node1.query( + """ + CREATE TABLE ttl_delete_test ON CLUSTER test_cluster (d UInt64, d1 DateTime) + ENGINE=ReplicatedMergeTree('/clickhouse/tables/ttl_delete_test', '{}') + ORDER BY d + TTL d1 + INTERVAL 2 DAY + SETTINGS storage_policy='tiered' + """ + .format('{replica}') + ) + + node1.query("INSERT INTO ttl_delete_test VALUES (10, now() - INTERVAL 3 DAY)") + node1.query("INSERT INTO ttl_delete_test VALUES (11, now() - INTERVAL 1 DAY)") + + assert node1.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1)" + assert node2.query("SELECT count() FROM ttl_delete_test FORMAT Values") == "(1)" + assert node1.query("SELECT d FROM ttl_delete_test ORDER BY d1 FORMAT Values") == "(11)" + assert node2.query("SELECT d FROM ttl_delete_test ORDER BY d1 FORMAT Values") == "(11)" + + node1.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY") + node2.query("DROP TABLE IF EXISTS ttl_delete_test NO DELAY") From b4dded3fc29a437c889f10ffb10c968709e91c02 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 12 Apr 2021 10:03:12 +0300 Subject: [PATCH 0107/1060] wip --- .../compose/docker_compose_mysql_cluster.yml | 60 +++++++++---------- tests/integration/parallel.txt | 2 + tests/integration/test_mysql_protocol/test.py | 2 +- tests/integration/test_storage_mysql/test.py | 35 +++++------ .../test_storage_postgresql/test.py | 8 +-- tests/integration/test_storage_s3/test.py | 40 ++++++------- 6 files changed, 75 insertions(+), 72 deletions(-) diff --git a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml index f3a5b9975c4..6ced7536812 100644 --- a/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml +++ b/docker/test/integration/runner/compose/docker_compose_mysql_cluster.yml @@ -5,10 +5,10 @@ services: restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} DATADIR: /mysql/ expose: - - ${MYSQL_PORT} + - ${MYSQL_CLUSTER_PORT} command: --server_id=100 --log-bin='mysql-bin-2.log' --default-time-zone='+3:00' @@ -19,40 +19,40 @@ services: --general-log=ON --general-log-file=/mysql/2_general.log volumes: - - type: ${MYSQL_LOGS_FS:-tmpfs} - source: ${MYSQL_LOGS:-} + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} target: /mysql/ mysql3: - image: mysql:5.7 - restart: always - environment: - MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} - DATADIR: /mysql/ - expose: - - ${MYSQL_PORT} - command: --server_id=100 - --log-bin='mysql-bin-3.log' - --default-time-zone='+3:00' - --gtid-mode="ON" - --enforce-gtid-consistency - --log-error-verbosity=3 - --log-error=/mysql/3_error.log - --general-log=ON - --general-log-file=/mysql/3_general.log - volumes: - - type: ${MYSQL_LOGS_FS:-tmpfs} - source: ${MYSQL_LOGS:-} - target: /mysql/ + image: mysql:5.7 + restart: always + environment: + MYSQL_ROOT_PASSWORD: clickhouse + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} + DATADIR: /mysql/ + expose: + - ${MYSQL_CLUSTER_PORT} + command: --server_id=100 + --log-bin='mysql-bin-3.log' + --default-time-zone='+3:00' + --gtid-mode="ON" + --enforce-gtid-consistency + --log-error-verbosity=3 + --log-error=/mysql/3_error.log + --general-log=ON + --general-log-file=/mysql/3_general.log + volumes: + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ mysql4: image: mysql:5.7 restart: always environment: MYSQL_ROOT_PASSWORD: clickhouse - MYSQL_ROOT_HOST: ${MYSQL_ROOT_HOST} + MYSQL_ROOT_HOST: ${MYSQL_CLUSTER_ROOT_HOST} DATADIR: /mysql/ expose: - - ${MYSQL_PORT} + - ${MYSQL_CLUSTER_PORT} command: --server_id=100 --log-bin='mysql-bin-4.log' --default-time-zone='+3:00' @@ -63,6 +63,6 @@ services: --general-log=ON --general-log-file=/mysql/4_general.log volumes: - - type: ${MYSQL_LOGS_FS:-tmpfs} - source: ${MYSQL_LOGS:-} - target: /mysql/ \ No newline at end of file + - type: ${MYSQL_CLUSTER_LOGS_FS:-tmpfs} + source: ${MYSQL_CLUSTER_LOGS:-} + target: /mysql/ \ No newline at end of file diff --git a/tests/integration/parallel.txt b/tests/integration/parallel.txt index 8fd6c6999cc..869fa8a52ca 100644 --- a/tests/integration/parallel.txt +++ b/tests/integration/parallel.txt @@ -181,7 +181,9 @@ test_storage_mysql/test.py::test_where test_storage_mysql/test.py::test_table_function test_storage_mysql/test.py::test_binary_type test_storage_mysql/test.py::test_enum_type +test_storage_mysql/test.py::test_mysql_distributed test_storage_postgresql/test.py::test_postgres_select_insert test_storage_postgresql/test.py::test_postgres_conversions test_storage_postgresql/test.py::test_non_default_scema test_storage_postgresql/test.py::test_concurrent_queries +test_storage_postgresql/test.py::test_postgres_distributed diff --git a/tests/integration/test_mysql_protocol/test.py b/tests/integration/test_mysql_protocol/test.py index 951a07c74cd..74ece5ad433 100644 --- a/tests/integration/test_mysql_protocol/test.py +++ b/tests/integration/test_mysql_protocol/test.py @@ -46,7 +46,7 @@ def mysql_server(mysql_client): :type mysql_client: Container :rtype: Container """ - retries = 30 + retries = 60 for i in range(retries): info = mysql_client.client.api.inspect_container(mysql_client.name) if info['State']['Health']['Status'] == 'healthy': diff --git a/tests/integration/test_storage_mysql/test.py b/tests/integration/test_storage_mysql/test.py index c4426817b1b..ca6ff6d71f1 100644 --- a/tests/integration/test_storage_mysql/test.py +++ b/tests/integration/test_storage_mysql/test.py @@ -36,7 +36,7 @@ def started_cluster(): try: cluster.start() - conn = get_mysql_conn(cluster, cluster.mysql57_ip) + conn = get_mysql_conn(cluster, cluster.mysql_ip) create_mysql_db(conn, 'clickhouse') ## create mysql db and table @@ -50,7 +50,7 @@ def started_cluster(): def test_many_connections(started_cluster): table_name = 'test_many_connections' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -70,7 +70,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_select(started_cluster): table_name = 'test_insert_select' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -86,7 +86,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_replace_select(started_cluster): table_name = 'test_replace_select' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -105,7 +105,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_insert_on_duplicate_select(started_cluster): table_name = 'test_insert_on_duplicate_select' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' @@ -124,7 +124,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_where(started_cluster): table_name = 'test_where' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse'); @@ -144,7 +144,7 @@ CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32) ENGINE = MySQL def test_table_function(started_cluster): - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, 'table_function') table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('table_function') assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0' @@ -166,7 +166,7 @@ def test_table_function(started_cluster): def test_binary_type(started_cluster): - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) with conn.cursor() as cursor: cursor.execute("CREATE TABLE clickhouse.binary_type (id INT PRIMARY KEY, data BINARY(16) NOT NULL)") table_function = "mysql('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format('binary_type') @@ -176,7 +176,7 @@ def test_binary_type(started_cluster): def test_enum_type(started_cluster): table_name = 'test_enum_type' - conn = get_mysql_conn(started_cluster, cluster.mysql57_ip) + conn = get_mysql_conn(started_cluster, cluster.mysql_ip) create_mysql_table(conn, table_name) node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, source Enum8('IP' = 1, 'URL' = 2)) ENGINE = MySQL('mysql57:3306', 'clickhouse', '{}', 'root', 'clickhouse', 1); @@ -202,7 +202,7 @@ def create_mysql_table(conn, tableName): def test_mysql_distributed(started_cluster): table_name = 'test_replicas' - conn1 = get_mysql_conn(started_cluster, started_cluster.mysql57_ip) + conn1 = get_mysql_conn(started_cluster, started_cluster.mysql_ip) conn2 = get_mysql_conn(started_cluster, started_cluster.mysql2_ip) conn3 = get_mysql_conn(started_cluster, started_cluster.mysql3_ip) conn4 = get_mysql_conn(started_cluster, started_cluster.mysql4_ip) @@ -210,6 +210,7 @@ def test_mysql_distributed(started_cluster): create_mysql_db(conn1, 'clickhouse') create_mysql_db(conn2, 'clickhouse') create_mysql_db(conn3, 'clickhouse') + create_mysql_db(conn4, 'clickhouse') create_mysql_table(conn1, table_name) create_mysql_table(conn2, table_name) @@ -228,13 +229,13 @@ def test_mysql_distributed(started_cluster): nodes[i-1].query(''' CREATE TABLE test_replica{} (id UInt32, name String, age UInt32, money UInt32) - ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, i)) + ENGINE = MySQL(`mysql{}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse');'''.format(i, 57 if i==1 else i)) nodes[i-1].query("INSERT INTO test_replica{} (id, name) SELECT number, 'host{}' from numbers(10) ".format(i, i)) # test multiple ports parsing - result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql{1|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql{57|2|3}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') assert(result == 'host1\n' or result == 'host2\n' or result == 'host3\n') - result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql1:3306|mysql2:3306|mysql3:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + result = node2.query('''SELECT DISTINCT(name) FROM mysql(`mysql57:3306|mysql2:3306|mysql3:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') assert(result == 'host1\n' or result == 'host2\n' or result == 'host3\n') # check all replicas are traversed @@ -250,7 +251,7 @@ def test_mysql_distributed(started_cluster): node2.query(''' CREATE TABLE test_shards (id UInt32, name String, age UInt32, money UInt32) - ENGINE = ExternalDistributed('MySQL', `mysql{1|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') + ENGINE = ExternalDistributed('MySQL', `mysql{57|2}:3306,mysql{3|4}:3306`, 'clickhouse', 'test_replicas', 'root', 'clickhouse'); ''') # Check only one replica in each shard is used result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") @@ -264,10 +265,10 @@ def test_mysql_distributed(started_cluster): result = node2.query(query) assert(result == 'host1\nhost2\nhost3\nhost4\n') - # disconnect mysql1 - started_cluster.pause_container('mysql1') + # disconnect mysql57 + started_cluster.pause_container('mysql57') result = node2.query("SELECT DISTINCT(name) FROM test_shards ORDER BY name") - started_cluster.unpause_container('mysql1') + started_cluster.unpause_container('mysql57') assert(result == 'host2\nhost4\n' or result == 'host3\nhost4\n') if __name__ == '__main__': diff --git a/tests/integration/test_storage_postgresql/test.py b/tests/integration/test_storage_postgresql/test.py index e5fffe389d2..bb71001ef52 100644 --- a/tests/integration/test_storage_postgresql/test.py +++ b/tests/integration/test_storage_postgresql/test.py @@ -243,10 +243,10 @@ def test_concurrent_queries(started_cluster): def test_postgres_distributed(started_cluster): - conn0 = get_postgres_conn(started_cluster.postgres_ip, database=True) - conn1 = get_postgres_conn(started_cluster.postgres2_ip, database=True) - conn2 = get_postgres_conn(started_cluster.postgres3_ip, database=True) - conn3 = get_postgres_conn(started_cluster.postgres4_ip, database=True) + conn0 = get_postgres_conn(started_cluster, started_cluster.postgres_ip, database=True) + conn1 = get_postgres_conn(started_cluster, started_cluster.postgres2_ip, database=True) + conn2 = get_postgres_conn(started_cluster, started_cluster.postgres3_ip, database=True) + conn3 = get_postgres_conn(started_cluster, started_cluster.postgres4_ip, database=True) cursor0 = conn0.cursor() cursor1 = conn1.cursor() diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index f4a208543ce..1c859fa782e 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -114,14 +114,14 @@ def run_query(instance, query, stdin=None, settings=None): # Test simple put. Also checks that wrong credentials produce an error with every compression method. @pytest.mark.parametrize("maybe_auth,positive,compression", [ - ("", True, 'auto'), - ("'minio','minio123',", True, 'auto'), - ("'wrongid','wrongkey',", False, 'auto'), - ("'wrongid','wrongkey',", False, 'gzip'), - ("'wrongid','wrongkey',", False, 'deflate'), - ("'wrongid','wrongkey',", False, 'brotli'), - ("'wrongid','wrongkey',", False, 'xz'), - ("'wrongid','wrongkey',", False, 'zstd') + pytest.param("", True, 'auto', id="positive"), + pytest.param("'minio','minio123',", True, 'auto', id="auth_positive"), + pytest.param("'wrongid','wrongkey',", False, 'auto', id="auto-"), + pytest.param("'wrongid','wrongkey',", False, 'gzip', id=""), + pytest.param("'wrongid','wrongkey',", False, 'deflate', id=""), + pytest.param("'wrongid','wrongkey',", False, 'brotli', id=""), + pytest.param("'wrongid','wrongkey',", False, 'xz', id=""), + pytest.param("'wrongid','wrongkey',", False, 'zstd, id=""') ]) def test_put(started_cluster, maybe_auth, positive, compression): # type: (ClickHouseCluster) -> None @@ -147,7 +147,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): # Test put no data to S3. @pytest.mark.parametrize("auth", [ - "'minio','minio123'," + pytest.param("'minio','minio123'", id="minio") ]) def test_empty_put(started_cluster, auth): # type: (ClickHouseCluster) -> None @@ -181,9 +181,9 @@ def test_empty_put(started_cluster, auth): # Test put values in CSV format. @pytest.mark.parametrize("maybe_auth,positive", [ - ("", True), - ("'minio','minio123',", True), - ("'wrongid','wrongkey',", False) + pytest.param("", True, id="positive"), + pytest.param("'minio','minio123',", True, id="auth_positive"), + pytest.param("'wrongid','wrongkey',", False, id="negative"), ]) def test_put_csv(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None @@ -285,9 +285,9 @@ def test_put_get_with_globs(started_cluster): # Test multipart put. @pytest.mark.parametrize("maybe_auth,positive", [ - ("", True), + pytest.param("", True, id="positive"), + pytest.param("'wrongid','wrongkey'", False, id="negative"), # ("'minio','minio123',",True), Redirect with credentials not working with nginx. - ("'wrongid','wrongkey',", False), ]) def test_multipart_put(started_cluster, maybe_auth, positive): # type: (ClickHouseCluster) -> None @@ -344,8 +344,8 @@ def test_remote_host_filter(started_cluster): @pytest.mark.parametrize("s3_storage_args", [ - "''", # 1 arguments - "'','','','','',''" # 6 arguments + pytest.param("''", id="1_argument"), + pytest.param("'','','','','',''", id="6_arguments"), ]) def test_wrong_s3_syntax(started_cluster, s3_storage_args): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance @@ -458,8 +458,8 @@ def test_custom_auth_headers_exclusion(started_cluster): assert '403 Forbidden' in ei.value.stderr @pytest.mark.parametrize("extension,method", [ - ("bin", "gzip"), - ("gz", "auto") + pytest.param("bin", "gzip", id="bin"), + pytest.param("gz", "auto", id="gz"), ]) def test_storage_s3_get_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket @@ -532,8 +532,8 @@ def test_storage_s3_put_uncompressed(started_cluster): @pytest.mark.parametrize("extension,method", [ - ("bin", "gzip"), - ("gz", "auto") + pytest.param("bin", "gzip", id="bin"), + pytest.param("gz", "auto", id="gz") ]) def test_storage_s3_put_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket From 7da322f95ab2856c7181769703160ad5885cb312 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 12 Apr 2021 10:04:07 +0300 Subject: [PATCH 0108/1060] better --- src/Server/MySQLHandler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Server/MySQLHandler.cpp b/src/Server/MySQLHandler.cpp index 75c88a6ff93..743a883b68a 100644 --- a/src/Server/MySQLHandler.cpp +++ b/src/Server/MySQLHandler.cpp @@ -219,7 +219,7 @@ void MySQLHandler::finishHandshake(MySQLProtocol::ConnectionPhase::HandshakeResp int ret = socket().receiveBytes(buf + pos, packet_size - pos); if (ret == 0) { - throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3.", ErrorCodes::CANNOT_READ_ALL_DATA); + throw Exception("Cannot read all data. Bytes read: " + std::to_string(pos) + ". Bytes expected: 3", ErrorCodes::CANNOT_READ_ALL_DATA); } pos += ret; } From f022f0744b1f2a6bb80da0c116a9124306efb827 Mon Sep 17 00:00:00 2001 From: vdimir Date: Mon, 12 Apr 2021 11:35:07 +0300 Subject: [PATCH 0109/1060] Add setting allow_experimental_funnel_functions and use in windowFunnel --- .../AggregateFunctionWindowFunnel.cpp | 14 ++++++++++++-- src/Core/Settings.h | 1 + 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp b/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp index ed732a197a1..9ec8e151502 100644 --- a/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp +++ b/src/AggregateFunctions/AggregateFunctionWindowFunnel.cpp @@ -1,7 +1,8 @@ #include #include -#include #include +#include +#include #include #include @@ -14,14 +15,23 @@ namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_ARGUMENT; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int UNKNOWN_AGGREGATE_FUNCTION; } namespace { template