From ce6968367315d7f299dc3dea6ae9152a2628b1b0 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 23 May 2019 09:03:39 +0000 Subject: [PATCH 001/108] Copy-paste from StorageFile. --- dbms/src/Storages/StorageS3.cpp | 335 ++++++++++++++++++++++++++++++++ dbms/src/Storages/StorageS3.h | 89 +++++++++ 2 files changed, 424 insertions(+) create mode 100644 dbms/src/Storages/StorageS3.cpp create mode 100644 dbms/src/Storages/StorageS3.h diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp new file mode 100644 index 00000000000..6f1b62dc240 --- /dev/null +++ b/dbms/src/Storages/StorageS3.cpp @@ -0,0 +1,335 @@ +#include +#include + +#include +#include + +#include +#include + +#include +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; + extern const int CANNOT_SEEK_THROUGH_FILE; + extern const int DATABASE_ACCESS_DENIED; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int UNKNOWN_IDENTIFIER; + extern const int INCORRECT_FILE_NAME; + extern const int FILE_DOESNT_EXIST; + extern const int EMPTY_LIST_OF_COLUMNS_PASSED; +} + + +static std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name) +{ + return db_dir_path + escapeForFileName(table_name) + "/data." + escapeForFileName(format_name); +} + +/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). +static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path, int table_fd) +{ + if (context_global.getApplicationType() != Context::ApplicationType::SERVER) + return; + + if (table_fd >= 0) + throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); + else if (!startsWith(table_path, db_dir_path)) + throw Exception("Part path " + table_path + " is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED); + + Poco::File table_path_poco_file = Poco::File(table_path); + if (!table_path_poco_file.exists()) + throw Exception("File " + table_path + " is not exist", ErrorCodes::FILE_DOESNT_EXIST); + else if (table_path_poco_file.isDirectory()) + throw Exception("File " + table_path + " must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); +} + + +StorageS3::StorageS3( + const std::string & table_path_, + int table_fd_, + const std::string & db_dir_path, + const std::string & table_name_, + const std::string & format_name_, + const ColumnsDescription & columns_, + Context & context_) + : IStorage(columns_), + table_name(table_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) +{ + if (table_fd < 0) /// Will use file + { + use_table_fd = false; + + if (!table_path_.empty()) /// Is user's file + { + Poco::Path poco_path = Poco::Path(table_path_); + if (poco_path.isRelative()) + poco_path = Poco::Path(db_dir_path, poco_path); + + path = poco_path.absolute().toString(); + checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); + is_db_table = false; + } + else /// Is DB's file + { + if (db_dir_path.empty()) + throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); + + path = getTablePath(db_dir_path, table_name, format_name); + is_db_table = true; + Poco::File(Poco::Path(path).parent()).createDirectories(); + } + } + else /// Will use FD + { + checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); + + is_db_table = false; + use_table_fd = true; + + /// Save initial offset, it will be used for repeating SELECTs + /// If FD isn't seekable (lseek returns -1), then the second and subsequent SELECTs will fail. + table_fd_init_offset = lseek(table_fd, 0, SEEK_CUR); + } +} + + +class StorageS3BlockInputStream : public IBlockInputStream +{ +public: + StorageS3BlockInputStream(StorageS3 & storage_, const Context & context, UInt64 max_block_size) + : storage(storage_) + { + if (storage.use_table_fd) + { + unique_lock = std::unique_lock(storage.rwlock); + + /// We could use common ReadBuffer and WriteBuffer in storage to leverage cache + /// and add ability to seek unseekable files, but cache sync isn't supported. + + if (storage.table_fd_was_used) /// We need seek to initial position + { + if (storage.table_fd_init_offset < 0) + throw Exception("File descriptor isn't seekable, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + + /// ReadBuffer's seek() doesn't make sense, since cache is empty + if (lseek(storage.table_fd, storage.table_fd_init_offset, SEEK_SET) < 0) + throwFromErrno("Cannot seek file descriptor, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); + } + + storage.table_fd_was_used = true; + read_buf = std::make_unique(storage.table_fd); + } + else + { + shared_lock = std::shared_lock(storage.rwlock); + + read_buf = std::make_unique(storage.path); + } + + reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); + } + + String getName() const override + { + return storage.getName(); + } + + Block readImpl() override + { + return reader->read(); + } + + Block getHeader() const override { return reader->getHeader(); } + + void readPrefixImpl() override + { + reader->readPrefix(); + } + + void readSuffixImpl() override + { + reader->readSuffix(); + } + +private: + StorageS3 & storage; + Block sample_block; + std::unique_ptr read_buf; + BlockInputStreamPtr reader; + + std::shared_lock shared_lock; + std::unique_lock unique_lock; +}; + + +BlockInputStreams StorageS3::read( + const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & context, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + unsigned /*num_streams*/) +{ + BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size); + const ColumnsDescription & columns = getColumns(); + auto column_defaults = columns.getDefaults(); + if (column_defaults.empty()) + return {block_input}; + return {std::make_shared(block_input, column_defaults, context)}; +} + + +class StorageS3BlockOutputStream : public IBlockOutputStream +{ +public: + explicit StorageS3BlockOutputStream(StorageS3 & storage_) + : storage(storage_), lock(storage.rwlock) + { + if (storage.use_table_fd) + { + /** NOTE: Using real file binded to FD may be misleading: + * SELECT *; INSERT insert_data; SELECT *; last SELECT returns initil_fd_data + insert_data + * INSERT data; SELECT *; last SELECT returns only insert_data + */ + storage.table_fd_was_used = true; + write_buf = std::make_unique(storage.table_fd); + } + else + { + write_buf = std::make_unique(storage.path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); + } + + writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); + } + + Block getHeader() const override { return storage.getSampleBlock(); } + + void write(const Block & block) override + { + writer->write(block); + } + + void writePrefix() override + { + writer->writePrefix(); + } + + void writeSuffix() override + { + writer->writeSuffix(); + } + + void flush() override + { + writer->flush(); + } + +private: + StorageS3 & storage; + std::unique_lock lock; + std::unique_ptr write_buf; + BlockOutputStreamPtr writer; +}; + +BlockOutputStreamPtr StorageS3::write( + const ASTPtr & /*query*/, + const Context & /*context*/) +{ + return std::make_shared(*this); +} + + +void StorageS3::drop() +{ + /// Extra actions are not required. +} + + +void StorageS3::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name) +{ + if (!is_db_table) + throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); + + std::unique_lock lock(rwlock); + + std::string path_new = getTablePath(new_path_to_db, new_table_name, format_name); + Poco::File(Poco::Path(path_new).parent()).createDirectories(); + Poco::File(path).renameTo(path_new); + + path = std::move(path_new); +} + + +void registerStorageS3(StorageFactory & factory) +{ + factory.registerStorage("S3", [](const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + + if (!(engine_args.size() == 1 || engine_args.size() == 2)) + throw Exception( + "Storage S3 requires 1 or 2 arguments: name of used format and source.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); + String format_name = engine_args[0]->as().value.safeGet(); + + int source_fd = -1; + String source_path; + if (engine_args.size() >= 2) + { + /// Will use FD if engine_args[1] is int literal or identifier with std* name + + if (auto opt_name = getIdentifierName(engine_args[1])) + { + if (*opt_name == "stdin") + source_fd = STDIN_FILENO; + else if (*opt_name == "stdout") + source_fd = STDOUT_FILENO; + else if (*opt_name == "stderr") + source_fd = STDERR_FILENO; + else + throw Exception("Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", + ErrorCodes::UNKNOWN_IDENTIFIER); + } + else if (const auto * literal = engine_args[1]->as()) + { + auto type = literal->value.getType(); + if (type == Field::Types::Int64) + source_fd = static_cast(literal->value.get()); + else if (type == Field::Types::UInt64) + source_fd = static_cast(literal->value.get()); + else if (type == Field::Types::String) + source_path = literal->value.get(); + } + } + + return StorageS3::create( + source_path, source_fd, + args.data_path, + args.table_name, format_name, args.columns, + args.context); + }); +} + +} diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h new file mode 100644 index 00000000000..9414fa70cbf --- /dev/null +++ b/dbms/src/Storages/StorageS3.h @@ -0,0 +1,89 @@ +#pragma once + +#include + +#include +#include + +#include + +#include +#include +#include + + +namespace DB +{ + +class StorageS3BlockInputStream; +class StorageS3BlockOutputStream; + +class StorageS3 : public ext::shared_ptr_helper, public IStorage +{ +public: + std::string getName() const override + { + return "S3"; + } + + std::string getTableName() const override + { + return table_name; + } + + BlockInputStreams read( + const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + unsigned num_streams) override; + + BlockOutputStreamPtr write( + const ASTPtr & query, + const Context & context) override; + + void drop() override; + + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + + String getDataPath() const override { return path; } + +protected: + friend class StorageS3BlockInputStream; + friend class StorageS3BlockOutputStream; + + /** there are three options (ordered by priority): + - use specified file descriptor if (fd >= 0) + - use specified table_path if it isn't empty + - create own table inside data/db/table/ + */ + StorageS3( + const std::string & table_path_, + int table_fd_, + const std::string & db_dir_path, + const std::string & table_name_, + const std::string & format_name_, + const ColumnsDescription & columns_, + Context & context_); + +private: + + std::string table_name; + std::string format_name; + Context & context_global; + + std::string path; + int table_fd = -1; + + bool is_db_table = true; /// Table is stored in real database, not user's file + bool use_table_fd = false; /// Use table_fd insted of path + std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin + off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads + + mutable std::shared_mutex rwlock; + + Logger * log = &Logger::get("StorageS3"); +}; + +} From 47985cf8a7a2b93b8453d43ba2582644cf80c386 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 29 May 2019 12:54:31 +0000 Subject: [PATCH 002/108] Wrong commit. --- dbms/src/Storages/StorageS3.cpp | 254 ++++++++++---------------------- dbms/src/Storages/StorageS3.h | 13 +- 2 files changed, 77 insertions(+), 190 deletions(-) diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 6f1b62dc240..080c1c7069e 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -7,9 +7,8 @@ #include #include -#include -#include -#include +#include +#include #include #include @@ -19,10 +18,9 @@ #include #include -#include - #include #include +#include namespace DB { @@ -40,118 +38,43 @@ namespace ErrorCodes } -static std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name) -{ - return db_dir_path + escapeForFileName(table_name) + "/data." + escapeForFileName(format_name); -} - -/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). -static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path, int table_fd) -{ - if (context_global.getApplicationType() != Context::ApplicationType::SERVER) - return; - - if (table_fd >= 0) - throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); - else if (!startsWith(table_path, db_dir_path)) - throw Exception("Part path " + table_path + " is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED); - - Poco::File table_path_poco_file = Poco::File(table_path); - if (!table_path_poco_file.exists()) - throw Exception("File " + table_path + " is not exist", ErrorCodes::FILE_DOESNT_EXIST); - else if (table_path_poco_file.isDirectory()) - throw Exception("File " + table_path + " must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); -} - - StorageS3::StorageS3( - const std::string & table_path_, - int table_fd_, - const std::string & db_dir_path, + const std::string & table_uri_, const std::string & table_name_, const std::string & format_name_, const ColumnsDescription & columns_, Context & context_) - : IStorage(columns_), - table_name(table_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) + : IStorage(columns_) + , table_name(table_name_) + , format_name(format_name_) + , context_global(context_) + , uri(table_uri_) { - if (table_fd < 0) /// Will use file - { - use_table_fd = false; - - if (!table_path_.empty()) /// Is user's file - { - Poco::Path poco_path = Poco::Path(table_path_); - if (poco_path.isRelative()) - poco_path = Poco::Path(db_dir_path, poco_path); - - path = poco_path.absolute().toString(); - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); - is_db_table = false; - } - else /// Is DB's file - { - if (db_dir_path.empty()) - throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - - path = getTablePath(db_dir_path, table_name, format_name); - is_db_table = true; - Poco::File(Poco::Path(path).parent()).createDirectories(); - } - } - else /// Will use FD - { - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); - - is_db_table = false; - use_table_fd = true; - - /// Save initial offset, it will be used for repeating SELECTs - /// If FD isn't seekable (lseek returns -1), then the second and subsequent SELECTs will fail. - table_fd_init_offset = lseek(table_fd, 0, SEEK_CUR); - } } class StorageS3BlockInputStream : public IBlockInputStream { public: - StorageS3BlockInputStream(StorageS3 & storage_, const Context & context, UInt64 max_block_size) - : storage(storage_) + StorageS3BlockInputStream(const Poco::URI & uri, + const std::string & method, + std::function callback, + const String & format, + const String & name_, + const Block & sample_block, + const Context & context, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts) + : name(name_) { - if (storage.use_table_fd) - { - unique_lock = std::unique_lock(storage.rwlock); + read_buf = std::make_unique(uri, method, callback, timeouts); - /// We could use common ReadBuffer and WriteBuffer in storage to leverage cache - /// and add ability to seek unseekable files, but cache sync isn't supported. - - if (storage.table_fd_was_used) /// We need seek to initial position - { - if (storage.table_fd_init_offset < 0) - throw Exception("File descriptor isn't seekable, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - - /// ReadBuffer's seek() doesn't make sense, since cache is empty - if (lseek(storage.table_fd, storage.table_fd_init_offset, SEEK_SET) < 0) - throwFromErrno("Cannot seek file descriptor, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - } - - storage.table_fd_was_used = true; - read_buf = std::make_unique(storage.table_fd); - } - else - { - shared_lock = std::shared_lock(storage.rwlock); - - read_buf = std::make_unique(storage.path); - } - - reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); + reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } String getName() const override { - return storage.getName(); + return name; } Block readImpl() override @@ -159,7 +82,10 @@ public: return reader->read(); } - Block getHeader() const override { return reader->getHeader(); } + Block getHeader() const override + { + return reader->getHeader(); + } void readPrefixImpl() override { @@ -172,27 +98,35 @@ public: } private: - StorageS3 & storage; - Block sample_block; - std::unique_ptr read_buf; + String name; + std::unique_ptr read_buf; BlockInputStreamPtr reader; - - std::shared_lock shared_lock; - std::unique_lock unique_lock; }; BlockInputStreams StorageS3::read( - const Names & /*column_names*/, + const Names & column_names, const SelectQueryInfo & /*query_info*/, const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned /*num_streams*/) { - BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size); - const ColumnsDescription & columns = getColumns(); - auto column_defaults = columns.getDefaults(); + auto request_uri = uri; + + BlockInputStreamPtr block_input = std::make_shared(request_uri, + Poco::Net::HTTPRequest::HTTP_GET, + nullptr, + //getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size), + format_name, + getName(), + getSampleBlockForColumns(column_names), + context, + max_block_size, + ConnectionTimeouts::getHTTPTimeouts(context)); + + + auto column_defaults = getColumns().getDefaults(); if (column_defaults.empty()) return {block_input}; return {std::make_shared(block_input, column_defaults, context)}; @@ -202,27 +136,21 @@ BlockInputStreams StorageS3::read( class StorageS3BlockOutputStream : public IBlockOutputStream { public: - explicit StorageS3BlockOutputStream(StorageS3 & storage_) - : storage(storage_), lock(storage.rwlock) + StorageS3BlockOutputStream(const Poco::URI & uri, + const String & format, + const Block & sample_block_, + const Context & context, + const ConnectionTimeouts & timeouts) + : sample_block(sample_block_) { - if (storage.use_table_fd) - { - /** NOTE: Using real file binded to FD may be misleading: - * SELECT *; INSERT insert_data; SELECT *; last SELECT returns initil_fd_data + insert_data - * INSERT data; SELECT *; last SELECT returns only insert_data - */ - storage.table_fd_was_used = true; - write_buf = std::make_unique(storage.table_fd); - } - else - { - write_buf = std::make_unique(storage.path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); - } - - writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); + write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); + writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } - Block getHeader() const override { return storage.getSampleBlock(); } + Block getHeader() const override + { + return sample_block; + } void write(const Block & block) override { @@ -237,17 +165,13 @@ public: void writeSuffix() override { writer->writeSuffix(); - } - - void flush() override - { writer->flush(); + write_buf->finalize(); } private: - StorageS3 & storage; - std::unique_lock lock; - std::unique_ptr write_buf; + Block sample_block; + std::unique_ptr write_buf; BlockOutputStreamPtr writer; }; @@ -255,7 +179,8 @@ BlockOutputStreamPtr StorageS3::write( const ASTPtr & /*query*/, const Context & /*context*/) { - return std::make_shared(*this); + return std::make_shared( + uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); } @@ -265,19 +190,7 @@ void StorageS3::drop() } -void StorageS3::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name) -{ - if (!is_db_table) - throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); - - std::unique_lock lock(rwlock); - - std::string path_new = getTablePath(new_path_to_db, new_table_name, format_name); - Poco::File(Poco::Path(path_new).parent()).createDirectories(); - Poco::File(path).renameTo(path_new); - - path = std::move(path_new); -} +void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {} void registerStorageS3(StorageFactory & factory) @@ -286,49 +199,30 @@ void registerStorageS3(StorageFactory & factory) { ASTs & engine_args = args.engine_args; - if (!(engine_args.size() == 1 || engine_args.size() == 2)) + if (!(engine_args.size() == 2)) throw Exception( - "Storage S3 requires 1 or 2 arguments: name of used format and source.", + "Storage S3 requires 2 arguments: name of used format and source.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); String format_name = engine_args[0]->as().value.safeGet(); - int source_fd = -1; String source_path; - if (engine_args.size() >= 2) + if (const auto * literal = engine_args[1]->as()) { - /// Will use FD if engine_args[1] is int literal or identifier with std* name - - if (auto opt_name = getIdentifierName(engine_args[1])) + auto type = literal->value.getType(); + if (type == Field::Types::String) { - if (*opt_name == "stdin") - source_fd = STDIN_FILENO; - else if (*opt_name == "stdout") - source_fd = STDOUT_FILENO; - else if (*opt_name == "stderr") - source_fd = STDERR_FILENO; - else - throw Exception("Unknown identifier '" + *opt_name + "' in second arg of File storage constructor", - ErrorCodes::UNKNOWN_IDENTIFIER); - } - else if (const auto * literal = engine_args[1]->as()) - { - auto type = literal->value.getType(); - if (type == Field::Types::Int64) - source_fd = static_cast(literal->value.get()); - else if (type == Field::Types::UInt64) - source_fd = static_cast(literal->value.get()); - else if (type == Field::Types::String) - source_path = literal->value.get(); + source_path = literal->value.get(); + return StorageS3::create( + source_path, + args.table_name, format_name, args.columns, + args.context); } } - return StorageS3::create( - source_path, source_fd, - args.data_path, - args.table_name, format_name, args.columns, - args.context); + throw Exception("Unknown entity in first arg of S3 storage constructor, String expected.", + ErrorCodes::UNKNOWN_IDENTIFIER); }); } diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 9414fa70cbf..263e5033962 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -4,6 +4,7 @@ #include #include +#include #include @@ -47,8 +48,6 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - String getDataPath() const override { return path; } - protected: friend class StorageS3BlockInputStream; friend class StorageS3BlockOutputStream; @@ -59,9 +58,7 @@ protected: - create own table inside data/db/table/ */ StorageS3( - const std::string & table_path_, - int table_fd_, - const std::string & db_dir_path, + const std::string & table_uri_, const std::string & table_name_, const std::string & format_name_, const ColumnsDescription & columns_, @@ -73,13 +70,9 @@ private: std::string format_name; Context & context_global; - std::string path; - int table_fd = -1; + Poco::URI uri; bool is_db_table = true; /// Table is stored in real database, not user's file - bool use_table_fd = false; /// Use table_fd insted of path - std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin - off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads mutable std::shared_mutex rwlock; From ff09934219ba99af077b4d5665fa664f290769a8 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 31 May 2019 07:27:14 +0000 Subject: [PATCH 003/108] Table function and storage. --- dbms/src/Storages/StorageS3.cpp | 292 ++++++++---------- dbms/src/Storages/StorageS3.h | 102 +++--- dbms/src/Storages/registerStorages.cpp | 2 + dbms/src/TableFunctions/TableFunctionS3.cpp | 19 ++ dbms/src/TableFunctions/TableFunctionS3.h | 25 ++ .../TableFunctions/registerTableFunctions.cpp | 2 + 6 files changed, 232 insertions(+), 210 deletions(-) create mode 100644 dbms/src/TableFunctions/TableFunctionS3.cpp create mode 100644 dbms/src/TableFunctions/TableFunctionS3.h diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 080c1c7069e..f49cd9e7a9e 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -1,126 +1,176 @@ -#include #include +#include #include #include - #include -#include #include #include #include -#include + #include +#include #include -#include -#include - -#include -#include #include + namespace DB { - namespace ErrorCodes { - extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; - extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int DATABASE_ACCESS_DENIED; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int UNKNOWN_IDENTIFIER; - extern const int INCORRECT_FILE_NAME; - extern const int FILE_DOESNT_EXIST; - extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } - -StorageS3::StorageS3( - const std::string & table_uri_, - const std::string & table_name_, - const std::string & format_name_, - const ColumnsDescription & columns_, - Context & context_) - : IStorage(columns_) - , table_name(table_name_) - , format_name(format_name_) - , context_global(context_) - , uri(table_uri_) +IStorageS3Base::IStorageS3Base(const Poco::URI & uri_, + const Context & context_, + const std::string & table_name_, + const String & format_name_, + const ColumnsDescription & columns_) + : IStorage(columns_), uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_) { } - -class StorageS3BlockInputStream : public IBlockInputStream +namespace { -public: - StorageS3BlockInputStream(const Poco::URI & uri, - const std::string & method, - std::function callback, - const String & format, - const String & name_, - const Block & sample_block, - const Context & context, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts) - : name(name_) + class StorageS3BlockInputStream : public IBlockInputStream { - read_buf = std::make_unique(uri, method, callback, timeouts); + public: + StorageS3BlockInputStream(const Poco::URI & uri, + const std::string & method, + std::function callback, + const String & format, + const String & name_, + const Block & sample_block, + const Context & context, + UInt64 max_block_size, + const ConnectionTimeouts & timeouts) + : name(name_) + { + read_buf = std::make_unique(uri, method, callback, timeouts); - reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); - } + reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); + } - String getName() const override + String getName() const override + { + return name; + } + + Block readImpl() override + { + return reader->read(); + } + + Block getHeader() const override + { + return reader->getHeader(); + } + + void readPrefixImpl() override + { + reader->readPrefix(); + } + + void readSuffixImpl() override + { + reader->readSuffix(); + } + + private: + String name; + std::unique_ptr read_buf; + BlockInputStreamPtr reader; + }; + + class StorageS3BlockOutputStream : public IBlockOutputStream { - return name; - } + public: + StorageS3BlockOutputStream(const Poco::URI & uri, + const String & format, + const Block & sample_block_, + const Context & context, + const ConnectionTimeouts & timeouts) + : sample_block(sample_block_) + { + write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); + writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); + } - Block readImpl() override - { - return reader->read(); - } + Block getHeader() const override + { + return sample_block; + } - Block getHeader() const override - { - return reader->getHeader(); - } + void write(const Block & block) override + { + writer->write(block); + } - void readPrefixImpl() override - { - reader->readPrefix(); - } + void writePrefix() override + { + writer->writePrefix(); + } - void readSuffixImpl() override - { - reader->readSuffix(); - } + void writeSuffix() override + { + writer->writeSuffix(); + writer->flush(); + write_buf->finalize(); + } -private: - String name; - std::unique_ptr read_buf; - BlockInputStreamPtr reader; -}; + private: + Block sample_block; + std::unique_ptr write_buf; + BlockOutputStreamPtr writer; + }; +} -BlockInputStreams StorageS3::read( - const Names & column_names, +std::string IStorageS3Base::getReadMethod() const +{ + return Poco::Net::HTTPRequest::HTTP_GET; +} + +std::vector> IStorageS3Base::getReadURIParams(const Names & /*column_names*/, const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum & /*processed_stage*/, + size_t /*max_block_size*/) const +{ + return {}; +} + +std::function IStorageS3Base::getReadPOSTDataCallback(const Names & /*column_names*/, + const SelectQueryInfo & /*query_info*/, + const Context & /*context*/, + QueryProcessingStage::Enum & /*processed_stage*/, + size_t /*max_block_size*/) const +{ + return nullptr; +} + + +BlockInputStreams IStorageS3Base::read(const Names & column_names, + const SelectQueryInfo & query_info, const Context & context, - QueryProcessingStage::Enum /*processed_stage*/, + QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned /*num_streams*/) { auto request_uri = uri; + auto params = getReadURIParams(column_names, query_info, context, processed_stage, max_block_size); + for (const auto & [param, value] : params) + request_uri.addQueryParameter(param, value); BlockInputStreamPtr block_input = std::make_shared(request_uri, - Poco::Net::HTTPRequest::HTTP_GET, - nullptr, - //getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size), + getReadMethod(), + getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size), format_name, getName(), - getSampleBlockForColumns(column_names), + getHeaderBlock(column_names), context, max_block_size, ConnectionTimeouts::getHTTPTimeouts(context)); @@ -132,98 +182,16 @@ BlockInputStreams StorageS3::read( return {std::make_shared(block_input, column_defaults, context)}; } +void IStorageS3Base::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {} -class StorageS3BlockOutputStream : public IBlockOutputStream -{ -public: - StorageS3BlockOutputStream(const Poco::URI & uri, - const String & format, - const Block & sample_block_, - const Context & context, - const ConnectionTimeouts & timeouts) - : sample_block(sample_block_) - { - write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); - writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); - } - - Block getHeader() const override - { - return sample_block; - } - - void write(const Block & block) override - { - writer->write(block); - } - - void writePrefix() override - { - writer->writePrefix(); - } - - void writeSuffix() override - { - writer->writeSuffix(); - writer->flush(); - write_buf->finalize(); - } - -private: - Block sample_block; - std::unique_ptr write_buf; - BlockOutputStreamPtr writer; -}; - -BlockOutputStreamPtr StorageS3::write( - const ASTPtr & /*query*/, - const Context & /*context*/) +BlockOutputStreamPtr IStorageS3Base::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared( uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); } - -void StorageS3::drop() +void registerStorageS3(StorageFactory & /*factory*/) { - /// Extra actions are not required. + // TODO. See #1394. } - - -void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {} - - -void registerStorageS3(StorageFactory & factory) -{ - factory.registerStorage("S3", [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - - if (!(engine_args.size() == 2)) - throw Exception( - "Storage S3 requires 2 arguments: name of used format and source.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); - String format_name = engine_args[0]->as().value.safeGet(); - - String source_path; - if (const auto * literal = engine_args[1]->as()) - { - auto type = literal->value.getType(); - if (type == Field::Types::String) - { - source_path = literal->value.get(); - return StorageS3::create( - source_path, - args.table_name, format_name, args.columns, - args.context); - } - } - - throw Exception("Unknown entity in first arg of S3 storage constructor, String expected.", - ErrorCodes::UNKNOWN_IDENTIFIER); - }); -} - } diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 263e5033962..2615563b57c 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -1,82 +1,88 @@ #pragma once #include - -#include -#include #include - #include - -#include -#include #include - namespace DB { - -class StorageS3BlockInputStream; -class StorageS3BlockOutputStream; - -class StorageS3 : public ext::shared_ptr_helper, public IStorage +/** + * This class represents table engine for external urls. + * It sends HTTP GET to server when select is called and + * HTTP POST when insert is called. In POST request the data is send + * using Chunked transfer encoding, so server have to support it. + */ +class IStorageS3Base : public IStorage { public: - std::string getName() const override - { - return "S3"; - } - - std::string getTableName() const override + String getTableName() const override { return table_name; } - BlockInputStreams read( - const Names & column_names, + BlockInputStreams read(const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, size_t max_block_size, unsigned num_streams) override; - BlockOutputStreamPtr write( - const ASTPtr & query, - const Context & context) override; - - void drop() override; + BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; protected: - friend class StorageS3BlockInputStream; - friend class StorageS3BlockOutputStream; - - /** there are three options (ordered by priority): - - use specified file descriptor if (fd >= 0) - - use specified table_path if it isn't empty - - create own table inside data/db/table/ - */ - StorageS3( - const std::string & table_uri_, + IStorageS3Base(const Poco::URI & uri_, + const Context & context_, const std::string & table_name_, - const std::string & format_name_, - const ColumnsDescription & columns_, - Context & context_); - -private: - - std::string table_name; - std::string format_name; - Context & context_global; + const String & format_name_, + const ColumnsDescription & columns_); Poco::URI uri; + const Context & context_global; - bool is_db_table = true; /// Table is stored in real database, not user's file +private: + String format_name; + String table_name; - mutable std::shared_mutex rwlock; + virtual std::string getReadMethod() const; - Logger * log = &Logger::get("StorageS3"); + virtual std::vector> getReadURIParams(const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size) const; + + virtual std::function getReadPOSTDataCallback(const Names & column_names, + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + size_t max_block_size) const; + + virtual Block getHeaderBlock(const Names & column_names) const = 0; }; +class StorageS3 : public ext::shared_ptr_helper, public IStorageS3Base +{ +public: + StorageS3(const Poco::URI & uri_, + const std::string & table_name_, + const String & format_name_, + const ColumnsDescription & columns_, + Context & context_) + : IStorageS3Base(uri_, context_, table_name_, format_name_, columns_) + { + } + + String getName() const override + { + return "S3"; + } + + Block getHeaderBlock(const Names & /*column_names*/) const override + { + return getSampleBlock(); + } +}; } diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index c21156ea44d..4c29884dfcf 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -19,6 +19,7 @@ void registerStorageDistributed(StorageFactory & factory); void registerStorageMemory(StorageFactory & factory); void registerStorageFile(StorageFactory & factory); void registerStorageURL(StorageFactory & factory); +void registerStorageS3(StorageFactory & factory); void registerStorageDictionary(StorageFactory & factory); void registerStorageSet(StorageFactory & factory); void registerStorageJoin(StorageFactory & factory); @@ -60,6 +61,7 @@ void registerStorages() registerStorageMemory(factory); registerStorageFile(factory); registerStorageURL(factory); + registerStorageS3(factory); registerStorageDictionary(factory); registerStorageSet(factory); registerStorageJoin(factory); diff --git a/dbms/src/TableFunctions/TableFunctionS3.cpp b/dbms/src/TableFunctions/TableFunctionS3.cpp new file mode 100644 index 00000000000..5c2c6215765 --- /dev/null +++ b/dbms/src/TableFunctions/TableFunctionS3.cpp @@ -0,0 +1,19 @@ +#include +#include +#include +#include + +namespace DB +{ +StoragePtr TableFunctionS3::getStorage( + const String & source, const String & format, const Block & sample_block, Context & global_context) const +{ + Poco::URI uri(source); + return StorageS3::create(uri, getName(), format, ColumnsDescription{sample_block.getNamesAndTypesList()}, global_context); +} + +void registerTableFunctionS3(TableFunctionFactory & factory) +{ + factory.registerFunction(); +} +} diff --git a/dbms/src/TableFunctions/TableFunctionS3.h b/dbms/src/TableFunctions/TableFunctionS3.h new file mode 100644 index 00000000000..83c49e0b8d1 --- /dev/null +++ b/dbms/src/TableFunctions/TableFunctionS3.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +/* url(source, format, structure) - creates a temporary storage from url + */ +class TableFunctionS3 : public ITableFunctionFileLike +{ +public: + static constexpr auto name = "s3"; + std::string getName() const override + { + return name; + } + +private: + StoragePtr getStorage( + const String & source, const String & format, const Block & sample_block, Context & global_context) const override; +}; +} diff --git a/dbms/src/TableFunctions/registerTableFunctions.cpp b/dbms/src/TableFunctions/registerTableFunctions.cpp index 61d0ec23f7d..aad5eebe935 100644 --- a/dbms/src/TableFunctions/registerTableFunctions.cpp +++ b/dbms/src/TableFunctions/registerTableFunctions.cpp @@ -11,6 +11,7 @@ void registerTableFunctionMerge(TableFunctionFactory & factory); void registerTableFunctionRemote(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory); void registerTableFunctionFile(TableFunctionFactory & factory); +void registerTableFunctionS3(TableFunctionFactory & factory); void registerTableFunctionURL(TableFunctionFactory & factory); void registerTableFunctionValues(TableFunctionFactory & factory); @@ -37,6 +38,7 @@ void registerTableFunctions() registerTableFunctionRemote(factory); registerTableFunctionNumbers(factory); registerTableFunctionFile(factory); + registerTableFunctionS3(factory); registerTableFunctionURL(factory); registerTableFunctionValues(factory); From 78f57c5f2a6aa1c7a4dfb87d31b7c5702e60eaec Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 31 May 2019 10:58:43 +0000 Subject: [PATCH 004/108] First attempt to make redirects. --- dbms/src/IO/HTTPCommon.cpp | 6 +- dbms/src/IO/HTTPCommon.h | 2 + dbms/src/IO/ReadWriteBufferFromS3.cpp | 29 +++++ dbms/src/IO/ReadWriteBufferFromS3.h | 147 ++++++++++++++++++++++++++ dbms/src/Storages/StorageS3.cpp | 11 +- 5 files changed, 188 insertions(+), 7 deletions(-) create mode 100644 dbms/src/IO/ReadWriteBufferFromS3.cpp create mode 100644 dbms/src/IO/ReadWriteBufferFromS3.h diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index ca5b5ab700b..53d2ac8c2dd 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -217,6 +217,11 @@ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response) { auto istr = &session.receiveResponse(response); + assertResponseIsOk(request, response, istr); + return istr; +} + +void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream * istr) { auto status = response.getStatus(); if (status != Poco::Net::HTTPResponse::HTTP_OK) @@ -229,7 +234,6 @@ std::istream * receiveResponse( status == HTTP_TOO_MANY_REQUESTS ? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS : ErrorCodes::RECEIVED_ERROR_FROM_REMOTE_IO_SERVER); } - return istr; } } diff --git a/dbms/src/IO/HTTPCommon.h b/dbms/src/IO/HTTPCommon.h index 6dc669c248e..1e7500cf230 100644 --- a/dbms/src/IO/HTTPCommon.h +++ b/dbms/src/IO/HTTPCommon.h @@ -57,4 +57,6 @@ PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const Connecti */ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response); +void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream * istr); + } diff --git a/dbms/src/IO/ReadWriteBufferFromS3.cpp b/dbms/src/IO/ReadWriteBufferFromS3.cpp new file mode 100644 index 00000000000..0ead1bdf32d --- /dev/null +++ b/dbms/src/IO/ReadWriteBufferFromS3.cpp @@ -0,0 +1,29 @@ +#include + +#include + + +namespace DB +{ + +WriteBufferFromS3::WriteBufferFromS3( + const Poco::URI & uri, const std::string & method, const ConnectionTimeouts & timeouts, size_t buffer_size_) + : WriteBufferFromOStream(buffer_size_) + , session{makeHTTPSession(uri, timeouts)} + , request{method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} +{ + request.setHost(uri.getHost()); + request.setChunkedTransferEncoding(true); + + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); + + ostr = &session->sendRequest(request); +} + +void WriteBufferFromS3::finalize() +{ + receiveResponse(*session, request, response); + /// TODO: Response body is ignored. +} + +} diff --git a/dbms/src/IO/ReadWriteBufferFromS3.h b/dbms/src/IO/ReadWriteBufferFromS3.h new file mode 100644 index 00000000000..1257031ae2a --- /dev/null +++ b/dbms/src/IO/ReadWriteBufferFromS3.h @@ -0,0 +1,147 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#define DEFAULT_S3_READ_BUFFER_TIMEOUT 1800 +#define DEFAULT_S3_READ_BUFFER_CONNECTION_TIMEOUT 1 +#define DEFAULT_S3_MAX_FOLLOW_REDIRECT 2 + +namespace DB +{ +/** Perform S3 HTTP POST request and provide response to read. + */ + +namespace detail +{ + template + class ReadWriteBufferFromS3Base : public ReadBuffer + { + protected: + Poco::URI uri; + std::string method; + + SessionPtr session; + std::istream * istr; /// owned by session + std::unique_ptr impl; + + public: + using OutStreamCallback = std::function; + + explicit ReadWriteBufferFromS3Base(SessionPtr session_, + Poco::URI uri, + const std::string & method = {}, + OutStreamCallback out_stream_callback = {}, + const Poco::Net::HTTPBasicCredentials & credentials = {}, + size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE) + : ReadBuffer(nullptr, 0) + , uri {uri} + , method {!method.empty() ? method : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} + , session {session_} + { + Poco::Net::HTTPResponse response; + std::unique_ptr request; + + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_REDIRECT; ++i) + { + // With empty path poco will send "POST HTTP/1.1" its bug. + if (uri.getPath().empty()) + uri.setPath("/"); + + request = std::make_unique(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request->setHost(uri.getHost()); // use original, not resolved host name in header + + if (out_stream_callback) + request->setChunkedTransferEncoding(true); + + if (!credentials.getUsername().empty()) + credentials.authenticate(*request); + + LOG_TRACE((&Logger::get("ReadWriteBufferFromS3")), "Sending request to " << uri.toString()); + + auto & stream_out = session->sendRequest(*request); + + if (out_stream_callback) + out_stream_callback(stream_out); + + istr = &session->receiveResponse(response); + + if (response.getStatus() != 307) + break; + + auto location_iterator = response.find("Location"); + if (location_iterator == response.end()) + break; + + uri = location_iterator->second; + } + + assertResponseIsOk(*request, response, istr); + impl = std::make_unique(*istr, buffer_size_); + } + + + bool nextImpl() override + { + if (!impl->next()) + return false; + internal_buffer = impl->buffer(); + working_buffer = internal_buffer; + return true; + } + }; +} + +class ReadWriteBufferFromS3 : public detail::ReadWriteBufferFromS3Base +{ + using Parent = detail::ReadWriteBufferFromS3Base; + +public: + explicit ReadWriteBufferFromS3(Poco::URI uri_, + const std::string & method_ = {}, + OutStreamCallback out_stream_callback = {}, + const ConnectionTimeouts & timeouts = {}, + const Poco::Net::HTTPBasicCredentials & credentials = {}, + size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE) + : Parent(makeHTTPSession(uri_, timeouts), uri_, method_, out_stream_callback, credentials, buffer_size_) + { + } +}; + +/* Perform S3 HTTP POST/PUT request. + */ +class WriteBufferFromS3 : public WriteBufferFromOStream +{ +private: + HTTPSessionPtr session; + Poco::Net::HTTPRequest request; + Poco::Net::HTTPResponse response; + +public: + explicit WriteBufferFromS3(const Poco::URI & uri, + const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only + const ConnectionTimeouts & timeouts = {}, + size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); + + /// Receives response from the server after sending all data. + void finalize(); +}; + +} diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index f49cd9e7a9e..972820a8449 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -5,8 +5,7 @@ #include #include -#include -#include +#include #include @@ -49,7 +48,7 @@ namespace const ConnectionTimeouts & timeouts) : name(name_) { - read_buf = std::make_unique(uri, method, callback, timeouts); + read_buf = std::make_unique(uri, method, callback, timeouts); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } @@ -81,7 +80,7 @@ namespace private: String name; - std::unique_ptr read_buf; + std::unique_ptr read_buf; BlockInputStreamPtr reader; }; @@ -95,7 +94,7 @@ namespace const ConnectionTimeouts & timeouts) : sample_block(sample_block_) { - write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); + write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } @@ -123,7 +122,7 @@ namespace private: Block sample_block; - std::unique_ptr write_buf; + std::unique_ptr write_buf; BlockOutputStreamPtr writer; }; } From caeacafb7676ac7e6ee5d17e1b4dbf75045cc5c2 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 31 May 2019 15:50:21 +0000 Subject: [PATCH 005/108] Fixed GET redirects. --- dbms/src/IO/HTTPCommon.cpp | 2 +- dbms/src/IO/ReadWriteBufferFromS3.h | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index 53d2ac8c2dd..32970276dd4 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -45,7 +45,7 @@ namespace ErrorCodes namespace { -void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) + void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts) { #if defined(POCO_CLICKHOUSE_PATCH) || POCO_VERSION >= 0x02000000 session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout); diff --git a/dbms/src/IO/ReadWriteBufferFromS3.h b/dbms/src/IO/ReadWriteBufferFromS3.h index 1257031ae2a..01fb0fb890a 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.h +++ b/dbms/src/IO/ReadWriteBufferFromS3.h @@ -31,7 +31,7 @@ namespace DB namespace detail { - template + template //FIXME Можно избавиться от template, или переделать на нормальное. class ReadWriteBufferFromS3Base : public ReadBuffer { protected: @@ -45,8 +45,8 @@ namespace detail public: using OutStreamCallback = std::function; - explicit ReadWriteBufferFromS3Base(SessionPtr session_, - Poco::URI uri, + explicit ReadWriteBufferFromS3Base(Poco::URI uri, + const ConnectionTimeouts & timeouts = {}, const std::string & method = {}, OutStreamCallback out_stream_callback = {}, const Poco::Net::HTTPBasicCredentials & credentials = {}, @@ -54,7 +54,7 @@ namespace detail : ReadBuffer(nullptr, 0) , uri {uri} , method {!method.empty() ? method : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} - , session {session_} + , session(makeHTTPSession(uri, timeouts)) { Poco::Net::HTTPResponse response; std::unique_ptr request; @@ -91,6 +91,7 @@ namespace detail break; uri = location_iterator->second; + session = makeHTTPSession(uri, timeouts); } assertResponseIsOk(*request, response, istr); @@ -120,7 +121,7 @@ public: const ConnectionTimeouts & timeouts = {}, const Poco::Net::HTTPBasicCredentials & credentials = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE) - : Parent(makeHTTPSession(uri_, timeouts), uri_, method_, out_stream_callback, credentials, buffer_size_) + : Parent(uri_, timeouts, method_, out_stream_callback, credentials, buffer_size_) { } }; From 531460396dcc8f059c447028f3b2c3c5b64921b3 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 31 May 2019 17:57:58 +0000 Subject: [PATCH 006/108] POST S3 requests. --- dbms/src/IO/ReadWriteBufferFromS3.cpp | 1 + dbms/src/IO/ReadWriteBufferFromS3.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/IO/ReadWriteBufferFromS3.cpp b/dbms/src/IO/ReadWriteBufferFromS3.cpp index 0ead1bdf32d..05fa3569572 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.cpp +++ b/dbms/src/IO/ReadWriteBufferFromS3.cpp @@ -14,6 +14,7 @@ WriteBufferFromS3::WriteBufferFromS3( { request.setHost(uri.getHost()); request.setChunkedTransferEncoding(true); + request.setExpectContinue(true); LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); diff --git a/dbms/src/IO/ReadWriteBufferFromS3.h b/dbms/src/IO/ReadWriteBufferFromS3.h index 01fb0fb890a..e28e51fb89c 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.h +++ b/dbms/src/IO/ReadWriteBufferFromS3.h @@ -137,7 +137,7 @@ private: public: explicit WriteBufferFromS3(const Poco::URI & uri, - const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST or PUT only + const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST for inserting, PUT for replacing. const ConnectionTimeouts & timeouts = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); From 062db0ec141e14e096c9b472f8e9a296265e8bd1 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 31 May 2019 18:14:39 +0000 Subject: [PATCH 007/108] Removed templateness of ReadWriteBufferFromS3. --- dbms/src/IO/ReadWriteBufferFromS3.cpp | 63 +++++++++++++++++ dbms/src/IO/ReadWriteBufferFromS3.h | 99 ++++----------------------- 2 files changed, 76 insertions(+), 86 deletions(-) diff --git a/dbms/src/IO/ReadWriteBufferFromS3.cpp b/dbms/src/IO/ReadWriteBufferFromS3.cpp index 05fa3569572..f4b6124267f 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.cpp +++ b/dbms/src/IO/ReadWriteBufferFromS3.cpp @@ -6,6 +6,69 @@ namespace DB { +ReadWriteBufferFromS3::ReadWriteBufferFromS3(Poco::URI uri_, + const std::string & method_, + OutStreamCallback out_stream_callback, + const ConnectionTimeouts & timeouts, + const Poco::Net::HTTPBasicCredentials & credentials, + size_t buffer_size_) + : ReadBuffer(nullptr, 0) + , uri {uri_} + , method {!method_.empty() ? method_ : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} + , session(makeHTTPSession(uri_, timeouts)) +{ + Poco::Net::HTTPResponse response; + std::unique_ptr request; + + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_REDIRECT; ++i) + { + // With empty path poco will send "POST HTTP/1.1" its bug. + if (uri.getPath().empty()) + uri.setPath("/"); + + request = std::make_unique(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request->setHost(uri.getHost()); // use original, not resolved host name in header + + if (out_stream_callback) + request->setChunkedTransferEncoding(true); + + if (!credentials.getUsername().empty()) + credentials.authenticate(*request); + + LOG_TRACE((&Logger::get("ReadWriteBufferFromS3")), "Sending request to " << uri.toString()); + + auto & stream_out = session->sendRequest(*request); + + if (out_stream_callback) + out_stream_callback(stream_out); + + istr = &session->receiveResponse(response); + + if (response.getStatus() != 307) + break; + + auto location_iterator = response.find("Location"); + if (location_iterator == response.end()) + break; + + uri = location_iterator->second; + session = makeHTTPSession(uri, timeouts); + } + + assertResponseIsOk(*request, response, istr); + impl = std::make_unique(*istr, buffer_size_); +} + + +bool ReadWriteBufferFromS3::nextImpl() +{ + if (!impl->next()) + return false; + internal_buffer = impl->buffer(); + working_buffer = internal_buffer; + return true; +} + WriteBufferFromS3::WriteBufferFromS3( const Poco::URI & uri, const std::string & method, const ConnectionTimeouts & timeouts, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) diff --git a/dbms/src/IO/ReadWriteBufferFromS3.h b/dbms/src/IO/ReadWriteBufferFromS3.h index e28e51fb89c..e33dbe80df0 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.h +++ b/dbms/src/IO/ReadWriteBufferFromS3.h @@ -29,103 +29,30 @@ namespace DB /** Perform S3 HTTP POST request and provide response to read. */ -namespace detail +class ReadWriteBufferFromS3 : public ReadBuffer { - template //FIXME Можно избавиться от template, или переделать на нормальное. - class ReadWriteBufferFromS3Base : public ReadBuffer - { - protected: - Poco::URI uri; - std::string method; +protected: + Poco::URI uri; + std::string method; - SessionPtr session; - std::istream * istr; /// owned by session - std::unique_ptr impl; - - public: - using OutStreamCallback = std::function; - - explicit ReadWriteBufferFromS3Base(Poco::URI uri, - const ConnectionTimeouts & timeouts = {}, - const std::string & method = {}, - OutStreamCallback out_stream_callback = {}, - const Poco::Net::HTTPBasicCredentials & credentials = {}, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE) - : ReadBuffer(nullptr, 0) - , uri {uri} - , method {!method.empty() ? method : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} - , session(makeHTTPSession(uri, timeouts)) - { - Poco::Net::HTTPResponse response; - std::unique_ptr request; - - for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_REDIRECT; ++i) - { - // With empty path poco will send "POST HTTP/1.1" its bug. - if (uri.getPath().empty()) - uri.setPath("/"); - - request = std::make_unique(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - request->setHost(uri.getHost()); // use original, not resolved host name in header - - if (out_stream_callback) - request->setChunkedTransferEncoding(true); - - if (!credentials.getUsername().empty()) - credentials.authenticate(*request); - - LOG_TRACE((&Logger::get("ReadWriteBufferFromS3")), "Sending request to " << uri.toString()); - - auto & stream_out = session->sendRequest(*request); - - if (out_stream_callback) - out_stream_callback(stream_out); - - istr = &session->receiveResponse(response); - - if (response.getStatus() != 307) - break; - - auto location_iterator = response.find("Location"); - if (location_iterator == response.end()) - break; - - uri = location_iterator->second; - session = makeHTTPSession(uri, timeouts); - } - - assertResponseIsOk(*request, response, istr); - impl = std::make_unique(*istr, buffer_size_); - } - - - bool nextImpl() override - { - if (!impl->next()) - return false; - internal_buffer = impl->buffer(); - working_buffer = internal_buffer; - return true; - } - }; -} - -class ReadWriteBufferFromS3 : public detail::ReadWriteBufferFromS3Base -{ - using Parent = detail::ReadWriteBufferFromS3Base; + HTTPSessionPtr session; + std::istream * istr; /// owned by session + std::unique_ptr impl; public: + using OutStreamCallback = std::function; + explicit ReadWriteBufferFromS3(Poco::URI uri_, const std::string & method_ = {}, OutStreamCallback out_stream_callback = {}, const ConnectionTimeouts & timeouts = {}, const Poco::Net::HTTPBasicCredentials & credentials = {}, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE) - : Parent(uri_, timeouts, method_, out_stream_callback, credentials, buffer_size_) - { - } + size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); + + bool nextImpl() override; }; + /* Perform S3 HTTP POST/PUT request. */ class WriteBufferFromS3 : public WriteBufferFromOStream From 52f242daf056a4d6725de1b87fd7436769ede79e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 31 May 2019 18:16:40 +0000 Subject: [PATCH 008/108] tests (WIP) --- .../clickhouse-test | 126 ++++++++++++++++++ .../00950_table_function_s3_wip/config.xml | 115 ++++++++++++++++ 2 files changed, 241 insertions(+) create mode 100755 dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test create mode 100644 dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test new file mode 100755 index 00000000000..09876ea0df6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -0,0 +1,126 @@ +#!/usr/bin/env python3 + +import http.server +import os +import subprocess +import threading +import unittest + + +format = 'column1 UInt32, column2 UInt32, column3 UInt32' +values = '(1, 2, 3), (2, 3, 1), (78, 43, 45)' +redirecting_host = '127.0.0.1' +redirecting_to_http_port = 12345 +redirecting_to_https_port = 12346 +preserving_data_port = 12347 + +queries = [ + "select *, column1*column2*column3 from file('{}', 'CSV', '{}')".format(os.path.expanduser('~/test.csv'), format), + "select *, column1*column2*column3 from url('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), + "select *, column1*column2*column3 from s3('http://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), + "select *, column1*column2*column3 from s3('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), + "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), + "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), +] + +put_queries = [ + "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}" + .format(redirecting_host, preserving_data_port, format, values), +] + + +class RedirectingToHTTPHTTPServer(http.server.BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') + self.end_headers() + self.wfile.write(bytes(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + johnsmith.s3-gztb4pa9sq.amazonaws.com +''', "utf-8")) + + +class RedirectingToHTTPSHTTPServer(http.server.BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') + self.end_headers() + self.wfile.write(bytes(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + johnsmith.s3-gztb4pa9sq.amazonaws.com +''', "utf-8")) + + +received_data = [] + + +class PreservingDataServer(http.server.BaseHTTPRequestHandler): + def do_POST(self): + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.end_headers() + received_data.append(self.rfile.read()) + self.wfile.flush() + + +servers = [] +def redirecting_to_https_thread(): + server = http.server.HTTPServer((redirecting_host, redirecting_to_https_port), RedirectingToHTTPSHTTPServer) + servers.append(server) + server.handle_request() + +def redirecting_to_http_thread(): + server = http.server.HTTPServer((redirecting_host, redirecting_to_http_port), RedirectingToHTTPHTTPServer) + servers.append(server) + server.handle_request() + +def preserving_thread(): + server = http.server.HTTPServer((redirecting_host, preserving_data_port), PreservingDataServer) + servers.append(server) + server.handle_request() + + +jobs = [] +jobs.append(threading.Thread(target=redirecting_to_http_thread)) +jobs.append(threading.Thread(target=redirecting_to_https_thread)) +jobs.append(threading.Thread(target=preserving_thread)) +[ job.start() for job in jobs ] + +for query in queries: + print(query) + result = subprocess.run([ + os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), + '-c', + os.path.expanduser('~/config.xml'), + '-q', + query + ], stdout=subprocess.PIPE, universal_newlines=True) + result.check_returncode() + unittest.TestCase().assertEqual(list(map(str.split, result.stdout.splitlines())), [ + ['1', '2', '3', '6'], + ['3', '2', '1', '6'], + ['78', '43', '45', '150930'], + ]) + +for query in put_queries: + print(query) + result = subprocess.run([ + os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), + '-c', + os.path.expanduser('~/config.xml'), + '-q', + query + ], stdout=subprocess.PIPE, universal_newlines=True) + result.check_returncode() + unittest.TestCase().assertEqual(received_data[-1].decode(), '15\r\n1,2,3\n2,3,1\n78,43,45\n\r\n0\r\n\r\n') + +[ server.socket.close() for server in servers ] +[ job.join() for job in jobs ] diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml new file mode 100644 index 00000000000..de0dd0a7087 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml @@ -0,0 +1,115 @@ + + default + + trace + /home/excitoon/clickhouse-server.log + /home/excitoon/clickhouse-server.err.log + 1000M + 10 + + + + + + 8 + + + + + + + + + + + ::/0 + + + + default + + + default + + + + + + + a = 1 + + + + + a + b < 1 or c - d > 5 + + + + + c = 1 + + + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + From 1944ff1a48874123d4fc316415731663b6936e16 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sat, 1 Jun 2019 21:18:20 +0000 Subject: [PATCH 009/108] POST to PUT, refactoring. --- ...eBufferFromS3.cpp => ReadBufferFromS3.cpp} | 39 ++-------- dbms/src/IO/ReadBufferFromS3.h | 50 +++++++++++++ dbms/src/IO/WriteBufferFromS3.cpp | 30 ++++++++ ...riteBufferFromS3.h => WriteBufferFromS3.h} | 30 +------- dbms/src/Storages/StorageS3.cpp | 63 +++------------- dbms/src/Storages/StorageS3.h | 75 +++++++------------ 6 files changed, 124 insertions(+), 163 deletions(-) rename dbms/src/IO/{ReadWriteBufferFromS3.cpp => ReadBufferFromS3.cpp} (55%) create mode 100644 dbms/src/IO/ReadBufferFromS3.h create mode 100644 dbms/src/IO/WriteBufferFromS3.cpp rename dbms/src/IO/{ReadWriteBufferFromS3.h => WriteBufferFromS3.h} (57%) diff --git a/dbms/src/IO/ReadWriteBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp similarity index 55% rename from dbms/src/IO/ReadWriteBufferFromS3.cpp rename to dbms/src/IO/ReadBufferFromS3.cpp index f4b6124267f..aa056191988 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -1,4 +1,4 @@ -#include +#include #include @@ -6,15 +6,13 @@ namespace DB { -ReadWriteBufferFromS3::ReadWriteBufferFromS3(Poco::URI uri_, - const std::string & method_, - OutStreamCallback out_stream_callback, +ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_, const ConnectionTimeouts & timeouts, const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_) : ReadBuffer(nullptr, 0) , uri {uri_} - , method {!method_.empty() ? method_ : out_stream_callback ? Poco::Net::HTTPRequest::HTTP_POST : Poco::Net::HTTPRequest::HTTP_GET} + , method {Poco::Net::HTTPRequest::HTTP_GET} , session(makeHTTPSession(uri_, timeouts)) { Poco::Net::HTTPResponse response; @@ -29,18 +27,12 @@ ReadWriteBufferFromS3::ReadWriteBufferFromS3(Poco::URI uri_, request = std::make_unique(method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(uri.getHost()); // use original, not resolved host name in header - if (out_stream_callback) - request->setChunkedTransferEncoding(true); - if (!credentials.getUsername().empty()) credentials.authenticate(*request); LOG_TRACE((&Logger::get("ReadWriteBufferFromS3")), "Sending request to " << uri.toString()); - auto & stream_out = session->sendRequest(*request); - - if (out_stream_callback) - out_stream_callback(stream_out); + session->sendRequest(*request); istr = &session->receiveResponse(response); @@ -60,7 +52,7 @@ ReadWriteBufferFromS3::ReadWriteBufferFromS3(Poco::URI uri_, } -bool ReadWriteBufferFromS3::nextImpl() +bool ReadBufferFromS3::nextImpl() { if (!impl->next()) return false; @@ -69,25 +61,4 @@ bool ReadWriteBufferFromS3::nextImpl() return true; } -WriteBufferFromS3::WriteBufferFromS3( - const Poco::URI & uri, const std::string & method, const ConnectionTimeouts & timeouts, size_t buffer_size_) - : WriteBufferFromOStream(buffer_size_) - , session{makeHTTPSession(uri, timeouts)} - , request{method, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} -{ - request.setHost(uri.getHost()); - request.setChunkedTransferEncoding(true); - request.setExpectContinue(true); - - LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); - - ostr = &session->sendRequest(request); -} - -void WriteBufferFromS3::finalize() -{ - receiveResponse(*session, request, response); - /// TODO: Response body is ignored. -} - } diff --git a/dbms/src/IO/ReadBufferFromS3.h b/dbms/src/IO/ReadBufferFromS3.h new file mode 100644 index 00000000000..bada3f76252 --- /dev/null +++ b/dbms/src/IO/ReadBufferFromS3.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +#define DEFAULT_S3_READ_BUFFER_TIMEOUT 1800 +#define DEFAULT_S3_READ_BUFFER_CONNECTION_TIMEOUT 1 +#define DEFAULT_S3_MAX_FOLLOW_REDIRECT 2 + +namespace DB +{ +/** Perform S3 HTTP GET request and provide response to read. + */ +class ReadBufferFromS3 : public ReadBuffer +{ +protected: + Poco::URI uri; + std::string method; + + HTTPSessionPtr session; + std::istream * istr; /// owned by session + std::unique_ptr impl; + +public: + explicit ReadBufferFromS3(Poco::URI uri_, + const ConnectionTimeouts & timeouts = {}, + const Poco::Net::HTTPBasicCredentials & credentials = {}, + size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); + + bool nextImpl() override; +}; + +} diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp new file mode 100644 index 00000000000..94b07a6a8df --- /dev/null +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -0,0 +1,30 @@ +#include + +#include + + +namespace DB +{ + +WriteBufferFromS3::WriteBufferFromS3( + const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t buffer_size_) + : WriteBufferFromOStream(buffer_size_) + , session{makeHTTPSession(uri, timeouts)} + , request{Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} +{ + request.setHost(uri.getHost()); + request.setChunkedTransferEncoding(true); + request.setExpectContinue(true); + + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); + + ostr = &session->sendRequest(request); +} + +void WriteBufferFromS3::finalize() +{ + receiveResponse(*session, request, response); + /// TODO: Response body is ignored. +} + +} diff --git a/dbms/src/IO/ReadWriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h similarity index 57% rename from dbms/src/IO/ReadWriteBufferFromS3.h rename to dbms/src/IO/WriteBufferFromS3.h index e33dbe80df0..ee8b2d1dde9 100644 --- a/dbms/src/IO/ReadWriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -26,34 +26,7 @@ namespace DB { -/** Perform S3 HTTP POST request and provide response to read. - */ - -class ReadWriteBufferFromS3 : public ReadBuffer -{ -protected: - Poco::URI uri; - std::string method; - - HTTPSessionPtr session; - std::istream * istr; /// owned by session - std::unique_ptr impl; - -public: - using OutStreamCallback = std::function; - - explicit ReadWriteBufferFromS3(Poco::URI uri_, - const std::string & method_ = {}, - OutStreamCallback out_stream_callback = {}, - const ConnectionTimeouts & timeouts = {}, - const Poco::Net::HTTPBasicCredentials & credentials = {}, - size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); - - bool nextImpl() override; -}; - - -/* Perform S3 HTTP POST/PUT request. +/* Perform S3 HTTP PUT request. */ class WriteBufferFromS3 : public WriteBufferFromOStream { @@ -64,7 +37,6 @@ private: public: explicit WriteBufferFromS3(const Poco::URI & uri, - const std::string & method = Poco::Net::HTTPRequest::HTTP_POST, // POST for inserting, PUT for replacing. const ConnectionTimeouts & timeouts = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 972820a8449..f15e19d9396 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -5,7 +5,8 @@ #include #include -#include +#include +#include #include @@ -23,23 +24,12 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } -IStorageS3Base::IStorageS3Base(const Poco::URI & uri_, - const Context & context_, - const std::string & table_name_, - const String & format_name_, - const ColumnsDescription & columns_) - : IStorage(columns_), uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_) -{ -} - namespace { class StorageS3BlockInputStream : public IBlockInputStream { public: StorageS3BlockInputStream(const Poco::URI & uri, - const std::string & method, - std::function callback, const String & format, const String & name_, const Block & sample_block, @@ -48,7 +38,7 @@ namespace const ConnectionTimeouts & timeouts) : name(name_) { - read_buf = std::make_unique(uri, method, callback, timeouts); + read_buf = std::make_unique(uri, timeouts); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); } @@ -80,7 +70,7 @@ namespace private: String name; - std::unique_ptr read_buf; + std::unique_ptr read_buf; BlockInputStreamPtr reader; }; @@ -94,7 +84,7 @@ namespace const ConnectionTimeouts & timeouts) : sample_block(sample_block_) { - write_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); + write_buf = std::make_unique(uri, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } @@ -128,45 +118,14 @@ namespace } -std::string IStorageS3Base::getReadMethod() const -{ - return Poco::Net::HTTPRequest::HTTP_GET; -} - -std::vector> IStorageS3Base::getReadURIParams(const Names & /*column_names*/, +BlockInputStreams StorageS3::read(const Names & column_names, const SelectQueryInfo & /*query_info*/, - const Context & /*context*/, - QueryProcessingStage::Enum & /*processed_stage*/, - size_t /*max_block_size*/) const -{ - return {}; -} - -std::function IStorageS3Base::getReadPOSTDataCallback(const Names & /*column_names*/, - const SelectQueryInfo & /*query_info*/, - const Context & /*context*/, - QueryProcessingStage::Enum & /*processed_stage*/, - size_t /*max_block_size*/) const -{ - return nullptr; -} - - -BlockInputStreams IStorageS3Base::read(const Names & column_names, - const SelectQueryInfo & query_info, const Context & context, - QueryProcessingStage::Enum processed_stage, + QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, unsigned /*num_streams*/) { - auto request_uri = uri; - auto params = getReadURIParams(column_names, query_info, context, processed_stage, max_block_size); - for (const auto & [param, value] : params) - request_uri.addQueryParameter(param, value); - - BlockInputStreamPtr block_input = std::make_shared(request_uri, - getReadMethod(), - getReadPOSTDataCallback(column_names, query_info, context, processed_stage, max_block_size), + BlockInputStreamPtr block_input = std::make_shared(uri, format_name, getName(), getHeaderBlock(column_names), @@ -181,9 +140,9 @@ BlockInputStreams IStorageS3Base::read(const Names & column_names, return {std::make_shared(block_input, column_defaults, context)}; } -void IStorageS3Base::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {} +void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {} -BlockOutputStreamPtr IStorageS3Base::write(const ASTPtr & /*query*/, const Context & /*context*/) +BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) { return std::make_shared( uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); @@ -191,6 +150,6 @@ BlockOutputStreamPtr IStorageS3Base::write(const ASTPtr & /*query*/, const Conte void registerStorageS3(StorageFactory & /*factory*/) { - // TODO. See #1394. + // TODO. See #1394? } } diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 2615563b57c..a38cd717e36 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -8,14 +8,37 @@ namespace DB { /** - * This class represents table engine for external urls. + * This class represents table engine for external S3 urls. * It sends HTTP GET to server when select is called and - * HTTP POST when insert is called. In POST request the data is send - * using Chunked transfer encoding, so server have to support it. + * HTTP PUT when insert is called. */ -class IStorageS3Base : public IStorage +class StorageS3 : public ext::shared_ptr_helper, public IStorage { public: + StorageS3(const Poco::URI & uri_, + const std::string & table_name_, + const String & format_name_, + const ColumnsDescription & columns_, + Context & context_ + ) + : IStorage(columns_) + , uri(uri_) + , context_global(context_) + , format_name(format_name_) + , table_name(table_name_) + { + } + + String getName() const override + { + return "S3"; + } + + Block getHeaderBlock(const Names & /*column_names*/) const + { + return getSampleBlock(); + } + String getTableName() const override { return table_name; @@ -33,56 +56,12 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; protected: - IStorageS3Base(const Poco::URI & uri_, - const Context & context_, - const std::string & table_name_, - const String & format_name_, - const ColumnsDescription & columns_); - Poco::URI uri; const Context & context_global; private: String format_name; String table_name; - - virtual std::string getReadMethod() const; - - virtual std::vector> getReadURIParams(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size) const; - - virtual std::function getReadPOSTDataCallback(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size) const; - - virtual Block getHeaderBlock(const Names & column_names) const = 0; }; -class StorageS3 : public ext::shared_ptr_helper, public IStorageS3Base -{ -public: - StorageS3(const Poco::URI & uri_, - const std::string & table_name_, - const String & format_name_, - const ColumnsDescription & columns_, - Context & context_) - : IStorageS3Base(uri_, context_, table_name_, format_name_, columns_) - { - } - - String getName() const override - { - return "S3"; - } - - Block getHeaderBlock(const Names & /*column_names*/) const override - { - return getSampleBlock(); - } -}; } From efade38d7b6d1c42f847c0c7613f401b36a28521 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sat, 1 Jun 2019 21:21:33 +0000 Subject: [PATCH 010/108] Fixed test, so it does not pass now. --- .../0_stateless/00950_table_function_s3_wip/clickhouse-test | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index 09876ea0df6..8140a140474 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -63,10 +63,11 @@ received_data = [] class PreservingDataServer(http.server.BaseHTTPRequestHandler): - def do_POST(self): + def do_PUT(self): self.send_response(200) self.send_header('Content-type', 'text/plain') self.end_headers() + assert self.headers.get('Content-Length') received_data.append(self.rfile.read()) self.wfile.flush() From 19642cf822efdb6f615ec7ae343d00194a11f3c4 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 00:51:44 +0000 Subject: [PATCH 011/108] Tests update. --- .../0_stateless/00950_table_function_s3_wip/clickhouse-test | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index 8140a140474..b0132d5d1cd 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -3,6 +3,7 @@ import http.server import os import subprocess +import sys import threading import unittest @@ -67,6 +68,7 @@ class PreservingDataServer(http.server.BaseHTTPRequestHandler): self.send_response(200) self.send_header('Content-type', 'text/plain') self.end_headers() + print('Content-Length =', self.headers.get('Content-Length'), file=sys.stderr) assert self.headers.get('Content-Length') received_data.append(self.rfile.read()) self.wfile.flush() From 0504eb58becefcde1ce65b319268581dc395b3d6 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 00:52:47 +0000 Subject: [PATCH 012/108] Correct S3 PUT requests in WriteBufferFromS3. --- dbms/src/IO/WriteBufferFromS3.cpp | 6 +++++- dbms/src/IO/WriteBufferFromS3.h | 2 ++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 94b07a6a8df..52b28821f7f 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -18,11 +18,15 @@ WriteBufferFromS3::WriteBufferFromS3( LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); - ostr = &session->sendRequest(request); + ostr = &temporary_stream; } void WriteBufferFromS3::finalize() { + const std::string & data = temporary_stream.str(); + request.setContentLength(data.size()); + ostr = &session->sendRequest(request); + *ostr << data; receiveResponse(*session, request, response); /// TODO: Response body is ignored. } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index ee8b2d1dde9..e43ecffb2f1 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -34,6 +35,7 @@ private: HTTPSessionPtr session; Poco::Net::HTTPRequest request; Poco::Net::HTTPResponse response; + std::ostringstream temporary_stream; /// Maybe one shall use some DB:: buffer. public: explicit WriteBufferFromS3(const Poco::URI & uri, From 7236ae0d8dcf60297255857759597ffc83c0fd02 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 01:22:43 +0000 Subject: [PATCH 013/108] Fixed 411 error when putting to S3. --- dbms/src/IO/WriteBufferFromS3.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 52b28821f7f..854bffca979 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -13,7 +13,11 @@ WriteBufferFromS3::WriteBufferFromS3( , request{Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} { request.setHost(uri.getHost()); - request.setChunkedTransferEncoding(true); + + // request.setChunkedTransferEncoding(true); + // Chunked transfers require additional logic, see: + // https://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-streaming.html + request.setExpectContinue(true); LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); From ae40d68eb0e425b4e43bf6be1a51c31987aabb24 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 01:22:54 +0000 Subject: [PATCH 014/108] Updated tests. --- .../clickhouse-test | 30 +++++++++++++++++-- 1 file changed, 28 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index b0132d5d1cd..c896b9fe114 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -8,17 +8,30 @@ import threading import unittest +# 1) Run Go FakeS3 server. +# go run cmd/gofakes3/main.go -backend memory -host :9990 +# 2) Create a bucket. +# curl -X PUT http://localhost:9990/abc/ + format = 'column1 UInt32, column2 UInt32, column3 UInt32' -values = '(1, 2, 3), (2, 3, 1), (78, 43, 45)' +values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' redirecting_host = '127.0.0.1' redirecting_to_http_port = 12345 redirecting_to_https_port = 12346 preserving_data_port = 12347 +fakes3_port = 9990 +localhost = '127.0.0.1' +bucket = 'abc' + +prepare_put_queries = [ + "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(localhost, fakes3_port, bucket, format, values), +] queries = [ "select *, column1*column2*column3 from file('{}', 'CSV', '{}')".format(os.path.expanduser('~/test.csv'), format), "select *, column1*column2*column3 from url('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), "select *, column1*column2*column3 from s3('http://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), + "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, fakes3_port, bucket, format), "select *, column1*column2*column3 from s3('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), @@ -97,6 +110,17 @@ jobs.append(threading.Thread(target=redirecting_to_https_thread)) jobs.append(threading.Thread(target=preserving_thread)) [ job.start() for job in jobs ] +for query in prepare_put_queries: + print(query) + result = subprocess.run([ + os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), + '-c', + os.path.expanduser('~/config.xml'), + '-q', + query + ], stdout=subprocess.PIPE, universal_newlines=True) + result.check_returncode() + for query in queries: print(query) result = subprocess.run([ @@ -123,7 +147,9 @@ for query in put_queries: query ], stdout=subprocess.PIPE, universal_newlines=True) result.check_returncode() - unittest.TestCase().assertEqual(received_data[-1].decode(), '15\r\n1,2,3\n2,3,1\n78,43,45\n\r\n0\r\n\r\n') + unittest.TestCase().assertEqual(received_data[-1].decode(), '1,2,3\n3,2,1\n78,43,45\n') + # In chunked encoding: + # unittest.TestCase().assertEqual(received_data[-1].decode(), '15\r\n1,2,3\n2,3,1\n78,43,45\n\r\n0\r\n\r\n') [ server.socket.close() for server in servers ] [ job.join() for job in jobs ] From cca3a9acc7b69746d7f3d966f0809c87f454b926 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 01:57:37 +0000 Subject: [PATCH 015/108] Added test for PUT redirect. --- .../clickhouse-test | 67 +++++++++++++++++++ 1 file changed, 67 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index c896b9fe114..554effe027b 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -15,10 +15,12 @@ import unittest format = 'column1 UInt32, column2 UInt32, column3 UInt32' values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' +other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' redirecting_host = '127.0.0.1' redirecting_to_http_port = 12345 redirecting_to_https_port = 12346 preserving_data_port = 12347 +redirecting_preserving_data_port = 12348 fakes3_port = 9990 localhost = '127.0.0.1' bucket = 'abc' @@ -40,6 +42,12 @@ queries = [ put_queries = [ "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}" .format(redirecting_host, preserving_data_port, format, values), + "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}" + .format(redirecting_host, redirecting_preserving_data_port, format, other_values), +] + +check_queries = [ + "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, fakes3_port, bucket, format), ] @@ -77,12 +85,49 @@ received_data = [] class PreservingDataServer(http.server.BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def handle_expect_100(self): + print('Received Expect-100', file=sys.stderr) + return True + def do_PUT(self): self.send_response(200) self.send_header('Content-type', 'text/plain') self.end_headers() print('Content-Length =', self.headers.get('Content-Length'), file=sys.stderr) assert self.headers.get('Content-Length') + assert self.headers['Expect'] == '100-continue' + received_data.append(self.rfile.read()) + self.wfile.flush() + + +class RedirectingPreservingDataServer(http.server.BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def handle_expect_100(self): + print('Received Expect-100', file=sys.stderr) + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://{}:{}/{}/test.csv'.format(localhost, fakes3_port, bucket)) + self.end_headers() + self.wfile.write(bytes(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + johnsmith.s3-gztb4pa9sq.amazonaws.com +''', "utf-8")) + return False + + def do_PUT(self): + assert False + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.end_headers() + print('Content-Length =', self.headers.get('Content-Length'), file=sys.stderr) + assert self.headers.get('Content-Length') + assert self.headers['Expect'] == '100-continue' received_data.append(self.rfile.read()) self.wfile.flush() @@ -103,11 +148,17 @@ def preserving_thread(): servers.append(server) server.handle_request() +def redirecting_preserving_thread(): + server = http.server.HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataServer) + servers.append(server) + server.handle_request() + jobs = [] jobs.append(threading.Thread(target=redirecting_to_http_thread)) jobs.append(threading.Thread(target=redirecting_to_https_thread)) jobs.append(threading.Thread(target=preserving_thread)) +jobs.append(threading.Thread(target=redirecting_preserving_thread)) [ job.start() for job in jobs ] for query in prepare_put_queries: @@ -151,5 +202,21 @@ for query in put_queries: # In chunked encoding: # unittest.TestCase().assertEqual(received_data[-1].decode(), '15\r\n1,2,3\n2,3,1\n78,43,45\n\r\n0\r\n\r\n') +for query in check_queries: + print(query) + result = subprocess.run([ + os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), + '-c', + os.path.expanduser('~/config.xml'), + '-q', + query + ], stdout=subprocess.PIPE, universal_newlines=True) + result.check_returncode() + unittest.TestCase().assertEqual(list(map(str.split, result.stdout.splitlines())), [ + ['1', '1', '1', '1'], + ['1', '1', '1', '1'], + ['11', '11', '11', '1331'], + ]) + [ server.socket.close() for server in servers ] [ job.join() for job in jobs ] From ff691129190233e50523c935634bb277bd5653e0 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 02:35:33 +0000 Subject: [PATCH 016/108] Attempt to make S3 PUT redirects (wip). --- dbms/src/IO/ReadBufferFromS3.cpp | 6 ++- dbms/src/IO/ReadBufferFromS3.h | 4 -- dbms/src/IO/WriteBufferFromS3.cpp | 66 ++++++++++++++++++++++--------- dbms/src/IO/WriteBufferFromS3.h | 10 ++--- 4 files changed, 57 insertions(+), 29 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index aa056191988..f6061c3a8c7 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -3,6 +3,8 @@ #include +#define DEFAULT_S3_MAX_FOLLOW_GET_REDIRECT 2 + namespace DB { @@ -13,12 +15,12 @@ ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_, : ReadBuffer(nullptr, 0) , uri {uri_} , method {Poco::Net::HTTPRequest::HTTP_GET} - , session(makeHTTPSession(uri_, timeouts)) + , session {makeHTTPSession(uri_, timeouts)} { Poco::Net::HTTPResponse response; std::unique_ptr request; - for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_REDIRECT; ++i) + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_GET_REDIRECT; ++i) { // With empty path poco will send "POST HTTP/1.1" its bug. if (uri.getPath().empty()) diff --git a/dbms/src/IO/ReadBufferFromS3.h b/dbms/src/IO/ReadBufferFromS3.h index bada3f76252..ec53a24c5a6 100644 --- a/dbms/src/IO/ReadBufferFromS3.h +++ b/dbms/src/IO/ReadBufferFromS3.h @@ -20,10 +20,6 @@ #include -#define DEFAULT_S3_READ_BUFFER_TIMEOUT 1800 -#define DEFAULT_S3_READ_BUFFER_CONNECTION_TIMEOUT 1 -#define DEFAULT_S3_MAX_FOLLOW_REDIRECT 2 - namespace DB { /** Perform S3 HTTP GET request and provide response to read. diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 854bffca979..01a1c03bc05 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -3,36 +3,66 @@ #include +#define DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT 2 + namespace DB { WriteBufferFromS3::WriteBufferFromS3( - const Poco::URI & uri, const ConnectionTimeouts & timeouts, size_t buffer_size_) + const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, + const Poco::Net::HTTPBasicCredentials & credentials_, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) - , session{makeHTTPSession(uri, timeouts)} - , request{Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} + , uri {uri_} + , timeouts {timeouts_} + , credentials {credentials_} + , session {makeHTTPSession(uri_, timeouts_)} { - request.setHost(uri.getHost()); - - // request.setChunkedTransferEncoding(true); - // Chunked transfers require additional logic, see: - // https://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-streaming.html - - request.setExpectContinue(true); - - LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); - ostr = &temporary_stream; } void WriteBufferFromS3::finalize() { const std::string & data = temporary_stream.str(); - request.setContentLength(data.size()); - ostr = &session->sendRequest(request); - *ostr << data; - receiveResponse(*session, request, response); - /// TODO: Response body is ignored. + + std::unique_ptr request; + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) + { + request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request->setHost(uri.getHost()); // use original, not resolved host name in header + + if (!credentials.getUsername().empty()) + credentials.authenticate(*request); + + // request.setChunkedTransferEncoding(true); + // Chunked transfers require additional logic, see: + // https://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-streaming.html + + request->setExpectContinue(true); + + request->setContentLength(data.size()); + + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); + + ostr = &session->sendRequest(*request); + if (session->peekResponse(response)) + { + // Received 100-continue. + *ostr << data; + } + + istr = &session->receiveResponse(response); + + if (response.getStatus() != 307) + break; + + auto location_iterator = response.find("Location"); + if (location_iterator == response.end()) + break; + + uri = location_iterator->second; + session = makeHTTPSession(uri, timeouts); + } + assertResponseIsOk(*request, response, istr); } } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index e43ecffb2f1..3c2c343b291 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -21,10 +21,6 @@ #include -#define DEFAULT_S3_READ_BUFFER_TIMEOUT 1800 -#define DEFAULT_S3_READ_BUFFER_CONNECTION_TIMEOUT 1 -#define DEFAULT_S3_MAX_FOLLOW_REDIRECT 2 - namespace DB { /* Perform S3 HTTP PUT request. @@ -32,14 +28,18 @@ namespace DB class WriteBufferFromS3 : public WriteBufferFromOStream { private: + Poco::URI uri; + ConnectionTimeouts timeouts; + const Poco::Net::HTTPBasicCredentials & credentials; HTTPSessionPtr session; - Poco::Net::HTTPRequest request; + std::istream * istr; /// owned by session Poco::Net::HTTPResponse response; std::ostringstream temporary_stream; /// Maybe one shall use some DB:: buffer. public: explicit WriteBufferFromS3(const Poco::URI & uri, const ConnectionTimeouts & timeouts = {}, + const Poco::Net::HTTPBasicCredentials & credentials = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); /// Receives response from the server after sending all data. From 634f82d0ad8ac9ba631ff15771598ae41075235e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 10 Jun 2019 23:33:43 +0000 Subject: [PATCH 017/108] Minor fix. --- dbms/src/IO/WriteBufferFromS3.cpp | 8 +++++--- dbms/src/IO/WriteBufferFromS3.h | 3 --- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 01a1c03bc05..c0d891ebf6d 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -15,7 +15,6 @@ WriteBufferFromS3::WriteBufferFromS3( , uri {uri_} , timeouts {timeouts_} , credentials {credentials_} - , session {makeHTTPSession(uri_, timeouts_)} { ostr = &temporary_stream; } @@ -24,9 +23,13 @@ void WriteBufferFromS3::finalize() { const std::string & data = temporary_stream.str(); + Poco::Net::HTTPResponse response; std::unique_ptr request; + HTTPSessionPtr session; + std::istream * istr; /// owned by session for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { + session = makeHTTPSession(uri, timeouts); request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(uri.getHost()); // use original, not resolved host name in header @@ -44,7 +47,7 @@ void WriteBufferFromS3::finalize() LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); ostr = &session->sendRequest(*request); - if (session->peekResponse(response)) +// if (session->peekResponse(response)) { // Received 100-continue. *ostr << data; @@ -60,7 +63,6 @@ void WriteBufferFromS3::finalize() break; uri = location_iterator->second; - session = makeHTTPSession(uri, timeouts); } assertResponseIsOk(*request, response, istr); } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 3c2c343b291..13815f4acb9 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -31,9 +31,6 @@ private: Poco::URI uri; ConnectionTimeouts timeouts; const Poco::Net::HTTPBasicCredentials & credentials; - HTTPSessionPtr session; - std::istream * istr; /// owned by session - Poco::Net::HTTPResponse response; std::ostringstream temporary_stream; /// Maybe one shall use some DB:: buffer. public: From 63164db01964eb5232ce218ed63d20135993747f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 11 Jun 2019 00:07:20 +0000 Subject: [PATCH 018/108] Fixed S3 PUT redirects. --- dbms/src/IO/WriteBufferFromS3.cpp | 11 ++++++++--- dbms/src/IO/WriteBufferFromS3.h | 2 +- 2 files changed, 9 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index c0d891ebf6d..7f84edf11b9 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -10,13 +10,15 @@ namespace DB WriteBufferFromS3::WriteBufferFromS3( const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, - const Poco::Net::HTTPBasicCredentials & credentials_, size_t buffer_size_) + const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_) : WriteBufferFromOStream(buffer_size_) , uri {uri_} , timeouts {timeouts_} - , credentials {credentials_} + , auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} { ostr = &temporary_stream; + if (!credentials.getUsername().empty()) + credentials.authenticate(auth_request); } void WriteBufferFromS3::finalize() @@ -26,6 +28,7 @@ void WriteBufferFromS3::finalize() Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; + std::istream * istr; /// owned by session for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { @@ -33,8 +36,10 @@ void WriteBufferFromS3::finalize() request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(uri.getHost()); // use original, not resolved host name in header - if (!credentials.getUsername().empty()) + if (auth_request.hasCredentials()) { + Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); + } // request.setChunkedTransferEncoding(true); // Chunked transfers require additional logic, see: diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 13815f4acb9..3f95529d74a 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -30,7 +30,7 @@ class WriteBufferFromS3 : public WriteBufferFromOStream private: Poco::URI uri; ConnectionTimeouts timeouts; - const Poco::Net::HTTPBasicCredentials & credentials; + Poco::Net::HTTPRequest auth_request; std::ostringstream temporary_stream; /// Maybe one shall use some DB:: buffer. public: From d3db5a38902d11bdec6d7335f4bec1804c5868c9 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 14 Jun 2019 15:56:25 +0300 Subject: [PATCH 019/108] Update TableFunctionS3.h --- dbms/src/TableFunctions/TableFunctionS3.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/TableFunctions/TableFunctionS3.h b/dbms/src/TableFunctions/TableFunctionS3.h index 83c49e0b8d1..04826a01d9b 100644 --- a/dbms/src/TableFunctions/TableFunctionS3.h +++ b/dbms/src/TableFunctions/TableFunctionS3.h @@ -7,7 +7,7 @@ namespace DB { -/* url(source, format, structure) - creates a temporary storage from url +/* s3(source, format, structure) - creates a temporary storage for a file in S3 */ class TableFunctionS3 : public ITableFunctionFileLike { From 6b9397e805f056dfb64b54e188268d82d7bf045b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 01:10:49 +0300 Subject: [PATCH 020/108] Tests fixes. --- .../clickhouse-test | 18 +++++++++--------- .../00950_table_function_s3_wip/config.xml | 4 ++-- .../00950_table_function_s3_wip/test.csv | 3 +++ 3 files changed, 14 insertions(+), 11 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index 554effe027b..39c62f835a5 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -8,11 +8,11 @@ import threading import unittest -# 1) Run Go FakeS3 server. -# go run cmd/gofakes3/main.go -backend memory -host :9990 -# 2) Create a bucket. -# curl -X PUT http://localhost:9990/abc/ +# Run Go FakeS3 server. +# go run cmd/gofakes3/main.go -backend memory -host :9990 -initialbucket abc +config = os.path.join(os.path.dirname(sys.argv[0]), 'config.xml') +test_csv = os.path.join(os.path.dirname(sys.argv[0]), 'test.csv') format = 'column1 UInt32, column2 UInt32, column3 UInt32' values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' @@ -30,7 +30,7 @@ prepare_put_queries = [ ] queries = [ - "select *, column1*column2*column3 from file('{}', 'CSV', '{}')".format(os.path.expanduser('~/test.csv'), format), + "select *, column1*column2*column3 from file('{}', 'CSV', '{}')".format(test_csv, format), "select *, column1*column2*column3 from url('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), "select *, column1*column2*column3 from s3('http://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, fakes3_port, bucket, format), @@ -166,7 +166,7 @@ for query in prepare_put_queries: result = subprocess.run([ os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), '-c', - os.path.expanduser('~/config.xml'), + config, '-q', query ], stdout=subprocess.PIPE, universal_newlines=True) @@ -177,7 +177,7 @@ for query in queries: result = subprocess.run([ os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), '-c', - os.path.expanduser('~/config.xml'), + config, '-q', query ], stdout=subprocess.PIPE, universal_newlines=True) @@ -193,7 +193,7 @@ for query in put_queries: result = subprocess.run([ os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), '-c', - os.path.expanduser('~/config.xml'), + config, '-q', query ], stdout=subprocess.PIPE, universal_newlines=True) @@ -207,7 +207,7 @@ for query in check_queries: result = subprocess.run([ os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), '-c', - os.path.expanduser('~/config.xml'), + config, '-q', query ], stdout=subprocess.PIPE, universal_newlines=True) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml index de0dd0a7087..7675c696456 100644 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml @@ -2,8 +2,8 @@ default trace - /home/excitoon/clickhouse-server.log - /home/excitoon/clickhouse-server.err.log + ~/clickhouse-server.log + ~/clickhouse-server.err.log 1000M 10 diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv new file mode 100644 index 00000000000..a2325127dec --- /dev/null +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv @@ -0,0 +1,3 @@ +1,2,3 +3,2,1 +78,43,45 From dd32c92f2a97b2f778e6a7a32610f3e23956552f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 03:06:14 +0300 Subject: [PATCH 021/108] Minor fixes. --- dbms/src/IO/WriteBufferFromS3.cpp | 37 ++++++++++++++++++++++++++----- dbms/src/IO/WriteBufferFromS3.h | 13 +++++++---- dbms/src/Storages/StorageS3.cpp | 23 ++++++++++++++++--- 3 files changed, 60 insertions(+), 13 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 7f84edf11b9..49862e8c8aa 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -11,19 +11,31 @@ namespace DB WriteBufferFromS3::WriteBufferFromS3( const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_) - : WriteBufferFromOStream(buffer_size_) + : BufferWithOwnMemory(buffer_size_, nullptr, 0) , uri {uri_} , timeouts {timeouts_} , auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} + , temporary_buffer {buffer_string} { - ostr = &temporary_stream; if (!credentials.getUsername().empty()) credentials.authenticate(auth_request); } + +void WriteBufferFromS3::nextImpl() +{ + if (!offset()) + return; + + temporary_buffer.write(working_buffer.begin(), offset()); +} + + void WriteBufferFromS3::finalize() { - const std::string & data = temporary_stream.str(); + temporary_buffer.finish(); + + const String & data = buffer_string; Poco::Net::HTTPResponse response; std::unique_ptr request; @@ -51,11 +63,11 @@ void WriteBufferFromS3::finalize() LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); - ostr = &session->sendRequest(*request); -// if (session->peekResponse(response)) + std::ostream & ostr = session->sendRequest(*request); +// if (session->peekResponse(response)) // FIXME, shall not go next if not received 100-continue { // Received 100-continue. - *ostr << data; + ostr << data; } istr = &session->receiveResponse(response); @@ -72,4 +84,17 @@ void WriteBufferFromS3::finalize() assertResponseIsOk(*request, response, istr); } + +WriteBufferFromS3::~WriteBufferFromS3() +{ + try + { + next(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 3f95529d74a..58ffcfdd4b9 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -2,14 +2,14 @@ #include #include -#include #include #include #include +#include #include #include #include -#include +#include #include #include #include @@ -25,13 +25,14 @@ namespace DB { /* Perform S3 HTTP PUT request. */ -class WriteBufferFromS3 : public WriteBufferFromOStream +class WriteBufferFromS3 : public BufferWithOwnMemory { private: Poco::URI uri; ConnectionTimeouts timeouts; Poco::Net::HTTPRequest auth_request; - std::ostringstream temporary_stream; /// Maybe one shall use some DB:: buffer. + String buffer_string; + DB::WriteBufferFromString temporary_buffer; public: explicit WriteBufferFromS3(const Poco::URI & uri, @@ -39,8 +40,12 @@ public: const Poco::Net::HTTPBasicCredentials & credentials = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); + void nextImpl() override; + /// Receives response from the server after sending all data. void finalize(); + + ~WriteBufferFromS3(); }; } diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index f15e19d9396..474d603878e 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -133,7 +133,6 @@ BlockInputStreams StorageS3::read(const Names & column_names, max_block_size, ConnectionTimeouts::getHTTPTimeouts(context)); - auto column_defaults = getColumns().getDefaults(); if (column_defaults.empty()) return {block_input}; @@ -148,8 +147,26 @@ BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); } -void registerStorageS3(StorageFactory & /*factory*/) +void registerStorageS3(StorageFactory & factory) { - // TODO. See #1394? + factory.registerStorage("S3", [](const StorageFactory::Arguments & args) + { + ASTs & engine_args = args.engine_args; + + if (!(engine_args.size() == 1 || engine_args.size() == 2)) + throw Exception( + "Storage S3 requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); + + String url = engine_args[0]->as().value.safeGet(); + Poco::URI uri(url); + + engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.local_context); + + String format_name = engine_args[1]->as().value.safeGet(); + + return StorageS3::create(uri, args.table_name, format_name, args.columns, args.context); + }); } } From 10a7f80af14562002738eb42d4b45031226818dc Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 03:42:47 +0300 Subject: [PATCH 022/108] Stubs for multipart uploads. --- dbms/src/IO/WriteBufferFromS3.cpp | 66 ++++++++++++++++++++++++------- dbms/src/IO/WriteBufferFromS3.h | 11 +++++- 2 files changed, 62 insertions(+), 15 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 49862e8c8aa..0d48d239e83 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -4,6 +4,7 @@ #define DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT 2 +#define DEFAULT_S3_MINIMUM_PART_SIZE 100'000'000 namespace DB { @@ -15,10 +16,14 @@ WriteBufferFromS3::WriteBufferFromS3( , uri {uri_} , timeouts {timeouts_} , auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} - , temporary_buffer {buffer_string} + , temporary_buffer {std::make_unique(buffer_string)} + , part_number {0} + , last_part_size {0} { if (!credentials.getUsername().empty()) credentials.authenticate(auth_request); + + initiate(); } @@ -27,16 +32,55 @@ void WriteBufferFromS3::nextImpl() if (!offset()) return; - temporary_buffer.write(working_buffer.begin(), offset()); + temporary_buffer->write(working_buffer.begin(), offset()); + + last_part_size += offset(); + + if (last_part_size > DEFAULT_S3_MINIMUM_PART_SIZE) + { + temporary_buffer->finish(); + writePart(buffer_string); + last_part_size = 0; + temporary_buffer = std::make_unique(buffer_string); + } } void WriteBufferFromS3::finalize() { - temporary_buffer.finish(); + temporary_buffer->finish(); + if (!buffer_string.empty()) + { + writePart(buffer_string); + } - const String & data = buffer_string; + complete(); +} + +WriteBufferFromS3::~WriteBufferFromS3() +{ + try + { + next(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } +} + + +void WriteBufferFromS3::initiate() +{ + // FIXME POST ?uploads + // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadInitiate.html +} + +void WriteBufferFromS3::writePart(const String & data) +{ + // FIXME PUT ?partNumber=PartNumber&uploadId=UploadId + // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; @@ -44,7 +88,7 @@ void WriteBufferFromS3::finalize() std::istream * istr; /// owned by session for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { - session = makeHTTPSession(uri, timeouts); + session = makeHTTPSession(uri, timeouts); // FIXME apply part number to URI request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(uri.getHost()); // use original, not resolved host name in header @@ -85,16 +129,10 @@ void WriteBufferFromS3::finalize() } -WriteBufferFromS3::~WriteBufferFromS3() +void WriteBufferFromS3::complete() { - try - { - next(); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - } + // FIXME POST ?uploads + // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadComplete.html } } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 58ffcfdd4b9..513ab167be5 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -32,7 +33,10 @@ private: ConnectionTimeouts timeouts; Poco::Net::HTTPRequest auth_request; String buffer_string; - DB::WriteBufferFromString temporary_buffer; + std::unique_ptr temporary_buffer; + size_t part_number; + size_t last_part_size; + std::vector part_tags; public: explicit WriteBufferFromS3(const Poco::URI & uri, @@ -46,6 +50,11 @@ public: void finalize(); ~WriteBufferFromS3(); + +private: + void initiate(); + void writePart(const String & data); + void complete(); }; } From 248e26d59f383b98adb35aa736e53c0bfd20fddf Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 10:16:43 +0300 Subject: [PATCH 023/108] Attempt to make multipart uploads. --- dbms/src/IO/WriteBufferFromS3.cpp | 163 ++++++++++++++++++++++++++---- dbms/src/IO/WriteBufferFromS3.h | 1 + 2 files changed, 145 insertions(+), 19 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 0d48d239e83..39f840f14c1 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -1,5 +1,13 @@ #include +#include + +#include +#include +#include +#include +#include + #include @@ -9,6 +17,12 @@ namespace DB { +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; +} + + WriteBufferFromS3::WriteBufferFromS3( const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_) @@ -17,7 +31,7 @@ WriteBufferFromS3::WriteBufferFromS3( , timeouts {timeouts_} , auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} , temporary_buffer {std::make_unique(buffer_string)} - , part_number {0} + , part_number {1} , last_part_size {0} { if (!credentials.getUsername().empty()) @@ -41,7 +55,7 @@ void WriteBufferFromS3::nextImpl() temporary_buffer->finish(); writePart(buffer_string); last_part_size = 0; - temporary_buffer = std::make_unique(buffer_string); + temporary_buffer = std::make_unique(buffer_string); } } @@ -52,6 +66,7 @@ void WriteBufferFromS3::finalize() if (!buffer_string.empty()) { writePart(buffer_string); + ++part_number; } complete(); @@ -73,39 +88,87 @@ WriteBufferFromS3::~WriteBufferFromS3() void WriteBufferFromS3::initiate() { - // FIXME POST ?uploads // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadInitiate.html -} - -void WriteBufferFromS3::writePart(const String & data) -{ - // FIXME PUT ?partNumber=PartNumber&uploadId=UploadId - // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; - std::istream * istr; /// owned by session + Poco::URI initiate_uri = uri; + initiate_uri.setRawQuery("uploads"); // FIXME find how to leave user params as is + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { - session = makeHTTPSession(uri, timeouts); // FIXME apply part number to URI - request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - request->setHost(uri.getHost()); // use original, not resolved host name in header + session = makeHTTPSession(initiate_uri, timeouts); + request = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, initiate_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request->setHost(initiate_uri.getHost()); // use original, not resolved host name in header if (auth_request.hasCredentials()) { Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); } - // request.setChunkedTransferEncoding(true); - // Chunked transfers require additional logic, see: - // https://docs.aws.amazon.com/AmazonS3/latest/API/sigv4-streaming.html + request->setContentLength(0); + + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << initiate_uri.toString()); + + session->sendRequest(*request); + + istr = &session->receiveResponse(response); + + if (response.getStatus() != 307) + break; + + auto location_iterator = response.find("Location"); + if (location_iterator == response.end()) + break; + + initiate_uri = location_iterator->second; + } + assertResponseIsOk(*request, response, istr); + + Poco::XML::InputSource src(*istr); + Poco::XML::DOMParser parser; + Poco::AutoPtr document = parser.parse(&src); + Poco::AutoPtr nodes = document->getElementsByTagName("UploadId"); + if (nodes->length() != 1) + { + throw Exception("Incorrect XML in response, no upload id", ErrorCodes::INCORRECT_DATA); + } + upload_id = nodes->item(0)->innerText(); + if (upload_id.empty()) + { + throw Exception("Incorrect XML in response, empty upload id", ErrorCodes::INCORRECT_DATA); + } +} + + +void WriteBufferFromS3::writePart(const String & data) +{ + // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html + Poco::Net::HTTPResponse response; + std::unique_ptr request; + HTTPSessionPtr session; + std::istream * istr; /// owned by session + Poco::URI part_uri = uri; + part_uri.addQueryParameter("partNumber", std::to_string(part_number)); + part_uri.addQueryParameter("uploadId", upload_id); + + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) + { + session = makeHTTPSession(part_uri, timeouts); + request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, part_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request->setHost(part_uri.getHost()); // use original, not resolved host name in header + + if (auth_request.hasCredentials()) { + Poco::Net::HTTPBasicCredentials credentials(auth_request); + credentials.authenticate(*request); + } request->setExpectContinue(true); request->setContentLength(data.size()); - LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << uri.toString()); + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << part_uri.toString()); std::ostream & ostr = session->sendRequest(*request); // if (session->peekResponse(response)) // FIXME, shall not go next if not received 100-continue @@ -123,16 +186,78 @@ void WriteBufferFromS3::writePart(const String & data) if (location_iterator == response.end()) break; - uri = location_iterator->second; + part_uri = location_iterator->second; } assertResponseIsOk(*request, response, istr); + + auto etag_iterator = response.find("ETag"); + if (etag_iterator == response.end()) + { + throw Exception("Incorrect response, no ETag", ErrorCodes::INCORRECT_DATA); + } + part_tags.push_back(etag_iterator->second); } void WriteBufferFromS3::complete() { - // FIXME POST ?uploads // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadComplete.html + Poco::Net::HTTPResponse response; + std::unique_ptr request; + HTTPSessionPtr session; + std::istream * istr; /// owned by session + Poco::URI complete_uri = uri; + complete_uri.addQueryParameter("uploadId", upload_id); + + String data; + WriteBufferFromString buffer(data); + writeString("", buffer); // FIXME move to Poco::XML maybe?? + for (size_t i = 0; i < part_tags.size(); ++i) { + writeString("", buffer); + writeIntText(i + 1, buffer); + writeString("", buffer); + writeString(part_tags[i], buffer); + writeString("", buffer); + } + writeString("", buffer); + buffer.finish(); + + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) + { + session = makeHTTPSession(complete_uri, timeouts); + request = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, complete_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request->setHost(complete_uri.getHost()); // use original, not resolved host name in header + + if (auth_request.hasCredentials()) { + Poco::Net::HTTPBasicCredentials credentials(auth_request); + credentials.authenticate(*request); + } + + request->setExpectContinue(true); + + request->setContentLength(data.size()); + + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << complete_uri.toString()); + + std::ostream & ostr = session->sendRequest(*request); +// if (session->peekResponse(response)) // FIXME, shall not go next if not received 100-continue + { + // Received 100-continue. + ostr << data; + } + + istr = &session->receiveResponse(response); + + if (response.getStatus() != 307) + break; + + auto location_iterator = response.find("Location"); + if (location_iterator == response.end()) + break; + + complete_uri = location_iterator->second; + } + assertResponseIsOk(*request, response, istr); } } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 513ab167be5..6aabfc593cf 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -36,6 +36,7 @@ private: std::unique_ptr temporary_buffer; size_t part_number; size_t last_part_size; + String upload_id; std::vector part_tags; public: From 96d093f9ad7fa13005a845ce3ef357a048e2c567 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 17:32:57 +0000 Subject: [PATCH 024/108] Clang compatibility fixes. --- dbms/src/IO/HTTPCommon.cpp | 8 ++++---- dbms/src/IO/HTTPCommon.h | 2 +- dbms/src/IO/ReadBufferFromS3.cpp | 2 +- dbms/src/IO/WriteBufferFromS3.cpp | 12 ++++++------ dbms/src/IO/WriteBufferFromS3.h | 2 +- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index 32970276dd4..9caad1fcbff 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -216,19 +216,19 @@ PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const Connecti std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response) { - auto istr = &session.receiveResponse(response); + auto & istr = session.receiveResponse(response); assertResponseIsOk(request, response, istr); - return istr; + return &istr; } -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream * istr) { +void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr) { auto status = response.getStatus(); if (status != Poco::Net::HTTPResponse::HTTP_OK) { std::stringstream error_message; error_message << "Received error from remote server " << request.getURI() << ". HTTP status code: " << status << " " - << response.getReason() << ", body: " << istr->rdbuf(); + << response.getReason() << ", body: " << istr.rdbuf(); throw Exception(error_message.str(), status == HTTP_TOO_MANY_REQUESTS ? ErrorCodes::RECEIVED_ERROR_TOO_MANY_REQUESTS diff --git a/dbms/src/IO/HTTPCommon.h b/dbms/src/IO/HTTPCommon.h index 1e7500cf230..412429e59d1 100644 --- a/dbms/src/IO/HTTPCommon.h +++ b/dbms/src/IO/HTTPCommon.h @@ -57,6 +57,6 @@ PooledHTTPSessionPtr makePooledHTTPSession(const Poco::URI & uri, const Connecti */ std::istream * receiveResponse( Poco::Net::HTTPClientSession & session, const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response); -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream * istr); +void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr); } diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index f6061c3a8c7..e26f683cdd4 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -49,7 +49,7 @@ ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_, session = makeHTTPSession(uri, timeouts); } - assertResponseIsOk(*request, response, istr); + assertResponseIsOk(*request, response, *istr); impl = std::make_unique(*istr, buffer_size_); } diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 39f840f14c1..ac2a2617397 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -92,7 +92,7 @@ void WriteBufferFromS3::initiate() Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; - std::istream * istr; /// owned by session + std::istream * istr = nullptr; /// owned by session Poco::URI initiate_uri = uri; initiate_uri.setRawQuery("uploads"); // FIXME find how to leave user params as is @@ -124,7 +124,7 @@ void WriteBufferFromS3::initiate() initiate_uri = location_iterator->second; } - assertResponseIsOk(*request, response, istr); + assertResponseIsOk(*request, response, *istr); Poco::XML::InputSource src(*istr); Poco::XML::DOMParser parser; @@ -148,7 +148,7 @@ void WriteBufferFromS3::writePart(const String & data) Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; - std::istream * istr; /// owned by session + std::istream * istr = nullptr; /// owned by session Poco::URI part_uri = uri; part_uri.addQueryParameter("partNumber", std::to_string(part_number)); part_uri.addQueryParameter("uploadId", upload_id); @@ -188,7 +188,7 @@ void WriteBufferFromS3::writePart(const String & data) part_uri = location_iterator->second; } - assertResponseIsOk(*request, response, istr); + assertResponseIsOk(*request, response, *istr); auto etag_iterator = response.find("ETag"); if (etag_iterator == response.end()) @@ -205,7 +205,7 @@ void WriteBufferFromS3::complete() Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; - std::istream * istr; /// owned by session + std::istream * istr = nullptr; /// owned by session Poco::URI complete_uri = uri; complete_uri.addQueryParameter("uploadId", upload_id); @@ -257,7 +257,7 @@ void WriteBufferFromS3::complete() complete_uri = location_iterator->second; } - assertResponseIsOk(*request, response, istr); + assertResponseIsOk(*request, response, *istr); } } diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 6aabfc593cf..23edbbe5fc0 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -50,7 +50,7 @@ public: /// Receives response from the server after sending all data. void finalize(); - ~WriteBufferFromS3(); + ~WriteBufferFromS3() override; private: void initiate(); From af45849ce0090148c99a6102647596f6f3f29a4e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 17:33:20 +0000 Subject: [PATCH 025/108] Minor test fixes. --- .../clickhouse-test | 176 +++++++++++------- 1 file changed, 108 insertions(+), 68 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index 39c62f835a5..62980b3e2ac 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -4,13 +4,13 @@ import http.server import os import subprocess import sys +import tempfile import threading +import time import unittest +import urllib -# Run Go FakeS3 server. -# go run cmd/gofakes3/main.go -backend memory -host :9990 -initialbucket abc - config = os.path.join(os.path.dirname(sys.argv[0]), 'config.xml') test_csv = os.path.join(os.path.dirname(sys.argv[0]), 'test.csv') format = 'column1 UInt32, column2 UInt32, column3 UInt32' @@ -25,6 +25,7 @@ fakes3_port = 9990 localhost = '127.0.0.1' bucket = 'abc' + prepare_put_queries = [ "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(localhost, fakes3_port, bucket, format, values), ] @@ -39,31 +40,36 @@ queries = [ "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), ] -put_queries = [ - "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}" - .format(redirecting_host, preserving_data_port, format, values), - "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}" - .format(redirecting_host, redirecting_preserving_data_port, format, other_values), -] +put_query = "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}".format(redirecting_host, preserving_data_port, format, values) + +redirect_put_query = "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}".format(redirecting_host, redirecting_preserving_data_port, format, other_values) check_queries = [ "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, fakes3_port, bucket, format), ] +def run_query(query): + result = subprocess.run([os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), '-c', config, '-q', query] + , stdout=subprocess.PIPE + , universal_newlines=True) + result.check_returncode() + return result.stdout + + class RedirectingToHTTPHTTPServer(http.server.BaseHTTPRequestHandler): def do_GET(self): self.send_response(307) self.send_header('Content-type', 'text/xml') self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') self.end_headers() - self.wfile.write(bytes(r''' + self.wfile.write(r''' TemporaryRedirect Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. - johnsmith.s3-gztb4pa9sq.amazonaws.com -''', "utf-8")) + storage.yandexcloud.net +'''.encode()) class RedirectingToHTTPSHTTPServer(http.server.BaseHTTPRequestHandler): @@ -72,16 +78,17 @@ class RedirectingToHTTPSHTTPServer(http.server.BaseHTTPRequestHandler): self.send_header('Content-type', 'text/xml') self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') self.end_headers() - self.wfile.write(bytes(r''' + self.wfile.write(r''' TemporaryRedirect Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. - johnsmith.s3-gztb4pa9sq.amazonaws.com -''', "utf-8")) + storage.yandexcloud.net +'''.encode()) received_data = [] +received_data_completed = False class PreservingDataServer(http.server.BaseHTTPRequestHandler): @@ -91,11 +98,34 @@ class PreservingDataServer(http.server.BaseHTTPRequestHandler): print('Received Expect-100', file=sys.stderr) return True + def do_POST(self): + self.send_response(200) + query = urllib.parse.urlparse(self.path).query + print('POST', query) + if query == 'uploads': + data = r''' +TEST'''.encode() + self.send_header('Content-length', str(len(data))) + self.send_header('Content-type', 'text/plain') + self.end_headers() + self.wfile.write(data) + else: + data = self.rfile.read(int(self.headers.get('Content-Length'))) + assert query == 'uploadId=TEST' + assert data == b'1hello-etag' + self.send_header('Content-type', 'text/plain') + self.end_headers() + global received_data_completed + received_data_completed = True + def do_PUT(self): self.send_response(200) self.send_header('Content-type', 'text/plain') + self.send_header('ETag', 'hello-etag') self.end_headers() + query = urllib.parse.urlparse(self.path).query print('Content-Length =', self.headers.get('Content-Length'), file=sys.stderr) + print('PUT', query) assert self.headers.get('Content-Length') assert self.headers['Expect'] == '100-continue' received_data.append(self.rfile.read()) @@ -107,29 +137,27 @@ class RedirectingPreservingDataServer(http.server.BaseHTTPRequestHandler): def handle_expect_100(self): print('Received Expect-100', file=sys.stderr) + query = urllib.parse.urlparse(self.path).query + if query: + query = '?{}'.format(query) self.send_response(307) self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{}:{}/{}/test.csv'.format(localhost, fakes3_port, bucket)) + self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=fakes3_port, bucket=bucket, query=query)) self.end_headers() - self.wfile.write(bytes(r''' + self.wfile.write(r''' TemporaryRedirect Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. - johnsmith.s3-gztb4pa9sq.amazonaws.com -''', "utf-8")) + {host}:{port} +'''.encode().format(host=localhost, port=fakes3_port)) return False + def do_POST(self): + assert False + def do_PUT(self): assert False - self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.end_headers() - print('Content-Length =', self.headers.get('Content-Length'), file=sys.stderr) - assert self.headers.get('Content-Length') - assert self.headers['Expect'] == '100-continue' - received_data.append(self.rfile.read()) - self.wfile.flush() servers = [] @@ -146,14 +174,48 @@ def redirecting_to_http_thread(): def preserving_thread(): server = http.server.HTTPServer((redirecting_host, preserving_data_port), PreservingDataServer) servers.append(server) - server.handle_request() + while True: + server.handle_request() def redirecting_preserving_thread(): server = http.server.HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataServer) servers.append(server) - server.handle_request() + while True: + server.handle_request() +def run_gofakes3(): + l = threading.Lock() + l.acquire() + + def gofakes3_thread(): + with tempfile.TemporaryDirectory() as d: + subprocess.run(['git', 'clone', 'https://github.com/johannesboyne/gofakes3'], cwd=d).check_returncode() + repo = os.path.join(d, 'gofakes3') + subprocess.run(['git', 'checkout', 'd419e1bd286f47170a4f87851a81f5c30107551a'], cwd=repo).check_returncode() + tool = os.path.join(repo, 'cmd', 'gofakes3', 'main.go') + subprocess.run(['go', 'build', tool], cwd=d).check_returncode() + l.release() + subprocess.run(['./main', '-backend', 'memory', '-host', ':{}'.format(fakes3_port), '-initialbucket', bucket], cwd=d).check_returncode() + + thread = threading.Thread(target=gofakes3_thread) + thread.start() + l.acquire() + time.sleep(0.5) + l.release() + return thread + + +def stop_subprocesses(): + pid = os.getpid() + result = subprocess.run(['pgrep', '-P', str(pid)], stdout=subprocess.PIPE) + result.check_returncode() + for child_pid in result.stdout.splitlines(): + subprocess.run(['kill', child_pid]).check_returncode() + + +run_gofakes3() + jobs = [] jobs.append(threading.Thread(target=redirecting_to_http_thread)) jobs.append(threading.Thread(target=redirecting_to_https_thread)) @@ -163,60 +225,38 @@ jobs.append(threading.Thread(target=redirecting_preserving_thread)) for query in prepare_put_queries: print(query) - result = subprocess.run([ - os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), - '-c', - config, - '-q', - query - ], stdout=subprocess.PIPE, universal_newlines=True) - result.check_returncode() + run_query(query) for query in queries: print(query) - result = subprocess.run([ - os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), - '-c', - config, - '-q', - query - ], stdout=subprocess.PIPE, universal_newlines=True) - result.check_returncode() - unittest.TestCase().assertEqual(list(map(str.split, result.stdout.splitlines())), [ + stdout = run_query(query) + unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ ['1', '2', '3', '6'], ['3', '2', '1', '6'], ['78', '43', '45', '150930'], ]) -for query in put_queries: - print(query) - result = subprocess.run([ - os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), - '-c', - config, - '-q', - query - ], stdout=subprocess.PIPE, universal_newlines=True) - result.check_returncode() - unittest.TestCase().assertEqual(received_data[-1].decode(), '1,2,3\n3,2,1\n78,43,45\n') - # In chunked encoding: - # unittest.TestCase().assertEqual(received_data[-1].decode(), '15\r\n1,2,3\n2,3,1\n78,43,45\n\r\n0\r\n\r\n') +query = put_query +print(query) +received_data_completed = False +run_query(query) +unittest.TestCase().assertEqual(received_data[-1].decode(), '1,2,3\n3,2,1\n78,43,45\n') +unittest.TestCase().assertTrue(received_data_completed) + +query = redirect_put_query +print(query) +run_query(query) for query in check_queries: print(query) - result = subprocess.run([ - os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), - '-c', - config, - '-q', - query - ], stdout=subprocess.PIPE, universal_newlines=True) - result.check_returncode() - unittest.TestCase().assertEqual(list(map(str.split, result.stdout.splitlines())), [ + stdout = run_query(query) + unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ ['1', '1', '1', '1'], ['1', '1', '1', '1'], ['11', '11', '11', '1331'], ]) +stop_subprocesses() + [ server.socket.close() for server in servers ] [ job.join() for job in jobs ] From a9153b2fb36109b0f192c4b17acda82cf78469e8 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 18:06:28 +0000 Subject: [PATCH 026/108] Style fixes. --- dbms/src/IO/HTTPCommon.cpp | 3 ++- dbms/src/IO/WriteBufferFromS3.cpp | 16 ++++++++++------ 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/dbms/src/IO/HTTPCommon.cpp b/dbms/src/IO/HTTPCommon.cpp index 9caad1fcbff..0a7c7e7af66 100644 --- a/dbms/src/IO/HTTPCommon.cpp +++ b/dbms/src/IO/HTTPCommon.cpp @@ -221,7 +221,8 @@ std::istream * receiveResponse( return &istr; } -void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr) { +void assertResponseIsOk(const Poco::Net::HTTPRequest & request, Poco::Net::HTTPResponse & response, std::istream & istr) +{ auto status = response.getStatus(); if (status != Poco::Net::HTTPResponse::HTTP_OK) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index ac2a2617397..69d216b88af 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -24,7 +24,7 @@ namespace ErrorCodes WriteBufferFromS3::WriteBufferFromS3( - const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, + const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_) : BufferWithOwnMemory(buffer_size_, nullptr, 0) , uri {uri_} @@ -102,7 +102,8 @@ void WriteBufferFromS3::initiate() request = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, initiate_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(initiate_uri.getHost()); // use original, not resolved host name in header - if (auth_request.hasCredentials()) { + if (auth_request.hasCredentials()) + { Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); } @@ -140,7 +141,7 @@ void WriteBufferFromS3::initiate() throw Exception("Incorrect XML in response, empty upload id", ErrorCodes::INCORRECT_DATA); } } - + void WriteBufferFromS3::writePart(const String & data) { @@ -159,7 +160,8 @@ void WriteBufferFromS3::writePart(const String & data) request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, part_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(part_uri.getHost()); // use original, not resolved host name in header - if (auth_request.hasCredentials()) { + if (auth_request.hasCredentials()) + { Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); } @@ -212,7 +214,8 @@ void WriteBufferFromS3::complete() String data; WriteBufferFromString buffer(data); writeString("", buffer); // FIXME move to Poco::XML maybe?? - for (size_t i = 0; i < part_tags.size(); ++i) { + for (size_t i = 0; i < part_tags.size(); ++i) + { writeString("", buffer); writeIntText(i + 1, buffer); writeString("", buffer); @@ -228,7 +231,8 @@ void WriteBufferFromS3::complete() request = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, complete_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); request->setHost(complete_uri.getHost()); // use original, not resolved host name in header - if (auth_request.hasCredentials()) { + if (auth_request.hasCredentials()) + { Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); } From fa0de006d06a5981beca624f69f1f65d53ee66c3 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 17 Jun 2019 18:17:51 +0000 Subject: [PATCH 027/108] More style fixes. --- dbms/src/IO/WriteBufferFromS3.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 69d216b88af..70ac6136650 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -161,7 +161,7 @@ void WriteBufferFromS3::writePart(const String & data) request->setHost(part_uri.getHost()); // use original, not resolved host name in header if (auth_request.hasCredentials()) - { + { Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); } @@ -232,7 +232,7 @@ void WriteBufferFromS3::complete() request->setHost(complete_uri.getHost()); // use original, not resolved host name in header if (auth_request.hasCredentials()) - { + { Poco::Net::HTTPBasicCredentials credentials(auth_request); credentials.authenticate(*request); } From c6136c2b16373e4e296dd326507d42bba02f893f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 18 Jun 2019 09:16:09 +0300 Subject: [PATCH 028/108] Test improvement. --- .../00950_table_function_s3_wip/clickhouse-test | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index 62980b3e2ac..9000ea7567f 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -190,13 +190,16 @@ def run_gofakes3(): def gofakes3_thread(): with tempfile.TemporaryDirectory() as d: - subprocess.run(['git', 'clone', 'https://github.com/johannesboyne/gofakes3'], cwd=d).check_returncode() - repo = os.path.join(d, 'gofakes3') - subprocess.run(['git', 'checkout', 'd419e1bd286f47170a4f87851a81f5c30107551a'], cwd=repo).check_returncode() - tool = os.path.join(repo, 'cmd', 'gofakes3', 'main.go') - subprocess.run(['go', 'build', tool], cwd=d).check_returncode() - l.release() - subprocess.run(['./main', '-backend', 'memory', '-host', ':{}'.format(fakes3_port), '-initialbucket', bucket], cwd=d).check_returncode() + try: + subprocess.run(['git', 'clone', 'https://github.com/johannesboyne/gofakes3'], cwd=d).check_returncode() + repo = os.path.join(d, 'gofakes3') + subprocess.run(['git', 'checkout', 'd419e1bd286f47170a4f87851a81f5c30107551a'], cwd=repo).check_returncode() + tool = os.path.join(repo, 'cmd', 'gofakes3', 'main.go') + subprocess.run(['go', 'build', tool], cwd=repo).check_returncode() + finally: + l.release() + binary = os.path.join(repo, 'main') + subprocess.run([binary, '-backend', 'memory', '-host', ':{}'.format(fakes3_port), '-initialbucket', bucket]).check_returncode() thread = threading.Thread(target=gofakes3_thread) thread.start() From 395560df1b5916702ebf7aa25daaa9b3c57c73c7 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 21 Jun 2019 01:16:31 +0300 Subject: [PATCH 029/108] Fixed multipart uploads and 100-continue. --- dbms/src/IO/WriteBufferFromS3.cpp | 4 +-- .../clickhouse-test | 29 ++++++++++++++----- 2 files changed, 24 insertions(+), 9 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 70ac6136650..c8406e00ce8 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -173,7 +173,7 @@ void WriteBufferFromS3::writePart(const String & data) LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << part_uri.toString()); std::ostream & ostr = session->sendRequest(*request); -// if (session->peekResponse(response)) // FIXME, shall not go next if not received 100-continue + if (session->peekResponse(response)) { // Received 100-continue. ostr << data; @@ -244,7 +244,7 @@ void WriteBufferFromS3::complete() LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << complete_uri.toString()); std::ostream & ostr = session->sendRequest(*request); -// if (session->peekResponse(response)) // FIXME, shall not go next if not received 100-continue + if (session->peekResponse(response)) { // Received 100-continue. ostr << data; diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test index 9000ea7567f..148570feede 100755 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test @@ -96,6 +96,8 @@ class PreservingDataServer(http.server.BaseHTTPRequestHandler): def handle_expect_100(self): print('Received Expect-100', file=sys.stderr) + self.send_response_only(100) + self.end_headers() return True def do_POST(self): @@ -137,6 +139,9 @@ class RedirectingPreservingDataServer(http.server.BaseHTTPRequestHandler): def handle_expect_100(self): print('Received Expect-100', file=sys.stderr) + return True + + def do_POST(self): query = urllib.parse.urlparse(self.path).query if query: query = '?{}'.format(query) @@ -150,14 +155,23 @@ class RedirectingPreservingDataServer(http.server.BaseHTTPRequestHandler): Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. {host}:{port} -'''.encode().format(host=localhost, port=fakes3_port)) - return False - - def do_POST(self): - assert False +'''.format(host=localhost, port=fakes3_port).encode()) def do_PUT(self): - assert False + query = urllib.parse.urlparse(self.path).query + if query: + query = '?{}'.format(query) + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=fakes3_port, bucket=bucket, query=query)) + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + {host}:{port} +'''.format(host=localhost, port=fakes3_port).encode()) servers = [] @@ -199,7 +213,7 @@ def run_gofakes3(): finally: l.release() binary = os.path.join(repo, 'main') - subprocess.run([binary, '-backend', 'memory', '-host', ':{}'.format(fakes3_port), '-initialbucket', bucket]).check_returncode() + subprocess.run([binary, '-backend', 'memory', '-host', ':{}'.format(fakes3_port), '-initialbucket', bucket]) thread = threading.Thread(target=gofakes3_thread) thread.start() @@ -262,4 +276,5 @@ for query in check_queries: stop_subprocesses() [ server.socket.close() for server in servers ] +os._exit(0) [ job.join() for job in jobs ] From 97b7635c8a7be42467bb90aef5f4773cb7a06d3d Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 21 Jun 2019 08:24:01 +0300 Subject: [PATCH 030/108] Minimum block size to configuration. --- dbms/src/IO/WriteBufferFromS3.cpp | 11 +++++++---- dbms/src/IO/WriteBufferFromS3.h | 2 ++ dbms/src/Storages/StorageS3.cpp | 3 ++- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index c8406e00ce8..b44170417b1 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -12,7 +12,6 @@ #define DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT 2 -#define DEFAULT_S3_MINIMUM_PART_SIZE 100'000'000 namespace DB { @@ -24,10 +23,14 @@ namespace ErrorCodes WriteBufferFromS3::WriteBufferFromS3( - const Poco::URI & uri_, const ConnectionTimeouts & timeouts_, - const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_) + const Poco::URI & uri_, + size_t minimum_upload_part_size_, + const ConnectionTimeouts & timeouts_, + const Poco::Net::HTTPBasicCredentials & credentials, size_t buffer_size_ +) : BufferWithOwnMemory(buffer_size_, nullptr, 0) , uri {uri_} + , minimum_upload_part_size {minimum_upload_part_size_} , timeouts {timeouts_} , auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} , temporary_buffer {std::make_unique(buffer_string)} @@ -50,7 +53,7 @@ void WriteBufferFromS3::nextImpl() last_part_size += offset(); - if (last_part_size > DEFAULT_S3_MINIMUM_PART_SIZE) + if (last_part_size > minimum_upload_part_size) { temporary_buffer->finish(); writePart(buffer_string); diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 23edbbe5fc0..9e4d8c3be2a 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -30,6 +30,7 @@ class WriteBufferFromS3 : public BufferWithOwnMemory { private: Poco::URI uri; + size_t minimum_upload_part_size; ConnectionTimeouts timeouts; Poco::Net::HTTPRequest auth_request; String buffer_string; @@ -41,6 +42,7 @@ private: public: explicit WriteBufferFromS3(const Poco::URI & uri, + size_t minimum_upload_part_size_, const ConnectionTimeouts & timeouts = {}, const Poco::Net::HTTPBasicCredentials & credentials = {}, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE); diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 474d603878e..1de1bdbccfa 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -84,7 +84,8 @@ namespace const ConnectionTimeouts & timeouts) : sample_block(sample_block_) { - write_buf = std::make_unique(uri, timeouts); + auto minimum_upload_part_size = context.getConfigRef().getUInt64("s3_minimum_upload_part_size", 512'000'000); + write_buf = std::make_unique(uri, minimum_upload_part_size, timeouts); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); } From ab456262d9df22f6cca76223ed16a805de9556d3 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sat, 22 Jun 2019 08:58:05 +0300 Subject: [PATCH 031/108] Fixed multipart mechanism and added a warning about 10k parts. --- dbms/src/IO/WriteBufferFromS3.cpp | 10 +++++++--- dbms/src/IO/WriteBufferFromS3.h | 1 - 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index b44170417b1..5e0714b6b7a 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -12,6 +12,7 @@ #define DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT 2 +#define S3_SOFT_MAX_PARTS 10000 namespace DB { @@ -34,7 +35,6 @@ WriteBufferFromS3::WriteBufferFromS3( , timeouts {timeouts_} , auth_request {Poco::Net::HTTPRequest::HTTP_PUT, uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1} , temporary_buffer {std::make_unique(buffer_string)} - , part_number {1} , last_part_size {0} { if (!credentials.getUsername().empty()) @@ -69,7 +69,6 @@ void WriteBufferFromS3::finalize() if (!buffer_string.empty()) { writePart(buffer_string); - ++part_number; } complete(); @@ -154,9 +153,14 @@ void WriteBufferFromS3::writePart(const String & data) HTTPSessionPtr session; std::istream * istr = nullptr; /// owned by session Poco::URI part_uri = uri; - part_uri.addQueryParameter("partNumber", std::to_string(part_number)); + part_uri.addQueryParameter("partNumber", std::to_string(part_tags.size() + 1)); part_uri.addQueryParameter("uploadId", upload_id); + if (part_tags.size() == S3_SOFT_MAX_PARTS) + { + LOG_WARNING(&Logger::get("WriteBufferFromS3"), "Maximum part number in S3 protocol has reached."); + } + for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { session = makeHTTPSession(part_uri, timeouts); diff --git a/dbms/src/IO/WriteBufferFromS3.h b/dbms/src/IO/WriteBufferFromS3.h index 9e4d8c3be2a..0eb689e468f 100644 --- a/dbms/src/IO/WriteBufferFromS3.h +++ b/dbms/src/IO/WriteBufferFromS3.h @@ -35,7 +35,6 @@ private: Poco::Net::HTTPRequest auth_request; String buffer_string; std::unique_ptr temporary_buffer; - size_t part_number; size_t last_part_size; String upload_id; std::vector part_tags; From c891590709743bc354c2a1141b847d1d0780fb5d Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sat, 22 Jun 2019 08:59:13 +0300 Subject: [PATCH 032/108] Added even better warning. --- dbms/src/IO/WriteBufferFromS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 5e0714b6b7a..e48081d5609 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -158,7 +158,7 @@ void WriteBufferFromS3::writePart(const String & data) if (part_tags.size() == S3_SOFT_MAX_PARTS) { - LOG_WARNING(&Logger::get("WriteBufferFromS3"), "Maximum part number in S3 protocol has reached."); + LOG_WARNING(&Logger::get("WriteBufferFromS3"), "Maximum part number in S3 protocol has reached (too much parts). Server may not accept this whole upload."); } for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) From 3e4af7b844c17914e8b323515a1c0dce77ae7af2 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 26 Jun 2019 00:41:14 +0000 Subject: [PATCH 033/108] Attempt to make integration tests. --- .../integration/test_storage_s3/__init__.py | 0 .../tests/integration/test_storage_s3/test.py | 275 +++++++++++++++++ .../clickhouse-test | 280 ------------------ .../00950_table_function_s3_wip/config.xml | 115 ------- .../00950_table_function_s3_wip/test.csv | 3 - 5 files changed, 275 insertions(+), 398 deletions(-) create mode 100644 dbms/tests/integration/test_storage_s3/__init__.py create mode 100644 dbms/tests/integration/test_storage_s3/test.py delete mode 100755 dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test delete mode 100644 dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml delete mode 100644 dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv diff --git a/dbms/tests/integration/test_storage_s3/__init__.py b/dbms/tests/integration/test_storage_s3/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py new file mode 100644 index 00000000000..155b502bb15 --- /dev/null +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -0,0 +1,275 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster = ClickHouseCluster(__file__) + instance = cluster.add_instance('dummy') + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +import os +import socket +import subprocess +import sys +import tempfile +import threading +import time +import unittest + + +try: + import urllib.parse as urlparse +except ImportError: + import urlparse + +try: + from BaseHTTPServer import BaseHTTPRequestHandler +except ImportError: + from http.server import BaseHTTPRequestHandler + +try: + from BaseHTTPServer import HTTPServer +except ImportError: + from http.server import HTTPServer + + +localhost = '127.0.0.1' + +def GetFreeTCPPorts(n): + result = [] + sockets = [] + for i in range(n): + tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + tcp.bind((localhost, 0)) + addr, port = tcp.getsockname() + result.append(port) + sockets.append(tcp) + [ s.close() for s in sockets ] + return result + +test_csv = os.path.join(os.path.dirname(sys.argv[0]), 'test.csv') +format = 'column1 UInt32, column2 UInt32, column3 UInt32' +values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' +other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' +redirecting_host = localhost +redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port = GetFreeTCPPorts(4) +bucket = 'abc' + + +def test_sophisticated_default(started_cluster): + instance = started_cluster.instances['dummy'] + def run_query(query): + return instance.query(query) + + + prepare_put_queries = [ + "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(localhost, preserving_data_port, bucket, format, values), + ] + + queries = [ + "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), + "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), + ] + + put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(redirecting_host, preserving_data_port, bucket, format, values) + + redirect_put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(redirecting_host, redirecting_preserving_data_port, bucket, format, other_values) + + check_queries = [ + "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, preserving_data_port, bucket, format), + ] + + + class RedirectingToHTTPHandler(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + storage.yandexcloud.net + '''.encode()) + self.finish() + + + class RedirectingToHTTPSHandler(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + storage.yandexcloud.net + '''.encode()) + self.finish() + + + received_data = [] + received_data_completed = False + + + class PreservingDataHandler(BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def handle_expect_100(self): + # FIXME it does not work in Python 2. :( + print('Received Expect-100') + self.send_response_only(100) + self.end_headers() + return True + + def do_POST(self): + self.send_response(200) + query = urlparse.urlparse(self.path).query + print('POST', query) + if query == 'uploads': + data = r''' + TEST'''.encode() + self.send_header('Content-length', str(len(data))) + self.send_header('Content-type', 'text/plain') + self.end_headers() + self.wfile.write(data) + else: + data = self.rfile.read(int(self.headers.get('Content-Length'))) + assert query == 'uploadId=TEST' + assert data == b'1hello-etag' + self.send_header('Content-type', 'text/plain') + self.end_headers() + global received_data_completed + received_data_completed = True + self.finish() + + def do_PUT(self): + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.send_header('ETag', 'hello-etag') + self.end_headers() + query = urlparse.urlparse(self.path).query + path = urlparse.urlparse(self.path).path + print('Content-Length =', self.headers.get('Content-Length')) + print('PUT', query) + assert self.headers.get('Content-Length') + assert self.headers['Expect'] == '100-continue' + data = self.rfile.read() + received_data.append(data) + print('PUT to {}'.format(path)) + self.server.storage[path] = data + self.finish() + + def do_GET(self): + path = urlparse.urlparse(self.path).path + if path in self.server.storage: + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.send_header('Content-length', str(len(self.server.storage[path]))) + self.end_headers() + self.wfile.write(self.server.storage[path]) + else: + self.send_response(404) + self.end_headers() + self.finish() + + + class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def handle_expect_100(self): + print('Received Expect-100') + return True + + def do_POST(self): + query = urlparse.urlparse(self.path).query + if query: + query = '?{}'.format(query) + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + {host}:{port} + '''.format(host=localhost, port=preserving_data_port).encode()) + self.finish() + + def do_PUT(self): + query = urlparse.urlparse(self.path).query + if query: + query = '?{}'.format(query) + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + {host}:{port} + '''.format(host=localhost, port=preserving_data_port).encode()) + self.finish() + + + servers = [] + servers.append(HTTPServer((redirecting_host, redirecting_to_https_port), RedirectingToHTTPSHandler)) + servers.append(HTTPServer((redirecting_host, redirecting_to_http_port), RedirectingToHTTPHandler)) + servers.append(HTTPServer((redirecting_host, preserving_data_port), PreservingDataHandler)) + servers[-1].storage = {} + servers.append(HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) + jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] + [ job.start() for job in jobs ] + + try: + for query in prepare_put_queries: + print(query) + run_query(query) + + for query in queries: + print(query) + stdout = run_query(query) + unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ + ['1', '2', '3', '6'], + ['3', '2', '1', '6'], + ['78', '43', '45', '150930'], + ]) + + query = put_query + print(query) + received_data_completed = False + run_query(query) + unittest.TestCase().assertEqual(received_data[-1].decode(), '1,2,3\n3,2,1\n78,43,45\n') + unittest.TestCase().assertTrue(received_data_completed) + + query = redirect_put_query + print(query) + run_query(query) + + for query in check_queries: + print(query) + stdout = run_query(query) + unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ + ['1', '1', '1', '1'], + ['1', '1', '1', '1'], + ['11', '11', '11', '1331'], + ]) + + finally: + [ server.shutdown() for server in servers ] + [ job.join() for job in jobs ] diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test deleted file mode 100755 index 148570feede..00000000000 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/clickhouse-test +++ /dev/null @@ -1,280 +0,0 @@ -#!/usr/bin/env python3 - -import http.server -import os -import subprocess -import sys -import tempfile -import threading -import time -import unittest -import urllib - - -config = os.path.join(os.path.dirname(sys.argv[0]), 'config.xml') -test_csv = os.path.join(os.path.dirname(sys.argv[0]), 'test.csv') -format = 'column1 UInt32, column2 UInt32, column3 UInt32' -values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' -other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' -redirecting_host = '127.0.0.1' -redirecting_to_http_port = 12345 -redirecting_to_https_port = 12346 -preserving_data_port = 12347 -redirecting_preserving_data_port = 12348 -fakes3_port = 9990 -localhost = '127.0.0.1' -bucket = 'abc' - - -prepare_put_queries = [ - "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(localhost, fakes3_port, bucket, format, values), -] - -queries = [ - "select *, column1*column2*column3 from file('{}', 'CSV', '{}')".format(test_csv, format), - "select *, column1*column2*column3 from url('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), - "select *, column1*column2*column3 from s3('http://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), - "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, fakes3_port, bucket, format), - "select *, column1*column2*column3 from s3('https://storage.yandexcloud.net/milovidov/test.csv', 'CSV', '{}')".format(format), - "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), - "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), -] - -put_query = "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}".format(redirecting_host, preserving_data_port, format, values) - -redirect_put_query = "insert into table function s3('http://{}:{}/', 'CSV', '{}') values {}".format(redirecting_host, redirecting_preserving_data_port, format, other_values) - -check_queries = [ - "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, fakes3_port, bucket, format), -] - - -def run_query(query): - result = subprocess.run([os.path.expanduser('~/ClickHouse-bin/dbms/programs/clickhouse-local'), '-c', config, '-q', query] - , stdout=subprocess.PIPE - , universal_newlines=True) - result.check_returncode() - return result.stdout - - -class RedirectingToHTTPHTTPServer(http.server.BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - storage.yandexcloud.net -'''.encode()) - - -class RedirectingToHTTPSHTTPServer(http.server.BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - storage.yandexcloud.net -'''.encode()) - - -received_data = [] -received_data_completed = False - - -class PreservingDataServer(http.server.BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' - - def handle_expect_100(self): - print('Received Expect-100', file=sys.stderr) - self.send_response_only(100) - self.end_headers() - return True - - def do_POST(self): - self.send_response(200) - query = urllib.parse.urlparse(self.path).query - print('POST', query) - if query == 'uploads': - data = r''' -TEST'''.encode() - self.send_header('Content-length', str(len(data))) - self.send_header('Content-type', 'text/plain') - self.end_headers() - self.wfile.write(data) - else: - data = self.rfile.read(int(self.headers.get('Content-Length'))) - assert query == 'uploadId=TEST' - assert data == b'1hello-etag' - self.send_header('Content-type', 'text/plain') - self.end_headers() - global received_data_completed - received_data_completed = True - - def do_PUT(self): - self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('ETag', 'hello-etag') - self.end_headers() - query = urllib.parse.urlparse(self.path).query - print('Content-Length =', self.headers.get('Content-Length'), file=sys.stderr) - print('PUT', query) - assert self.headers.get('Content-Length') - assert self.headers['Expect'] == '100-continue' - received_data.append(self.rfile.read()) - self.wfile.flush() - - -class RedirectingPreservingDataServer(http.server.BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' - - def handle_expect_100(self): - print('Received Expect-100', file=sys.stderr) - return True - - def do_POST(self): - query = urllib.parse.urlparse(self.path).query - if query: - query = '?{}'.format(query) - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=fakes3_port, bucket=bucket, query=query)) - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - {host}:{port} -'''.format(host=localhost, port=fakes3_port).encode()) - - def do_PUT(self): - query = urllib.parse.urlparse(self.path).query - if query: - query = '?{}'.format(query) - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=fakes3_port, bucket=bucket, query=query)) - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - {host}:{port} -'''.format(host=localhost, port=fakes3_port).encode()) - - -servers = [] -def redirecting_to_https_thread(): - server = http.server.HTTPServer((redirecting_host, redirecting_to_https_port), RedirectingToHTTPSHTTPServer) - servers.append(server) - server.handle_request() - -def redirecting_to_http_thread(): - server = http.server.HTTPServer((redirecting_host, redirecting_to_http_port), RedirectingToHTTPHTTPServer) - servers.append(server) - server.handle_request() - -def preserving_thread(): - server = http.server.HTTPServer((redirecting_host, preserving_data_port), PreservingDataServer) - servers.append(server) - while True: - server.handle_request() - -def redirecting_preserving_thread(): - server = http.server.HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataServer) - servers.append(server) - while True: - server.handle_request() - - -def run_gofakes3(): - l = threading.Lock() - l.acquire() - - def gofakes3_thread(): - with tempfile.TemporaryDirectory() as d: - try: - subprocess.run(['git', 'clone', 'https://github.com/johannesboyne/gofakes3'], cwd=d).check_returncode() - repo = os.path.join(d, 'gofakes3') - subprocess.run(['git', 'checkout', 'd419e1bd286f47170a4f87851a81f5c30107551a'], cwd=repo).check_returncode() - tool = os.path.join(repo, 'cmd', 'gofakes3', 'main.go') - subprocess.run(['go', 'build', tool], cwd=repo).check_returncode() - finally: - l.release() - binary = os.path.join(repo, 'main') - subprocess.run([binary, '-backend', 'memory', '-host', ':{}'.format(fakes3_port), '-initialbucket', bucket]) - - thread = threading.Thread(target=gofakes3_thread) - thread.start() - l.acquire() - time.sleep(0.5) - l.release() - return thread - - -def stop_subprocesses(): - pid = os.getpid() - result = subprocess.run(['pgrep', '-P', str(pid)], stdout=subprocess.PIPE) - result.check_returncode() - for child_pid in result.stdout.splitlines(): - subprocess.run(['kill', child_pid]).check_returncode() - - -run_gofakes3() - -jobs = [] -jobs.append(threading.Thread(target=redirecting_to_http_thread)) -jobs.append(threading.Thread(target=redirecting_to_https_thread)) -jobs.append(threading.Thread(target=preserving_thread)) -jobs.append(threading.Thread(target=redirecting_preserving_thread)) -[ job.start() for job in jobs ] - -for query in prepare_put_queries: - print(query) - run_query(query) - -for query in queries: - print(query) - stdout = run_query(query) - unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ - ['1', '2', '3', '6'], - ['3', '2', '1', '6'], - ['78', '43', '45', '150930'], - ]) - -query = put_query -print(query) -received_data_completed = False -run_query(query) -unittest.TestCase().assertEqual(received_data[-1].decode(), '1,2,3\n3,2,1\n78,43,45\n') -unittest.TestCase().assertTrue(received_data_completed) - -query = redirect_put_query -print(query) -run_query(query) - -for query in check_queries: - print(query) - stdout = run_query(query) - unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ - ['1', '1', '1', '1'], - ['1', '1', '1', '1'], - ['11', '11', '11', '1331'], - ]) - -stop_subprocesses() - -[ server.socket.close() for server in servers ] -os._exit(0) -[ job.join() for job in jobs ] diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml deleted file mode 100644 index 7675c696456..00000000000 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/config.xml +++ /dev/null @@ -1,115 +0,0 @@ - - default - - trace - ~/clickhouse-server.log - ~/clickhouse-server.err.log - 1000M - 10 - - - - - - 8 - - - - - - - - - - - ::/0 - - - - default - - - default - - - - - - - a = 1 - - - - - a + b < 1 or c - d > 5 - - - - - c = 1 - - - - - - - - - - - - - - - - 3600 - - - 0 - 0 - 0 - 0 - 0 - - - - diff --git a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv b/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv deleted file mode 100644 index a2325127dec..00000000000 --- a/dbms/tests/queries/0_stateless/00950_table_function_s3_wip/test.csv +++ /dev/null @@ -1,3 +0,0 @@ -1,2,3 -3,2,1 -78,43,45 From 8c4eb13be6f59677349b707303b8f65bb0737d0e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 2 Jul 2019 23:17:00 +0000 Subject: [PATCH 034/108] Fixed unavailable test servers issue in test_storage_s3. --- .../tests/integration/test_storage_s3/test.py | 53 +++++++++++-------- 1 file changed, 30 insertions(+), 23 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 155b502bb15..43de0e142ef 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -40,31 +40,37 @@ except ImportError: from http.server import HTTPServer -localhost = '127.0.0.1' - -def GetFreeTCPPorts(n): - result = [] - sockets = [] - for i in range(n): - tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - tcp.bind((localhost, 0)) - addr, port = tcp.getsockname() - result.append(port) - sockets.append(tcp) - [ s.close() for s in sockets ] - return result - -test_csv = os.path.join(os.path.dirname(sys.argv[0]), 'test.csv') -format = 'column1 UInt32, column2 UInt32, column3 UInt32' -values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' -other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' -redirecting_host = localhost -redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port = GetFreeTCPPorts(4) -bucket = 'abc' - - def test_sophisticated_default(started_cluster): instance = started_cluster.instances['dummy'] + + def GetCurrentIP(): + s = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) + s.connect(("8.8.8.8", 80)) + ip = s.getsockname()[0] + s.close() + return ip + + localhost = GetCurrentIP() + + def GetFreeTCPPorts(n): + result = [] + sockets = [] + for i in range(n): + tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + tcp.bind((localhost, 0)) + addr, port = tcp.getsockname() + result.append(port) + sockets.append(tcp) + [ s.close() for s in sockets ] + return result + + format = 'column1 UInt32, column2 UInt32, column3 UInt32' + values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' + other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' + redirecting_host = localhost + redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port = GetFreeTCPPorts(4) + bucket = 'abc' + def run_query(query): return instance.query(query) @@ -237,6 +243,7 @@ def test_sophisticated_default(started_cluster): [ job.start() for job in jobs ] try: + subprocess.check_call(['ss', '-an']) for query in prepare_put_queries: print(query) run_query(query) From e9300ffbb55faff295650a8dc8e3411e5afce6cd Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sun, 21 Jul 2019 11:45:01 +0000 Subject: [PATCH 035/108] Tests finally works! --- dbms/tests/integration/helpers/cluster.py | 4 +- dbms/tests/integration/runner | 2 +- .../tests/integration/test_storage_s3/test.py | 85 ++++++++++++++----- 3 files changed, 68 insertions(+), 23 deletions(-) diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index aadd2e70a52..4131ee08653 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -225,12 +225,12 @@ class ClickHouseCluster: def restart_instance_with_ip_change(self, node, new_ip): if '::' in new_ip: if node.ipv6_address is None: - raise Exception("You shoud specity ipv6_address in add_node method") + raise Exception("You should specity ipv6_address in add_node method") self._replace(node.docker_compose_path, node.ipv6_address, new_ip) node.ipv6_address = new_ip else: if node.ipv4_address is None: - raise Exception("You shoud specity ipv4_address in add_node method") + raise Exception("You should specity ipv4_address in add_node method") self._replace(node.docker_compose_path, node.ipv4_address, new_ip) node.ipv4_address = new_ip subprocess.check_call(self.base_cmd + ["stop", node.name]) diff --git a/dbms/tests/integration/runner b/dbms/tests/integration/runner index 0d0ec929b96..071df8b1fd0 100755 --- a/dbms/tests/integration/runner +++ b/dbms/tests/integration/runner @@ -107,4 +107,4 @@ if __name__ == "__main__": ) #print(cmd) - subprocess.check_call(cmd, shell=True) \ No newline at end of file + subprocess.check_call(cmd, shell=True) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 43de0e142ef..8dba1e9a440 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -21,7 +21,6 @@ import sys import tempfile import threading import time -import unittest try: @@ -40,6 +39,10 @@ except ImportError: from http.server import HTTPServer +received_data = [] +received_data_completed = False + + def test_sophisticated_default(started_cluster): instance = started_cluster.instances['dummy'] @@ -72,7 +75,10 @@ def test_sophisticated_default(started_cluster): bucket = 'abc' def run_query(query): - return instance.query(query) + print('Running query "{}"...'.format(query)) + result = instance.query(query) + print('Query finished') + return result prepare_put_queries = [ @@ -81,7 +87,8 @@ def test_sophisticated_default(started_cluster): queries = [ "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), - "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), +# "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), +# FIXME ] put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(redirecting_host, preserving_data_port, bucket, format, values) @@ -125,13 +132,28 @@ def test_sophisticated_default(started_cluster): self.finish() - received_data = [] - received_data_completed = False - - class PreservingDataHandler(BaseHTTPRequestHandler): protocol_version = 'HTTP/1.1' - + + def parse_request(self): + result = BaseHTTPRequestHandler.parse_request(self) + # Adaptation to Python 3. + if sys.version_info.major == 2 and result == True: + expect = self.headers.get('Expect', "") + if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): + if not self.handle_expect_100(): + return False + return result + + def send_response_only(self, code, message=None): + if message is None: + if code in self.responses: + message = self.responses[code][0] + else: + message = '' + if self.request_version != 'HTTP/0.9': + self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) + def handle_expect_100(self): # FIXME it does not work in Python 2. :( print('Received Expect-100') @@ -154,6 +176,7 @@ def test_sophisticated_default(started_cluster): data = self.rfile.read(int(self.headers.get('Content-Length'))) assert query == 'uploadId=TEST' assert data == b'1hello-etag' + self.send_header('Content-length', '0') # FIXME on python2 somehow connection does not close without this self.send_header('Content-type', 'text/plain') self.end_headers() global received_data_completed @@ -193,7 +216,26 @@ def test_sophisticated_default(started_cluster): class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): protocol_version = 'HTTP/1.1' - + + def parse_request(self): + result = BaseHTTPRequestHandler.parse_request(self) + # Adaptation to Python 3. + if sys.version_info.major == 2 and result == True: + expect = self.headers.get('Expect', "") + if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): + if not self.handle_expect_100(): + return False + return result + + def send_response_only(self, code, message=None): + if message is None: + if code in self.responses: + message = self.responses[code][0] + else: + message = '' + if self.request_version != 'HTTP/0.9': + self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) + def handle_expect_100(self): print('Received Expect-100') return True @@ -243,40 +285,43 @@ def test_sophisticated_default(started_cluster): [ job.start() for job in jobs ] try: - subprocess.check_call(['ss', '-an']) + print('Phase 1') for query in prepare_put_queries: - print(query) run_query(query) + print('Phase 2') for query in queries: - print(query) stdout = run_query(query) - unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ + assert list(map(str.split, stdout.splitlines())) == [ ['1', '2', '3', '6'], ['3', '2', '1', '6'], ['78', '43', '45', '150930'], - ]) + ] + print('Phase 3') query = put_query - print(query) + global received_data_completed received_data_completed = False run_query(query) - unittest.TestCase().assertEqual(received_data[-1].decode(), '1,2,3\n3,2,1\n78,43,45\n') - unittest.TestCase().assertTrue(received_data_completed) + assert received_data[-1].decode() == '1,2,3\n3,2,1\n78,43,45\n' + assert received_data_completed + print('Phase 4') query = redirect_put_query - print(query) run_query(query) for query in check_queries: print(query) stdout = run_query(query) - unittest.TestCase().assertEqual(list(map(str.split, stdout.splitlines())), [ + assert list(map(str.split, stdout.splitlines())) == [ ['1', '1', '1', '1'], ['1', '1', '1', '1'], ['11', '11', '11', '1331'], - ]) + ] finally: + print('Shutting down') [ server.shutdown() for server in servers ] + print('Joining threads') [ job.join() for job in jobs ] + print('Done') From 81f49d97bd3484ebeabe56bc84c5555824ff87f2 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 28 Aug 2019 13:30:26 +0000 Subject: [PATCH 036/108] Minor fix. --- dbms/tests/integration/test_storage_s3/test.py | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 8dba1e9a440..57b86e74c64 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -87,8 +87,6 @@ def test_sophisticated_default(started_cluster): queries = [ "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), -# "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_https_port, format), -# FIXME ] put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(redirecting_host, preserving_data_port, bucket, format, values) @@ -155,7 +153,6 @@ def test_sophisticated_default(started_cluster): self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) def handle_expect_100(self): - # FIXME it does not work in Python 2. :( print('Received Expect-100') self.send_response_only(100) self.end_headers() @@ -176,7 +173,6 @@ def test_sophisticated_default(started_cluster): data = self.rfile.read(int(self.headers.get('Content-Length'))) assert query == 'uploadId=TEST' assert data == b'1hello-etag' - self.send_header('Content-length', '0') # FIXME on python2 somehow connection does not close without this self.send_header('Content-type', 'text/plain') self.end_headers() global received_data_completed From fa01cc162c5ea98f075519be24237f36a854089e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Wed, 4 Sep 2019 16:10:25 +0000 Subject: [PATCH 037/108] Merge fix. --- dbms/src/Storages/StorageS3.cpp | 5 ++++- dbms/src/Storages/StorageS3.h | 2 +- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 1de1bdbccfa..23ad2b35f13 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -140,7 +140,10 @@ BlockInputStreams StorageS3::read(const Names & column_names, return {std::make_shared(block_input, column_defaults, context)}; } -void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/) {} +void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/, TableStructureWriteLockHolder &) +{ + // FIXME +} BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) { diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index a38cd717e36..ecc93f0f616 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -53,7 +53,7 @@ public: BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; + void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) override; protected: Poco::URI uri; From 599ff389f7c15a583ffd74eee1ea896a44413729 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 12 Sep 2019 11:57:55 +0000 Subject: [PATCH 038/108] Merge fix. --- dbms/src/Storages/StorageS3.cpp | 2 +- dbms/src/Storages/StorageS3.h | 3 +++ dbms/src/TableFunctions/TableFunctionS3.cpp | 4 ++-- dbms/src/TableFunctions/TableFunctionS3.h | 2 +- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index 23ad2b35f13..bbc478dc151 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -170,7 +170,7 @@ void registerStorageS3(StorageFactory & factory) String format_name = engine_args[1]->as().value.safeGet(); - return StorageS3::create(uri, args.table_name, format_name, args.columns, args.context); + return StorageS3::create(uri, args.database_name, args.table_name, format_name, args.columns, args.context); }); } } diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index ecc93f0f616..12f223fe0ca 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -16,6 +16,7 @@ class StorageS3 : public ext::shared_ptr_helper, public IStorage { public: StorageS3(const Poco::URI & uri_, + const std::string & database_name_, const std::string & table_name_, const String & format_name_, const ColumnsDescription & columns_, @@ -25,6 +26,7 @@ public: , uri(uri_) , context_global(context_) , format_name(format_name_) + , database_name(database_name_) , table_name(table_name_) { } @@ -61,6 +63,7 @@ protected: private: String format_name; + String database_name; String table_name; }; diff --git a/dbms/src/TableFunctions/TableFunctionS3.cpp b/dbms/src/TableFunctions/TableFunctionS3.cpp index 5c2c6215765..38ca0830e5b 100644 --- a/dbms/src/TableFunctions/TableFunctionS3.cpp +++ b/dbms/src/TableFunctions/TableFunctionS3.cpp @@ -6,10 +6,10 @@ namespace DB { StoragePtr TableFunctionS3::getStorage( - const String & source, const String & format, const Block & sample_block, Context & global_context) const + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const { Poco::URI uri(source); - return StorageS3::create(uri, getName(), format, ColumnsDescription{sample_block.getNamesAndTypesList()}, global_context); + return StorageS3::create(uri, getDatabaseName(), table_name, format, columns, global_context); } void registerTableFunctionS3(TableFunctionFactory & factory) diff --git a/dbms/src/TableFunctions/TableFunctionS3.h b/dbms/src/TableFunctions/TableFunctionS3.h index 04826a01d9b..a4966be13c7 100644 --- a/dbms/src/TableFunctions/TableFunctionS3.h +++ b/dbms/src/TableFunctions/TableFunctionS3.h @@ -20,6 +20,6 @@ public: private: StoragePtr getStorage( - const String & source, const String & format, const Block & sample_block, Context & global_context) const override; + const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; }; } From 1b715069fb832c26267a7ff37c486171df0a43c3 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 12 Sep 2019 14:38:53 +0000 Subject: [PATCH 039/108] Merge fix. --- dbms/src/IO/WriteBufferFromS3.cpp | 5 +++++ dbms/src/Storages/StorageS3.cpp | 5 +++-- dbms/src/Storages/StorageS3.h | 1 + 3 files changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index e48081d5609..4e574a11c0b 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -49,6 +49,9 @@ void WriteBufferFromS3::nextImpl() if (!offset()) return; + + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "nextImpl(), offset() == " << offset()); + temporary_buffer->write(working_buffer.begin(), offset()); last_part_size += offset(); @@ -65,9 +68,11 @@ void WriteBufferFromS3::nextImpl() void WriteBufferFromS3::finalize() { + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "finalize()"); temporary_buffer->finish(); if (!buffer_string.empty()) { + LOG_TRACE((&Logger::get("WriteBufferFromS3")), "finalize(), writing last part"); writePart(buffer_string); } diff --git a/dbms/src/Storages/StorageS3.cpp b/dbms/src/Storages/StorageS3.cpp index bbc478dc151..59b2ef589a9 100644 --- a/dbms/src/Storages/StorageS3.cpp +++ b/dbms/src/Storages/StorageS3.cpp @@ -140,9 +140,10 @@ BlockInputStreams StorageS3::read(const Names & column_names, return {std::make_shared(block_input, column_defaults, context)}; } -void StorageS3::rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & /*new_table_name*/, TableStructureWriteLockHolder &) +void StorageS3::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) { - // FIXME + table_name = new_table_name; + database_name = new_database_name; } BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) diff --git a/dbms/src/Storages/StorageS3.h b/dbms/src/Storages/StorageS3.h index 12f223fe0ca..ad073aaa14c 100644 --- a/dbms/src/Storages/StorageS3.h +++ b/dbms/src/Storages/StorageS3.h @@ -29,6 +29,7 @@ public: , database_name(database_name_) , table_name(table_name_) { + setColumns(columns_); } String getName() const override From a54b43cd01952698f5786d807fd74138ca66ba1f Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 13 Sep 2019 13:04:21 +0300 Subject: [PATCH 040/108] Better test, minor fix. --- dbms/tests/integration/test_storage_s3/test.py | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 57b86e74c64..73ad752a5a8 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -46,14 +46,7 @@ received_data_completed = False def test_sophisticated_default(started_cluster): instance = started_cluster.instances['dummy'] - def GetCurrentIP(): - s = socket.socket(socket.AF_INET, socket.SOCK_DGRAM) - s.connect(("8.8.8.8", 80)) - ip = s.getsockname()[0] - s.close() - return ip - - localhost = GetCurrentIP() + localhost = 'localhost' def GetFreeTCPPorts(n): result = [] From b3242612854cef4d1efe28371b1cd72792ddc1c8 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 13 Sep 2019 10:18:09 +0000 Subject: [PATCH 041/108] Minor test fix. --- dbms/tests/integration/test_storage_s3/test.py | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 73ad752a5a8..68765f4a6df 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -46,25 +46,23 @@ received_data_completed = False def test_sophisticated_default(started_cluster): instance = started_cluster.instances['dummy'] - localhost = 'localhost' - - def GetFreeTCPPorts(n): + def GetFreeTCPPortsAndIP(n): result = [] sockets = [] for i in range(n): tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - tcp.bind((localhost, 0)) + tcp.bind(('localhost', 0)) addr, port = tcp.getsockname() result.append(port) sockets.append(tcp) [ s.close() for s in sockets ] - return result + return result, addr format = 'column1 UInt32, column2 UInt32, column3 UInt32' values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' + (redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port), localhost = GetFreeTCPPortsAndIP(4) redirecting_host = localhost - redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port = GetFreeTCPPorts(4) bucket = 'abc' def run_query(query): From 2cddcebc312c100d179d842bcbbb0dfde34bbfd6 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 13 Sep 2019 13:17:58 +0000 Subject: [PATCH 042/108] Moved server to clickhouse instance in test. --- .../tests/integration/test_storage_s3/test.py | 262 +++--------------- .../test_storage_s3/test_server.py | 250 +++++++++++++++++ 2 files changed, 285 insertions(+), 227 deletions(-) create mode 100644 dbms/tests/integration/test_storage_s3/test_server.py diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 68765f4a6df..9cedc5b8b4f 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -14,54 +14,36 @@ def started_cluster(): cluster.shutdown() +import json import os -import socket -import subprocess -import sys -import tempfile -import threading import time -try: - import urllib.parse as urlparse -except ImportError: - import urlparse - -try: - from BaseHTTPServer import BaseHTTPRequestHandler -except ImportError: - from http.server import BaseHTTPRequestHandler - -try: - from BaseHTTPServer import HTTPServer -except ImportError: - from http.server import HTTPServer - - -received_data = [] -received_data_completed = False - - def test_sophisticated_default(started_cluster): instance = started_cluster.instances['dummy'] - - def GetFreeTCPPortsAndIP(n): - result = [] - sockets = [] - for i in range(n): - tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) - tcp.bind(('localhost', 0)) - addr, port = tcp.getsockname() - result.append(port) - sockets.append(tcp) - [ s.close() for s in sockets ] - return result, addr + instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), 'test_server.py'), 'test_server.py') + communication_path = '/test_sophisticated_default' + instance.exec_in_container(['python', 'test_server.py', communication_path], detach=True) format = 'column1 UInt32, column2 UInt32, column3 UInt32' values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' - (redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port), localhost = GetFreeTCPPortsAndIP(4) + for i in range(10): + try: + raw = instance.exec_in_container(['cat', communication_path]) + data = json.loads(instance.exec_in_container(['cat', communication_path])) + redirecting_to_http_port = data['redirecting_to_http_port'] + redirecting_to_https_port = data['redirecting_to_https_port'] + preserving_data_port = data['preserving_data_port'] + redirecting_preserving_data_port = data['redirecting_preserving_data_port'] + localhost = data['localhost'] + except: + time.sleep(0.5) + else: + break + else: + assert False, 'Could not initialize mock server' + str(raw) + redirecting_host = localhost bucket = 'abc' @@ -88,189 +70,6 @@ def test_sophisticated_default(started_cluster): "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, preserving_data_port, bucket, format), ] - - class RedirectingToHTTPHandler(BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - storage.yandexcloud.net - '''.encode()) - self.finish() - - - class RedirectingToHTTPSHandler(BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - storage.yandexcloud.net - '''.encode()) - self.finish() - - - class PreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' - - def parse_request(self): - result = BaseHTTPRequestHandler.parse_request(self) - # Adaptation to Python 3. - if sys.version_info.major == 2 and result == True: - expect = self.headers.get('Expect', "") - if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): - if not self.handle_expect_100(): - return False - return result - - def send_response_only(self, code, message=None): - if message is None: - if code in self.responses: - message = self.responses[code][0] - else: - message = '' - if self.request_version != 'HTTP/0.9': - self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) - - def handle_expect_100(self): - print('Received Expect-100') - self.send_response_only(100) - self.end_headers() - return True - - def do_POST(self): - self.send_response(200) - query = urlparse.urlparse(self.path).query - print('POST', query) - if query == 'uploads': - data = r''' - TEST'''.encode() - self.send_header('Content-length', str(len(data))) - self.send_header('Content-type', 'text/plain') - self.end_headers() - self.wfile.write(data) - else: - data = self.rfile.read(int(self.headers.get('Content-Length'))) - assert query == 'uploadId=TEST' - assert data == b'1hello-etag' - self.send_header('Content-type', 'text/plain') - self.end_headers() - global received_data_completed - received_data_completed = True - self.finish() - - def do_PUT(self): - self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('ETag', 'hello-etag') - self.end_headers() - query = urlparse.urlparse(self.path).query - path = urlparse.urlparse(self.path).path - print('Content-Length =', self.headers.get('Content-Length')) - print('PUT', query) - assert self.headers.get('Content-Length') - assert self.headers['Expect'] == '100-continue' - data = self.rfile.read() - received_data.append(data) - print('PUT to {}'.format(path)) - self.server.storage[path] = data - self.finish() - - def do_GET(self): - path = urlparse.urlparse(self.path).path - if path in self.server.storage: - self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('Content-length', str(len(self.server.storage[path]))) - self.end_headers() - self.wfile.write(self.server.storage[path]) - else: - self.send_response(404) - self.end_headers() - self.finish() - - - class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' - - def parse_request(self): - result = BaseHTTPRequestHandler.parse_request(self) - # Adaptation to Python 3. - if sys.version_info.major == 2 and result == True: - expect = self.headers.get('Expect', "") - if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): - if not self.handle_expect_100(): - return False - return result - - def send_response_only(self, code, message=None): - if message is None: - if code in self.responses: - message = self.responses[code][0] - else: - message = '' - if self.request_version != 'HTTP/0.9': - self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) - - def handle_expect_100(self): - print('Received Expect-100') - return True - - def do_POST(self): - query = urlparse.urlparse(self.path).query - if query: - query = '?{}'.format(query) - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - {host}:{port} - '''.format(host=localhost, port=preserving_data_port).encode()) - self.finish() - - def do_PUT(self): - query = urlparse.urlparse(self.path).query - if query: - query = '?{}'.format(query) - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - {host}:{port} - '''.format(host=localhost, port=preserving_data_port).encode()) - self.finish() - - - servers = [] - servers.append(HTTPServer((redirecting_host, redirecting_to_https_port), RedirectingToHTTPSHandler)) - servers.append(HTTPServer((redirecting_host, redirecting_to_http_port), RedirectingToHTTPHandler)) - servers.append(HTTPServer((redirecting_host, preserving_data_port), PreservingDataHandler)) - servers[-1].storage = {} - servers.append(HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) - jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] - [ job.start() for job in jobs ] - try: print('Phase 1') for query in prepare_put_queries: @@ -287,11 +86,24 @@ def test_sophisticated_default(started_cluster): print('Phase 3') query = put_query - global received_data_completed - received_data_completed = False run_query(query) + for i in range(10): + try: + data = json.loads(instance.exec_in_container(['cat', communication_path])) + received_data_completed = data['received_data_completed'] + received_data = data['received_data'] + finalize_data = data['finalize_data'] + finalize_data_query = data['finalize_data_query'] + except: + time.sleep(0.5) + else: + break + else: + assert False, 'Could not read data from mock server'+str(data) assert received_data[-1].decode() == '1,2,3\n3,2,1\n78,43,45\n' assert received_data_completed + assert finalize_data == '1hello-etag' + assert finalize_data_query == 'uploadId=TEST' print('Phase 4') query = redirect_put_query @@ -307,8 +119,4 @@ def test_sophisticated_default(started_cluster): ] finally: - print('Shutting down') - [ server.shutdown() for server in servers ] - print('Joining threads') - [ job.join() for job in jobs ] print('Done') diff --git a/dbms/tests/integration/test_storage_s3/test_server.py b/dbms/tests/integration/test_storage_s3/test_server.py new file mode 100644 index 00000000000..9b2ac3bdb60 --- /dev/null +++ b/dbms/tests/integration/test_storage_s3/test_server.py @@ -0,0 +1,250 @@ +try: + from BaseHTTPServer import BaseHTTPRequestHandler +except ImportError: + from http.server import BaseHTTPRequestHandler + +try: + from BaseHTTPServer import HTTPServer +except ImportError: + from http.server import HTTPServer + +try: + import urllib.parse as urlparse +except ImportError: + import urlparse + +import json +import logging +import os +import socket +import sys +import threading +import time + + +logging.getLogger().setLevel(logging.INFO) +file_handler = logging.FileHandler('/var/log/clickhouse-server/test-server.log', 'a', encoding='utf-8') +file_handler.setFormatter(logging.Formatter('%(asctime)s %(message)s')) +logging.getLogger().addHandler(file_handler) +logging.getLogger().addHandler(logging.StreamHandler()) + +comm_path = sys.argv[1] + +def GetFreeTCPPortsAndIP(n): + result = [] + sockets = [] + for i in range(n): + tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM) + tcp.bind((socket.gethostname(), 0)) + addr, port = tcp.getsockname() + result.append(port) + sockets.append(tcp) + [ s.close() for s in sockets ] + return result, addr + +(redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port), localhost = GetFreeTCPPortsAndIP(4) +data = { + 'redirecting_to_http_port': redirecting_to_http_port, + 'redirecting_to_https_port': redirecting_to_https_port, + 'preserving_data_port': preserving_data_port, + 'redirecting_preserving_data_port': redirecting_preserving_data_port, + 'localhost': localhost +} +redirecting_host = localhost + +with open(comm_path, 'w') as f: + f.write(json.dumps(data)) + + +class RedirectingToHTTPHandler(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + storage.yandexcloud.net +'''.encode()) + self.finish() + + +class RedirectingToHTTPSHandler(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + storage.yandexcloud.net +'''.encode()) + self.finish() + + +class PreservingDataHandler(BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def parse_request(self): + result = BaseHTTPRequestHandler.parse_request(self) + # Adaptation to Python 3. + if sys.version_info.major == 2 and result == True: + expect = self.headers.get('Expect', "") + if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): + if not self.handle_expect_100(): + return False + return result + + def send_response_only(self, code, message=None): + if message is None: + if code in self.responses: + message = self.responses[code][0] + else: + message = '' + if self.request_version != 'HTTP/0.9': + self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) + + def handle_expect_100(self): + logging.info('Received Expect-100') + self.send_response_only(100) + self.end_headers() + return True + + def do_POST(self): + self.send_response(200) + query = urlparse.urlparse(self.path).query + logging.info('POST ' + query) + if query == 'uploads': + post_data = r''' +TEST'''.encode() + self.send_header('Content-length', str(len(post_data))) + self.send_header('Content-type', 'text/plain') + self.end_headers() + self.wfile.write(post_data) + else: + post_data = self.rfile.read(int(self.headers.get('Content-Length'))) + self.send_header('Content-type', 'text/plain') + self.end_headers() + data['received_data_completed'] = True + data['finalize_data'] = post_data + data['finalize_data_query'] = query + with open(comm_path, 'w') as f: + f.write(json.dumps(data)) + self.finish() + + def do_PUT(self): + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.send_header('ETag', 'hello-etag') + self.end_headers() + query = urlparse.urlparse(self.path).query + path = urlparse.urlparse(self.path).path + logging.info('Content-Length = ' + self.headers.get('Content-Length')) + logging.info('PUT ' + query) + assert self.headers.get('Content-Length') + assert self.headers['Expect'] == '100-continue' + put_data = self.rfile.read() + data.setdefault('received_data', []).append(put_data) + with open(comm_path, 'w') as f: + f.write(json.dumps(data)) + logging.info('PUT to {}'.format(path)) + self.server.storage[path] = put_data + self.finish() + + def do_GET(self): + path = urlparse.urlparse(self.path).path + if path in self.server.storage: + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.send_header('Content-length', str(len(self.server.storage[path]))) + self.end_headers() + self.wfile.write(self.server.storage[path]) + else: + self.send_response(404) + self.end_headers() + self.finish() + + +class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def parse_request(self): + result = BaseHTTPRequestHandler.parse_request(self) + # Adaptation to Python 3. + if sys.version_info.major == 2 and result == True: + expect = self.headers.get('Expect', "") + if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): + if not self.handle_expect_100(): + return False + return result + + def send_response_only(self, code, message=None): + if message is None: + if code in self.responses: + message = self.responses[code][0] + else: + message = '' + if self.request_version != 'HTTP/0.9': + self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) + + def handle_expect_100(self): + logging.info('Received Expect-100') + return True + + def do_POST(self): + query = urlparse.urlparse(self.path).query + if query: + query = '?{}'.format(query) + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + {host}:{port} +'''.format(host=localhost, port=preserving_data_port).encode()) + self.finish() + + def do_PUT(self): + query = urlparse.urlparse(self.path).query + if query: + query = '?{}'.format(query) + self.send_response(307) + self.send_header('Content-type', 'text/xml') + self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) + self.end_headers() + self.wfile.write(r''' + + TemporaryRedirect + Please re-send this request to the specified temporary endpoint. + Continue to use the original request endpoint for future requests. + {host}:{port} +'''.format(host=localhost, port=preserving_data_port).encode()) + self.finish() + + +servers = [] +servers.append(HTTPServer((redirecting_host, redirecting_to_https_port), RedirectingToHTTPSHandler)) +servers.append(HTTPServer((redirecting_host, redirecting_to_http_port), RedirectingToHTTPHandler)) +servers.append(HTTPServer((redirecting_host, preserving_data_port), PreservingDataHandler)) +servers[-1].storage = {} +servers.append(HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) +jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] +[ job.start() for job in jobs ] + +time.sleep(60) # Timeout + +logging.info('Shutting down') +[ server.shutdown() for server in servers ] +logging.info('Joining threads') +[ job.join() for job in jobs ] +logging.info('Done') From 4406ad10611071c12c3d869204b0a41052983660 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Fri, 13 Sep 2019 13:53:17 +0000 Subject: [PATCH 043/108] Tests fix. --- dbms/tests/integration/test_storage_s3/test.py | 7 +++++-- dbms/tests/integration/test_storage_s3/test_server.py | 1 + 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 9cedc5b8b4f..c383a2a7bea 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -23,7 +23,8 @@ def test_sophisticated_default(started_cluster): instance = started_cluster.instances['dummy'] instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), 'test_server.py'), 'test_server.py') communication_path = '/test_sophisticated_default' - instance.exec_in_container(['python', 'test_server.py', communication_path], detach=True) + bucket = 'abc' + instance.exec_in_container(['python', 'test_server.py', communication_path, bucket], detach=True) format = 'column1 UInt32, column2 UInt32, column3 UInt32' values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' @@ -45,7 +46,6 @@ def test_sophisticated_default(started_cluster): assert False, 'Could not initialize mock server' + str(raw) redirecting_host = localhost - bucket = 'abc' def run_query(query): print('Running query "{}"...'.format(query)) @@ -117,6 +117,9 @@ def test_sophisticated_default(started_cluster): ['1', '1', '1', '1'], ['11', '11', '11', '1331'], ] + # FIXME check result + + # FIXME tests for multipart finally: print('Done') diff --git a/dbms/tests/integration/test_storage_s3/test_server.py b/dbms/tests/integration/test_storage_s3/test_server.py index 9b2ac3bdb60..aed5996212b 100644 --- a/dbms/tests/integration/test_storage_s3/test_server.py +++ b/dbms/tests/integration/test_storage_s3/test_server.py @@ -29,6 +29,7 @@ logging.getLogger().addHandler(file_handler) logging.getLogger().addHandler(logging.StreamHandler()) comm_path = sys.argv[1] +bucket = sys.argv[2] def GetFreeTCPPortsAndIP(n): result = [] From d53872c30011e92502efa5d7f50f3d463cdce408 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sat, 14 Sep 2019 07:44:46 +0000 Subject: [PATCH 044/108] Tests improvement. --- .../tests/integration/test_storage_s3/test.py | 90 ++++++++++--------- .../test_storage_s3/test_server.py | 60 ++++++------- 2 files changed, 79 insertions(+), 71 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index c383a2a7bea..b975c4c92d5 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -14,112 +14,122 @@ def started_cluster(): cluster.shutdown() +import httplib import json +import logging import os import time +import traceback -def test_sophisticated_default(started_cluster): +logging.getLogger().setLevel(logging.INFO) +logging.getLogger().addHandler(logging.StreamHandler()) + +def test_simple(started_cluster): instance = started_cluster.instances['dummy'] instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), 'test_server.py'), 'test_server.py') - communication_path = '/test_sophisticated_default' + communication_port = 10000 bucket = 'abc' - instance.exec_in_container(['python', 'test_server.py', communication_path, bucket], detach=True) + instance.exec_in_container(['python', 'test_server.py', str(communication_port), bucket], detach=True) + + def get_data(): + conn = httplib.HTTPConnection(started_cluster.instances['dummy'].ip_address, communication_port) + conn.request("GET", "/") + r = conn.getresponse() + raw_data = r.read() + conn.close() + return json.loads(raw_data) format = 'column1 UInt32, column2 UInt32, column3 UInt32' values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' for i in range(10): try: - raw = instance.exec_in_container(['cat', communication_path]) - data = json.loads(instance.exec_in_container(['cat', communication_path])) + data = get_data() redirecting_to_http_port = data['redirecting_to_http_port'] - redirecting_to_https_port = data['redirecting_to_https_port'] preserving_data_port = data['preserving_data_port'] redirecting_preserving_data_port = data['redirecting_preserving_data_port'] - localhost = data['localhost'] except: + logging.error(traceback.format_exc()) time.sleep(0.5) else: break else: - assert False, 'Could not initialize mock server' + str(raw) + assert False, 'Could not initialize mock server' - redirecting_host = localhost + mock_host = started_cluster.instances['dummy'].ip_address def run_query(query): - print('Running query "{}"...'.format(query)) + logging.info('Running query "{}"...'.format(query)) result = instance.query(query) - print('Query finished') + logging.info('Query finished') return result prepare_put_queries = [ - "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(localhost, preserving_data_port, bucket, format, values), + "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(mock_host, preserving_data_port, bucket, format, values), ] queries = [ - "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(redirecting_host, redirecting_to_http_port, format), + "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(mock_host, redirecting_to_http_port, format), ] - put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(redirecting_host, preserving_data_port, bucket, format, values) + put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(mock_host, preserving_data_port, bucket, format, values) - redirect_put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(redirecting_host, redirecting_preserving_data_port, bucket, format, other_values) + redirect_put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(mock_host, redirecting_preserving_data_port, bucket, format, other_values) check_queries = [ - "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(localhost, preserving_data_port, bucket, format), + "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(mock_host, preserving_data_port, bucket, format), ] try: - print('Phase 1') + logging.info('Phase 1') for query in prepare_put_queries: run_query(query) - print('Phase 2') + logging.info('Phase 2') for query in queries: stdout = run_query(query) assert list(map(str.split, stdout.splitlines())) == [ - ['1', '2', '3', '6'], - ['3', '2', '1', '6'], - ['78', '43', '45', '150930'], + ['42', '87', '44', '160776'], + ['55', '33', '81', '147015'], + ['1', '0', '9', '0'], ] - print('Phase 3') + logging.info('Phase 3') query = put_query run_query(query) - for i in range(10): - try: - data = json.loads(instance.exec_in_container(['cat', communication_path])) - received_data_completed = data['received_data_completed'] - received_data = data['received_data'] - finalize_data = data['finalize_data'] - finalize_data_query = data['finalize_data_query'] - except: - time.sleep(0.5) - else: - break - else: - assert False, 'Could not read data from mock server'+str(data) + data = get_data() + received_data_completed = data['received_data_completed'] + received_data = data['received_data'] + finalize_data = data['finalize_data'] + finalize_data_query = data['finalize_data_query'] assert received_data[-1].decode() == '1,2,3\n3,2,1\n78,43,45\n' assert received_data_completed assert finalize_data == '1hello-etag' assert finalize_data_query == 'uploadId=TEST' - print('Phase 4') + logging.info('Phase 4') query = redirect_put_query run_query(query) for query in check_queries: - print(query) + logging.info(query) stdout = run_query(query) assert list(map(str.split, stdout.splitlines())) == [ ['1', '1', '1', '1'], ['1', '1', '1', '1'], ['11', '11', '11', '1331'], ] - # FIXME check result + data = get_data() + received_data = data['received_data'] + assert received_data[-1].decode() == '1,1,1\n1,1,1\n11,11,11\n' # FIXME tests for multipart - finally: - print('Done') + except: + logging.error(traceback.format_exc()) + raise + + else: + logging.info('Done') diff --git a/dbms/tests/integration/test_storage_s3/test_server.py b/dbms/tests/integration/test_storage_s3/test_server.py index aed5996212b..bc22b0df085 100644 --- a/dbms/tests/integration/test_storage_s3/test_server.py +++ b/dbms/tests/integration/test_storage_s3/test_server.py @@ -28,7 +28,7 @@ file_handler.setFormatter(logging.Formatter('%(asctime)s %(message)s')) logging.getLogger().addHandler(file_handler) logging.getLogger().addHandler(logging.StreamHandler()) -comm_path = sys.argv[1] +communication_port = int(sys.argv[1]) bucket = sys.argv[2] def GetFreeTCPPortsAndIP(n): @@ -43,41 +43,34 @@ def GetFreeTCPPortsAndIP(n): [ s.close() for s in sockets ] return result, addr -(redirecting_to_http_port, redirecting_to_https_port, preserving_data_port, redirecting_preserving_data_port), localhost = GetFreeTCPPortsAndIP(4) +(redirecting_to_http_port, simple_server_port, preserving_data_port, redirecting_preserving_data_port), localhost = GetFreeTCPPortsAndIP(4) data = { 'redirecting_to_http_port': redirecting_to_http_port, - 'redirecting_to_https_port': redirecting_to_https_port, 'preserving_data_port': preserving_data_port, 'redirecting_preserving_data_port': redirecting_preserving_data_port, - 'localhost': localhost } redirecting_host = localhost -with open(comm_path, 'w') as f: - f.write(json.dumps(data)) + +class SimpleHTTPServerHandler(BaseHTTPRequestHandler): + def do_GET(self): + logging.info('GET {}'.format(self.path)) + if self.path == '/milovidov/test.csv': + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.end_headers() + self.wfile.write('42,87,44\n55,33,81\n1,0,9\n') + else: + self.send_response(404) + self.end_headers() + self.finish() class RedirectingToHTTPHandler(BaseHTTPRequestHandler): def do_GET(self): self.send_response(307) self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://storage.yandexcloud.net/milovidov/test.csv') - self.end_headers() - self.wfile.write(r''' - - TemporaryRedirect - Please re-send this request to the specified temporary endpoint. - Continue to use the original request endpoint for future requests. - storage.yandexcloud.net -'''.encode()) - self.finish() - - -class RedirectingToHTTPSHandler(BaseHTTPRequestHandler): - def do_GET(self): - self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'https://storage.yandexcloud.net/milovidov/test.csv') + self.send_header('Location', 'http://{}:{}/milovidov/test.csv'.format(localhost, simple_server_port)) self.end_headers() self.wfile.write(r''' @@ -135,8 +128,6 @@ class PreservingDataHandler(BaseHTTPRequestHandler): data['received_data_completed'] = True data['finalize_data'] = post_data data['finalize_data_query'] = query - with open(comm_path, 'w') as f: - f.write(json.dumps(data)) self.finish() def do_PUT(self): @@ -152,8 +143,6 @@ class PreservingDataHandler(BaseHTTPRequestHandler): assert self.headers['Expect'] == '100-continue' put_data = self.rfile.read() data.setdefault('received_data', []).append(put_data) - with open(comm_path, 'w') as f: - f.write(json.dumps(data)) logging.info('PUT to {}'.format(path)) self.server.storage[path] = put_data self.finish() @@ -233,12 +222,21 @@ class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): self.finish() +class CommunicationServerHandler(BaseHTTPRequestHandler): + def do_GET(self): + self.send_response(200) + self.end_headers() + self.wfile.write(json.dumps(data)) + self.finish() + + servers = [] -servers.append(HTTPServer((redirecting_host, redirecting_to_https_port), RedirectingToHTTPSHandler)) -servers.append(HTTPServer((redirecting_host, redirecting_to_http_port), RedirectingToHTTPHandler)) -servers.append(HTTPServer((redirecting_host, preserving_data_port), PreservingDataHandler)) +servers.append(HTTPServer((localhost, communication_port), CommunicationServerHandler)) +servers.append(HTTPServer((localhost, redirecting_to_http_port), RedirectingToHTTPHandler)) +servers.append(HTTPServer((localhost, preserving_data_port), PreservingDataHandler)) servers[-1].storage = {} -servers.append(HTTPServer((redirecting_host, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) +servers.append(HTTPServer((localhost, simple_server_port), SimpleHTTPServerHandler)) +servers.append(HTTPServer((localhost, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] [ job.start() for job in jobs ] From 0bad4b4a05870dd76a23027a65af37afcfa15f71 Mon Sep 17 00:00:00 2001 From: sfod Date: Wed, 18 Sep 2019 16:08:51 +0300 Subject: [PATCH 045/108] Throw exceptions if WITH TOTALS/ROLLUP/CUBE are specified without aggregate functions --- .../Interpreters/InterpreterSelectQuery.cpp | 28 ++----------------- ...01013_totals_without_aggregation.reference | 3 ++ .../01013_totals_without_aggregation.sh | 15 ++++++++++ 3 files changed, 21 insertions(+), 25 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference create mode 100755 dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 39a1976d2d4..10f71b9337e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1209,33 +1209,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeExpression(pipeline, expressions.before_order_and_select); executeDistinct(pipeline, true, expressions.selected_columns); - need_second_distinct_pass = query.distinct && pipeline.hasMixedStreams(); } - else - { - need_second_distinct_pass = query.distinct && pipeline.hasMixedStreams(); + else if (query.group_by_with_totals || query.group_by_with_rollup || query.group_by_with_cube) + throw Exception("WITH TOTALS, ROLLUP or CUBE are not supported without aggregation", ErrorCodes::LOGICAL_ERROR); - if (query.group_by_with_totals && !aggregate_final) - { - bool final = !query.group_by_with_rollup && !query.group_by_with_cube; - executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, final); - } - - if ((query.group_by_with_rollup || query.group_by_with_cube) && !aggregate_final) - { - if (query.group_by_with_rollup) - executeRollupOrCube(pipeline, Modificator::ROLLUP); - else if (query.group_by_with_cube) - executeRollupOrCube(pipeline, Modificator::CUBE); - - if (expressions.has_having) - { - if (query.group_by_with_totals) - throw Exception("WITH TOTALS and WITH ROLLUP or CUBE are not supported together in presence of HAVING", ErrorCodes::NOT_IMPLEMENTED); - executeHaving(pipeline, expressions.before_having); - } - } - } + need_second_distinct_pass = query.distinct && pipeline.hasMixedStreams(); if (expressions.has_order_by) { diff --git a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference new file mode 100644 index 00000000000..7614df8ec46 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference @@ -0,0 +1,3 @@ +ok +ok +ok diff --git a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh new file mode 100755 index 00000000000..c159a73388d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +EXCEPTION_SUCCESS_TEXT=ok + +# Must throw an exception +EXCEPTION_TEXT="WITH TOTALS, ROLLUP or CUBE are not supported without aggregation" +$CLICKHOUSE_CLIENT --query="SELECT 1 AS id, 'hello' AS s WITH TOTALS" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" +$CLICKHOUSE_CLIENT --query="SELECT 1 AS id, 'hello' AS s WITH ROLLUP" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" +$CLICKHOUSE_CLIENT --query="SELECT 1 AS id, 'hello' AS s WITH CUBE" 2>&1 \ + | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" From 67e1cf9b73d3cb78c00f24b3dd406c0f0014cadf Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Wed, 18 Sep 2019 17:35:45 +0200 Subject: [PATCH 046/108] Adding performance test for huge pk (issue #6924) --- dbms/tests/performance/merge_tree_huge_pk.xml | 206 ++++++++++++++++++ 1 file changed, 206 insertions(+) create mode 100644 dbms/tests/performance/merge_tree_huge_pk.xml diff --git a/dbms/tests/performance/merge_tree_huge_pk.xml b/dbms/tests/performance/merge_tree_huge_pk.xml new file mode 100644 index 00000000000..351cc17cb6c --- /dev/null +++ b/dbms/tests/performance/merge_tree_huge_pk.xml @@ -0,0 +1,206 @@ + + loop + + + + 10 + 12000 + + + 50 + 60000 + + + + + CREATE TABLE huge_pk ENGINE = MergeTree ORDER BY ( + c001, c002, c003, c004, c005, c006, c007, c008, c009, c010, c011, c012, c013, c014, c015, c016, c017, c018, c019, c020, + c021, c022, c023, c024, c025, c026, c027, c028, c029, c030, c031, c032, c033, c034, c035, c036, c037, c038, c039, c040, + c041, c042, c043, c044, c045, c046, c047, c048, c049, c050, c051, c052, c053, c054, c055, c056, c057, c058, c059, c060, + c061, c062, c063, c064, c065, c066, c067, c068, c069, c070, c071, c072, c073, c074, c075, c076, c077, c078, c079, c080, + c081, c082, c083, c084, c085, c086, c087, c088, c089, c090, c091, c092, c093, c094, c095, c096, c097, c098, c099, c100, + c101, c102, c103, c104, c105, c106, c107, c108, c109, c110, c111, c112, c113, c114, c115, c116, c117, c118, c119, c120, + c121, c122, c123, c124, c125, c126, c127, c128, c129, c130, c131, c132, c133, c134, c135, c136, c137, c138, c139, c140, + c141, c142, c143, c144, c145, c146, c147, c148, c149, c150, c151, c152, c153, c154, c155, c156, c157, c158, c159, c160, + c161, c162, c163, c164, c165, c166, c167, c168, c169, c170, c171, c172, c173, c174, c175, c176, c177, c178, c179, c180, + c181, c182, c183, c184, c185, c186, c187, c188, c189, c190, c191, c192, c193, c194, c195, c196, c197, c198, c199, c200, + c201, c202, c203, c204, c205, c206, c207, c208, c209, c210, c211, c212, c213, c214, c215, c216, c217, c218, c219, c220, + c221, c222, c223, c224, c225, c226, c227, c228, c229, c230, c231, c232, c233, c234, c235, c236, c237, c238, c239, c240, + c241, c242, c243, c244, c245, c246, c247, c248, c249, c250, c251, c252, c253, c254, c255, c256, c257, c258, c259, c260, + c261, c262, c263, c264, c265, c266, c267, c268, c269, c270, c271, c272, c273, c274, c275, c276, c277, c278, c279, c280, + c281, c282, c283, c284, c285, c286, c287, c288, c289, c290, c291, c292, c293, c294, c295, c296, c297, c298, c299, c300, + c301, c302, c303, c304, c305, c306, c307, c308, c309, c310, c311, c312, c313, c314, c315, c316, c317, c318, c319, c320, + c321, c322, c323, c324, c325, c326, c327, c328, c329, c330, c331, c332, c333, c334, c335, c336, c337, c338, c339, c340, + c341, c342, c343, c344, c345, c346, c347, c348, c349, c350, c351, c352, c353, c354, c355, c356, c357, c358, c359, c360, + c361, c362, c363, c364, c365, c366, c367, c368, c369, c370, c371, c372, c373, c374, c375, c376, c377, c378, c379, c380, + c381, c382, c383, c384, c385, c386, c387, c388, c389, c390, c391, c392, c393, c394, c395, c396, c397, c398, c399, c400, + c401, c402, c403, c404, c405, c406, c407, c408, c409, c410, c411, c412, c413, c414, c415, c416, c417, c418, c419, c420, + c421, c422, c423, c424, c425, c426, c427, c428, c429, c430, c431, c432, c433, c434, c435, c436, c437, c438, c439, c440, + c441, c442, c443, c444, c445, c446, c447, c448, c449, c450, c451, c452, c453, c454, c455, c456, c457, c458, c459, c460, + c461, c462, c463, c464, c465, c466, c467, c468, c469, c470, c471, c472, c473, c474, c475, c476, c477, c478, c479, c480, + c481, c482, c483, c484, c485, c486, c487, c488, c489, c490, c491, c492, c493, c494, c495, c496, c497, c498, c499, c500, + c501, c502, c503, c504, c505, c506, c507, c508, c509, c510, c511, c512, c513, c514, c515, c516, c517, c518, c519, c520, + c521, c522, c523, c524, c525, c526, c527, c528, c529, c530, c531, c532, c533, c534, c535, c536, c537, c538, c539, c540, + c541, c542, c543, c544, c545, c546, c547, c548, c549, c550, c551, c552, c553, c554, c555, c556, c557, c558, c559, c560, + c561, c562, c563, c564, c565, c566, c567, c568, c569, c570, c571, c572, c573, c574, c575, c576, c577, c578, c579, c580, + c581, c582, c583, c584, c585, c586, c587, c588, c589, c590, c591, c592, c593, c594, c595, c596, c597, c598, c599, c600, + c601, c602, c603, c604, c605, c606, c607, c608, c609, c610, c611, c612, c613, c614, c615, c616, c617, c618, c619, c620, + c621, c622, c623, c624, c625, c626, c627, c628, c629, c630, c631, c632, c633, c634, c635, c636, c637, c638, c639, c640, + c641, c642, c643, c644, c645, c646, c647, c648, c649, c650, c651, c652, c653, c654, c655, c656, c657, c658, c659, c660, + c661, c662, c663, c664, c665, c666, c667, c668, c669, c670, c671, c672, c673, c674, c675, c676, c677, c678, c679, c680, + c681, c682, c683, c684, c685, c686, c687, c688, c689, c690, c691, c692, c693, c694, c695, c696, c697, c698, c699, c700) + AS SELECT + rand64( 1) % 5 as c001, rand64( 2) % 5 as c002, rand64( 3) % 5 as c003, rand64( 4) % 5 as c004, rand64( 5) % 5 as c005, + rand64( 6) % 5 as c006, rand64( 7) % 5 as c007, rand64( 8) % 5 as c008, rand64( 9) % 5 as c009, rand64( 10) % 5 as c010, + rand64( 11) % 5 as c011, rand64( 12) % 5 as c012, rand64( 13) % 5 as c013, rand64( 14) % 5 as c014, rand64( 15) % 5 as c015, + rand64( 16) % 5 as c016, rand64( 17) % 5 as c017, rand64( 18) % 5 as c018, rand64( 19) % 5 as c019, rand64( 20) % 5 as c020, + rand64( 21) % 5 as c021, rand64( 22) % 5 as c022, rand64( 23) % 5 as c023, rand64( 24) % 5 as c024, rand64( 25) % 5 as c025, + rand64( 26) % 5 as c026, rand64( 27) % 5 as c027, rand64( 28) % 5 as c028, rand64( 29) % 5 as c029, rand64( 30) % 5 as c030, + rand64( 31) % 5 as c031, rand64( 32) % 5 as c032, rand64( 33) % 5 as c033, rand64( 34) % 5 as c034, rand64( 35) % 5 as c035, + rand64( 36) % 5 as c036, rand64( 37) % 5 as c037, rand64( 38) % 5 as c038, rand64( 39) % 5 as c039, rand64( 40) % 5 as c040, + rand64( 41) % 5 as c041, rand64( 42) % 5 as c042, rand64( 43) % 5 as c043, rand64( 44) % 5 as c044, rand64( 45) % 5 as c045, + rand64( 46) % 5 as c046, rand64( 47) % 5 as c047, rand64( 48) % 5 as c048, rand64( 49) % 5 as c049, rand64( 50) % 5 as c050, + rand64( 51) % 5 as c051, rand64( 52) % 5 as c052, rand64( 53) % 5 as c053, rand64( 54) % 5 as c054, rand64( 55) % 5 as c055, + rand64( 56) % 5 as c056, rand64( 57) % 5 as c057, rand64( 58) % 5 as c058, rand64( 59) % 5 as c059, rand64( 60) % 5 as c060, + rand64( 61) % 5 as c061, rand64( 62) % 5 as c062, rand64( 63) % 5 as c063, rand64( 64) % 5 as c064, rand64( 65) % 5 as c065, + rand64( 66) % 5 as c066, rand64( 67) % 5 as c067, rand64( 68) % 5 as c068, rand64( 69) % 5 as c069, rand64( 70) % 5 as c070, + rand64( 71) % 5 as c071, rand64( 72) % 5 as c072, rand64( 73) % 5 as c073, rand64( 74) % 5 as c074, rand64( 75) % 5 as c075, + rand64( 76) % 5 as c076, rand64( 77) % 5 as c077, rand64( 78) % 5 as c078, rand64( 79) % 5 as c079, rand64( 80) % 5 as c080, + rand64( 81) % 5 as c081, rand64( 82) % 5 as c082, rand64( 83) % 5 as c083, rand64( 84) % 5 as c084, rand64( 85) % 5 as c085, + rand64( 86) % 5 as c086, rand64( 87) % 5 as c087, rand64( 88) % 5 as c088, rand64( 89) % 5 as c089, rand64( 90) % 5 as c090, + rand64( 91) % 5 as c091, rand64( 92) % 5 as c092, rand64( 93) % 5 as c093, rand64( 94) % 5 as c094, rand64( 95) % 5 as c095, + rand64( 96) % 5 as c096, rand64( 97) % 5 as c097, rand64( 98) % 5 as c098, rand64( 99) % 5 as c099, rand64(100) % 5 as c100, + rand64(101) % 5 as c101, rand64(102) % 5 as c102, rand64(103) % 5 as c103, rand64(104) % 5 as c104, rand64(105) % 5 as c105, + rand64(106) % 5 as c106, rand64(107) % 5 as c107, rand64(108) % 5 as c108, rand64(109) % 5 as c109, rand64(110) % 5 as c110, + rand64(111) % 5 as c111, rand64(112) % 5 as c112, rand64(113) % 5 as c113, rand64(114) % 5 as c114, rand64(115) % 5 as c115, + rand64(116) % 5 as c116, rand64(117) % 5 as c117, rand64(118) % 5 as c118, rand64(119) % 5 as c119, rand64(120) % 5 as c120, + rand64(121) % 5 as c121, rand64(122) % 5 as c122, rand64(123) % 5 as c123, rand64(124) % 5 as c124, rand64(125) % 5 as c125, + rand64(126) % 5 as c126, rand64(127) % 5 as c127, rand64(128) % 5 as c128, rand64(129) % 5 as c129, rand64(130) % 5 as c130, + rand64(131) % 5 as c131, rand64(132) % 5 as c132, rand64(133) % 5 as c133, rand64(134) % 5 as c134, rand64(135) % 5 as c135, + rand64(136) % 5 as c136, rand64(137) % 5 as c137, rand64(138) % 5 as c138, rand64(139) % 5 as c139, rand64(140) % 5 as c140, + rand64(141) % 5 as c141, rand64(142) % 5 as c142, rand64(143) % 5 as c143, rand64(144) % 5 as c144, rand64(145) % 5 as c145, + rand64(146) % 5 as c146, rand64(147) % 5 as c147, rand64(148) % 5 as c148, rand64(149) % 5 as c149, rand64(150) % 5 as c150, + rand64(151) % 5 as c151, rand64(152) % 5 as c152, rand64(153) % 5 as c153, rand64(154) % 5 as c154, rand64(155) % 5 as c155, + rand64(156) % 5 as c156, rand64(157) % 5 as c157, rand64(158) % 5 as c158, rand64(159) % 5 as c159, rand64(160) % 5 as c160, + rand64(161) % 5 as c161, rand64(162) % 5 as c162, rand64(163) % 5 as c163, rand64(164) % 5 as c164, rand64(165) % 5 as c165, + rand64(166) % 5 as c166, rand64(167) % 5 as c167, rand64(168) % 5 as c168, rand64(169) % 5 as c169, rand64(170) % 5 as c170, + rand64(171) % 5 as c171, rand64(172) % 5 as c172, rand64(173) % 5 as c173, rand64(174) % 5 as c174, rand64(175) % 5 as c175, + rand64(176) % 5 as c176, rand64(177) % 5 as c177, rand64(178) % 5 as c178, rand64(179) % 5 as c179, rand64(180) % 5 as c180, + rand64(181) % 5 as c181, rand64(182) % 5 as c182, rand64(183) % 5 as c183, rand64(184) % 5 as c184, rand64(185) % 5 as c185, + rand64(186) % 5 as c186, rand64(187) % 5 as c187, rand64(188) % 5 as c188, rand64(189) % 5 as c189, rand64(190) % 5 as c190, + rand64(191) % 5 as c191, rand64(192) % 5 as c192, rand64(193) % 5 as c193, rand64(194) % 5 as c194, rand64(195) % 5 as c195, + rand64(196) % 5 as c196, rand64(197) % 5 as c197, rand64(198) % 5 as c198, rand64(199) % 5 as c199, rand64(200) % 5 as c200, + rand64(201) % 5 as c201, rand64(202) % 5 as c202, rand64(203) % 5 as c203, rand64(204) % 5 as c204, rand64(205) % 5 as c205, + rand64(206) % 5 as c206, rand64(207) % 5 as c207, rand64(208) % 5 as c208, rand64(209) % 5 as c209, rand64(210) % 5 as c210, + rand64(211) % 5 as c211, rand64(212) % 5 as c212, rand64(213) % 5 as c213, rand64(214) % 5 as c214, rand64(215) % 5 as c215, + rand64(216) % 5 as c216, rand64(217) % 5 as c217, rand64(218) % 5 as c218, rand64(219) % 5 as c219, rand64(220) % 5 as c220, + rand64(221) % 5 as c221, rand64(222) % 5 as c222, rand64(223) % 5 as c223, rand64(224) % 5 as c224, rand64(225) % 5 as c225, + rand64(226) % 5 as c226, rand64(227) % 5 as c227, rand64(228) % 5 as c228, rand64(229) % 5 as c229, rand64(230) % 5 as c230, + rand64(231) % 5 as c231, rand64(232) % 5 as c232, rand64(233) % 5 as c233, rand64(234) % 5 as c234, rand64(235) % 5 as c235, + rand64(236) % 5 as c236, rand64(237) % 5 as c237, rand64(238) % 5 as c238, rand64(239) % 5 as c239, rand64(240) % 5 as c240, + rand64(241) % 5 as c241, rand64(242) % 5 as c242, rand64(243) % 5 as c243, rand64(244) % 5 as c244, rand64(245) % 5 as c245, + rand64(246) % 5 as c246, rand64(247) % 5 as c247, rand64(248) % 5 as c248, rand64(249) % 5 as c249, rand64(250) % 5 as c250, + rand64(251) % 5 as c251, rand64(252) % 5 as c252, rand64(253) % 5 as c253, rand64(254) % 5 as c254, rand64(255) % 5 as c255, + rand64(256) % 5 as c256, rand64(257) % 5 as c257, rand64(258) % 5 as c258, rand64(259) % 5 as c259, rand64(260) % 5 as c260, + rand64(261) % 5 as c261, rand64(262) % 5 as c262, rand64(263) % 5 as c263, rand64(264) % 5 as c264, rand64(265) % 5 as c265, + rand64(266) % 5 as c266, rand64(267) % 5 as c267, rand64(268) % 5 as c268, rand64(269) % 5 as c269, rand64(270) % 5 as c270, + rand64(271) % 5 as c271, rand64(272) % 5 as c272, rand64(273) % 5 as c273, rand64(274) % 5 as c274, rand64(275) % 5 as c275, + rand64(276) % 5 as c276, rand64(277) % 5 as c277, rand64(278) % 5 as c278, rand64(279) % 5 as c279, rand64(280) % 5 as c280, + rand64(281) % 5 as c281, rand64(282) % 5 as c282, rand64(283) % 5 as c283, rand64(284) % 5 as c284, rand64(285) % 5 as c285, + rand64(286) % 5 as c286, rand64(287) % 5 as c287, rand64(288) % 5 as c288, rand64(289) % 5 as c289, rand64(290) % 5 as c290, + rand64(291) % 5 as c291, rand64(292) % 5 as c292, rand64(293) % 5 as c293, rand64(294) % 5 as c294, rand64(295) % 5 as c295, + rand64(296) % 5 as c296, rand64(297) % 5 as c297, rand64(298) % 5 as c298, rand64(299) % 5 as c299, rand64(300) % 5 as c300, + rand64(301) % 5 as c301, rand64(302) % 5 as c302, rand64(303) % 5 as c303, rand64(304) % 5 as c304, rand64(305) % 5 as c305, + rand64(306) % 5 as c306, rand64(307) % 5 as c307, rand64(308) % 5 as c308, rand64(309) % 5 as c309, rand64(310) % 5 as c310, + rand64(311) % 5 as c311, rand64(312) % 5 as c312, rand64(313) % 5 as c313, rand64(314) % 5 as c314, rand64(315) % 5 as c315, + rand64(316) % 5 as c316, rand64(317) % 5 as c317, rand64(318) % 5 as c318, rand64(319) % 5 as c319, rand64(320) % 5 as c320, + rand64(321) % 5 as c321, rand64(322) % 5 as c322, rand64(323) % 5 as c323, rand64(324) % 5 as c324, rand64(325) % 5 as c325, + rand64(326) % 5 as c326, rand64(327) % 5 as c327, rand64(328) % 5 as c328, rand64(329) % 5 as c329, rand64(330) % 5 as c330, + rand64(331) % 5 as c331, rand64(332) % 5 as c332, rand64(333) % 5 as c333, rand64(334) % 5 as c334, rand64(335) % 5 as c335, + rand64(336) % 5 as c336, rand64(337) % 5 as c337, rand64(338) % 5 as c338, rand64(339) % 5 as c339, rand64(340) % 5 as c340, + rand64(341) % 5 as c341, rand64(342) % 5 as c342, rand64(343) % 5 as c343, rand64(344) % 5 as c344, rand64(345) % 5 as c345, + rand64(346) % 5 as c346, rand64(347) % 5 as c347, rand64(348) % 5 as c348, rand64(349) % 5 as c349, rand64(350) % 5 as c350, + rand64(351) % 5 as c351, rand64(352) % 5 as c352, rand64(353) % 5 as c353, rand64(354) % 5 as c354, rand64(355) % 5 as c355, + rand64(356) % 5 as c356, rand64(357) % 5 as c357, rand64(358) % 5 as c358, rand64(359) % 5 as c359, rand64(360) % 5 as c360, + rand64(361) % 5 as c361, rand64(362) % 5 as c362, rand64(363) % 5 as c363, rand64(364) % 5 as c364, rand64(365) % 5 as c365, + rand64(366) % 5 as c366, rand64(367) % 5 as c367, rand64(368) % 5 as c368, rand64(369) % 5 as c369, rand64(370) % 5 as c370, + rand64(371) % 5 as c371, rand64(372) % 5 as c372, rand64(373) % 5 as c373, rand64(374) % 5 as c374, rand64(375) % 5 as c375, + rand64(376) % 5 as c376, rand64(377) % 5 as c377, rand64(378) % 5 as c378, rand64(379) % 5 as c379, rand64(380) % 5 as c380, + rand64(381) % 5 as c381, rand64(382) % 5 as c382, rand64(383) % 5 as c383, rand64(384) % 5 as c384, rand64(385) % 5 as c385, + rand64(386) % 5 as c386, rand64(387) % 5 as c387, rand64(388) % 5 as c388, rand64(389) % 5 as c389, rand64(390) % 5 as c390, + rand64(391) % 5 as c391, rand64(392) % 5 as c392, rand64(393) % 5 as c393, rand64(394) % 5 as c394, rand64(395) % 5 as c395, + rand64(396) % 5 as c396, rand64(397) % 5 as c397, rand64(398) % 5 as c398, rand64(399) % 5 as c399, rand64(400) % 5 as c400, + rand64(401) % 5 as c401, rand64(402) % 5 as c402, rand64(403) % 5 as c403, rand64(404) % 5 as c404, rand64(405) % 5 as c405, + rand64(406) % 5 as c406, rand64(407) % 5 as c407, rand64(408) % 5 as c408, rand64(409) % 5 as c409, rand64(410) % 5 as c410, + rand64(411) % 5 as c411, rand64(412) % 5 as c412, rand64(413) % 5 as c413, rand64(414) % 5 as c414, rand64(415) % 5 as c415, + rand64(416) % 5 as c416, rand64(417) % 5 as c417, rand64(418) % 5 as c418, rand64(419) % 5 as c419, rand64(420) % 5 as c420, + rand64(421) % 5 as c421, rand64(422) % 5 as c422, rand64(423) % 5 as c423, rand64(424) % 5 as c424, rand64(425) % 5 as c425, + rand64(426) % 5 as c426, rand64(427) % 5 as c427, rand64(428) % 5 as c428, rand64(429) % 5 as c429, rand64(430) % 5 as c430, + rand64(431) % 5 as c431, rand64(432) % 5 as c432, rand64(433) % 5 as c433, rand64(434) % 5 as c434, rand64(435) % 5 as c435, + rand64(436) % 5 as c436, rand64(437) % 5 as c437, rand64(438) % 5 as c438, rand64(439) % 5 as c439, rand64(440) % 5 as c440, + rand64(441) % 5 as c441, rand64(442) % 5 as c442, rand64(443) % 5 as c443, rand64(444) % 5 as c444, rand64(445) % 5 as c445, + rand64(446) % 5 as c446, rand64(447) % 5 as c447, rand64(448) % 5 as c448, rand64(449) % 5 as c449, rand64(450) % 5 as c450, + rand64(451) % 5 as c451, rand64(452) % 5 as c452, rand64(453) % 5 as c453, rand64(454) % 5 as c454, rand64(455) % 5 as c455, + rand64(456) % 5 as c456, rand64(457) % 5 as c457, rand64(458) % 5 as c458, rand64(459) % 5 as c459, rand64(460) % 5 as c460, + rand64(461) % 5 as c461, rand64(462) % 5 as c462, rand64(463) % 5 as c463, rand64(464) % 5 as c464, rand64(465) % 5 as c465, + rand64(466) % 5 as c466, rand64(467) % 5 as c467, rand64(468) % 5 as c468, rand64(469) % 5 as c469, rand64(470) % 5 as c470, + rand64(471) % 5 as c471, rand64(472) % 5 as c472, rand64(473) % 5 as c473, rand64(474) % 5 as c474, rand64(475) % 5 as c475, + rand64(476) % 5 as c476, rand64(477) % 5 as c477, rand64(478) % 5 as c478, rand64(479) % 5 as c479, rand64(480) % 5 as c480, + rand64(481) % 5 as c481, rand64(482) % 5 as c482, rand64(483) % 5 as c483, rand64(484) % 5 as c484, rand64(485) % 5 as c485, + rand64(486) % 5 as c486, rand64(487) % 5 as c487, rand64(488) % 5 as c488, rand64(489) % 5 as c489, rand64(490) % 5 as c490, + rand64(491) % 5 as c491, rand64(492) % 5 as c492, rand64(493) % 5 as c493, rand64(494) % 5 as c494, rand64(495) % 5 as c495, + rand64(496) % 5 as c496, rand64(497) % 5 as c497, rand64(498) % 5 as c498, rand64(499) % 5 as c499, rand64(500) % 5 as c500, + rand64(501) % 5 as c501, rand64(502) % 5 as c502, rand64(503) % 5 as c503, rand64(504) % 5 as c504, rand64(505) % 5 as c505, + rand64(506) % 5 as c506, rand64(507) % 5 as c507, rand64(508) % 5 as c508, rand64(509) % 5 as c509, rand64(510) % 5 as c510, + rand64(511) % 5 as c511, rand64(512) % 5 as c512, rand64(513) % 5 as c513, rand64(514) % 5 as c514, rand64(515) % 5 as c515, + rand64(516) % 5 as c516, rand64(517) % 5 as c517, rand64(518) % 5 as c518, rand64(519) % 5 as c519, rand64(520) % 5 as c520, + rand64(521) % 5 as c521, rand64(522) % 5 as c522, rand64(523) % 5 as c523, rand64(524) % 5 as c524, rand64(525) % 5 as c525, + rand64(526) % 5 as c526, rand64(527) % 5 as c527, rand64(528) % 5 as c528, rand64(529) % 5 as c529, rand64(530) % 5 as c530, + rand64(531) % 5 as c531, rand64(532) % 5 as c532, rand64(533) % 5 as c533, rand64(534) % 5 as c534, rand64(535) % 5 as c535, + rand64(536) % 5 as c536, rand64(537) % 5 as c537, rand64(538) % 5 as c538, rand64(539) % 5 as c539, rand64(540) % 5 as c540, + rand64(541) % 5 as c541, rand64(542) % 5 as c542, rand64(543) % 5 as c543, rand64(544) % 5 as c544, rand64(545) % 5 as c545, + rand64(546) % 5 as c546, rand64(547) % 5 as c547, rand64(548) % 5 as c548, rand64(549) % 5 as c549, rand64(550) % 5 as c550, + rand64(551) % 5 as c551, rand64(552) % 5 as c552, rand64(553) % 5 as c553, rand64(554) % 5 as c554, rand64(555) % 5 as c555, + rand64(556) % 5 as c556, rand64(557) % 5 as c557, rand64(558) % 5 as c558, rand64(559) % 5 as c559, rand64(560) % 5 as c560, + rand64(561) % 5 as c561, rand64(562) % 5 as c562, rand64(563) % 5 as c563, rand64(564) % 5 as c564, rand64(565) % 5 as c565, + rand64(566) % 5 as c566, rand64(567) % 5 as c567, rand64(568) % 5 as c568, rand64(569) % 5 as c569, rand64(570) % 5 as c570, + rand64(571) % 5 as c571, rand64(572) % 5 as c572, rand64(573) % 5 as c573, rand64(574) % 5 as c574, rand64(575) % 5 as c575, + rand64(576) % 5 as c576, rand64(577) % 5 as c577, rand64(578) % 5 as c578, rand64(579) % 5 as c579, rand64(580) % 5 as c580, + rand64(581) % 5 as c581, rand64(582) % 5 as c582, rand64(583) % 5 as c583, rand64(584) % 5 as c584, rand64(585) % 5 as c585, + rand64(586) % 5 as c586, rand64(587) % 5 as c587, rand64(588) % 5 as c588, rand64(589) % 5 as c589, rand64(590) % 5 as c590, + rand64(591) % 5 as c591, rand64(592) % 5 as c592, rand64(593) % 5 as c593, rand64(594) % 5 as c594, rand64(595) % 5 as c595, + rand64(596) % 5 as c596, rand64(597) % 5 as c597, rand64(598) % 5 as c598, rand64(599) % 5 as c599, rand64(600) % 5 as c600, + rand64(601) % 5 as c601, rand64(602) % 5 as c602, rand64(603) % 5 as c603, rand64(604) % 5 as c604, rand64(605) % 5 as c605, + rand64(606) % 5 as c606, rand64(607) % 5 as c607, rand64(608) % 5 as c608, rand64(609) % 5 as c609, rand64(610) % 5 as c610, + rand64(611) % 5 as c611, rand64(612) % 5 as c612, rand64(613) % 5 as c613, rand64(614) % 5 as c614, rand64(615) % 5 as c615, + rand64(616) % 5 as c616, rand64(617) % 5 as c617, rand64(618) % 5 as c618, rand64(619) % 5 as c619, rand64(620) % 5 as c620, + rand64(621) % 5 as c621, rand64(622) % 5 as c622, rand64(623) % 5 as c623, rand64(624) % 5 as c624, rand64(625) % 5 as c625, + rand64(626) % 5 as c626, rand64(627) % 5 as c627, rand64(628) % 5 as c628, rand64(629) % 5 as c629, rand64(630) % 5 as c630, + rand64(631) % 5 as c631, rand64(632) % 5 as c632, rand64(633) % 5 as c633, rand64(634) % 5 as c634, rand64(635) % 5 as c635, + rand64(636) % 5 as c636, rand64(637) % 5 as c637, rand64(638) % 5 as c638, rand64(639) % 5 as c639, rand64(640) % 5 as c640, + rand64(641) % 5 as c641, rand64(642) % 5 as c642, rand64(643) % 5 as c643, rand64(644) % 5 as c644, rand64(645) % 5 as c645, + rand64(646) % 5 as c646, rand64(647) % 5 as c647, rand64(648) % 5 as c648, rand64(649) % 5 as c649, rand64(650) % 5 as c650, + rand64(651) % 5 as c651, rand64(652) % 5 as c652, rand64(653) % 5 as c653, rand64(654) % 5 as c654, rand64(655) % 5 as c655, + rand64(656) % 5 as c656, rand64(657) % 5 as c657, rand64(658) % 5 as c658, rand64(659) % 5 as c659, rand64(660) % 5 as c660, + rand64(661) % 5 as c661, rand64(662) % 5 as c662, rand64(663) % 5 as c663, rand64(664) % 5 as c664, rand64(665) % 5 as c665, + rand64(666) % 5 as c666, rand64(667) % 5 as c667, rand64(668) % 5 as c668, rand64(669) % 5 as c669, rand64(670) % 5 as c670, + rand64(671) % 5 as c671, rand64(672) % 5 as c672, rand64(673) % 5 as c673, rand64(674) % 5 as c674, rand64(675) % 5 as c675, + rand64(676) % 5 as c676, rand64(677) % 5 as c677, rand64(678) % 5 as c678, rand64(679) % 5 as c679, rand64(680) % 5 as c680, + rand64(681) % 5 as c681, rand64(682) % 5 as c682, rand64(683) % 5 as c683, rand64(684) % 5 as c684, rand64(685) % 5 as c685, + rand64(686) % 5 as c686, rand64(687) % 5 as c687, rand64(688) % 5 as c688, rand64(689) % 5 as c689, rand64(690) % 5 as c690, + rand64(691) % 5 as c691, rand64(692) % 5 as c692, rand64(693) % 5 as c693, rand64(694) % 5 as c694, rand64(695) % 5 as c695, + rand64(696) % 5 as c696, rand64(697) % 5 as c697, rand64(698) % 5 as c698, rand64(699) % 5 as c699, rand64(700) % 5 as c700, + rand64(701) % 5 as c701 + FROM system.numbers + LIMIT 1048576 + + + + 10]]> + + 10]]> + + 10]]> + + DROP TABLE IF EXISTS huge_pk + \ No newline at end of file From 38a73fb5514f93ffc33a3dd67e98b3b0521f44df Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 18 Sep 2019 20:04:10 +0300 Subject: [PATCH 047/108] add simple perf test for partial merge join --- .../tests/performance/joins_in_memory_pmj.xml | 39 +++++++++++++++++++ 1 file changed, 39 insertions(+) create mode 100644 dbms/tests/performance/joins_in_memory_pmj.xml diff --git a/dbms/tests/performance/joins_in_memory_pmj.xml b/dbms/tests/performance/joins_in_memory_pmj.xml new file mode 100644 index 00000000000..3908df8c978 --- /dev/null +++ b/dbms/tests/performance/joins_in_memory_pmj.xml @@ -0,0 +1,39 @@ + + loop + + + + 10 + + + + + + + + CREATE TABLE ints (i64 Int64, i32 Int32, i16 Int16, i8 Int8) ENGINE = Memory + SET partial_merge_join = 1 + + INSERT INTO ints SELECT number AS i64, i64 AS i32, i64 AS i16, i64 AS i8 FROM numbers(5000) + INSERT INTO ints SELECT 10000 + number % 1000 AS i64, i64 AS i32, i64 AS i16, i64 AS i8 FROM numbers(5000) + INSERT INTO ints SELECT 20000 + number % 100 AS i64, i64 AS i32, i64 AS i16, i64 AS i8 FROM numbers(5000) + INSERT INTO ints SELECT 30000 + number % 10 AS i64, i64 AS i32, i64 AS i16, i64 AS i8 FROM numbers(5000) + INSERT INTO ints SELECT 40000 + number % 1 AS i64, i64 AS i32, i64 AS i16, i64 AS i8 FROM numbers(5000) + + SELECT COUNT() FROM ints l ANY LEFT JOIN ints r USING i64 WHERE i32 = 200042 + SELECT COUNT() FROM ints l ANY LEFT JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 200042 + SELECT COUNT() FROM ints l ANY LEFT JOIN ints r ON l.i64 = r.i64 WHERE i32 = 200042 + SELECT COUNT() FROM ints l ANY LEFT JOIN ints r USING i64 WHERE i32 IN(42, 100042, 200042, 300042, 400042) + + SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 = 200042 + SELECT COUNT() FROM ints l INNER JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 200042 + SELECT COUNT() FROM ints l INNER JOIN ints r ON l.i64 = r.i64 WHERE i32 = 200042 + SELECT COUNT() FROM ints l INNER JOIN ints r USING i64 WHERE i32 IN(42, 100042, 200042, 300042, 400042) + + SELECT COUNT() FROM ints l LEFT JOIN ints r USING i64 WHERE i32 = 200042 + SELECT COUNT() FROM ints l LEFT JOIN ints r USING i64,i32,i16,i8 WHERE i32 = 200042 + SELECT COUNT() FROM ints l LEFT JOIN ints r ON l.i64 = r.i64 WHERE i32 = 200042 + SELECT COUNT() FROM ints l LEFT JOIN ints r USING i64 WHERE i32 IN(42, 100042, 200042, 300042, 400042) + + DROP TABLE IF EXISTS ints + From 08a28d2dc23ace952c19992bc1cf4367c2aef2f0 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 18 Sep 2019 21:44:44 +0300 Subject: [PATCH 048/108] fix pm join bug (wrong right equal range copy) --- dbms/src/Columns/ColumnNullable.cpp | 6 ++++++ dbms/src/Columns/ColumnNullable.h | 1 + dbms/src/Columns/IColumn.h | 14 ++++++++++++++ dbms/src/Interpreters/MergeJoin.cpp | 7 +++---- .../01010_pm_join_all_join_bug.reference | 9 +++++++++ .../0_stateless/01010_pm_join_all_join_bug.sql | 12 ++++++++++++ 6 files changed, 45 insertions(+), 4 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.reference create mode 100644 dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index e9655310452..4c1f46f771f 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -168,6 +168,12 @@ void ColumnNullable::insertRangeFromNotNullable(const IColumn & src, size_t star getNullMapData().resize_fill(getNullMapData().size() + length, 0); } +void ColumnNullable::insertManyFromNotNullable(const IColumn & src, size_t position, size_t length) +{ + for (size_t i = 0; i < length; ++i) + insertFromNotNullable(src, position); +} + void ColumnNullable::popBack(size_t n) { getNestedColumn().popBack(n); diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index 083d6f64052..452b77bb7e2 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -63,6 +63,7 @@ public: void insertFromNotNullable(const IColumn & src, size_t n); void insertRangeFromNotNullable(const IColumn & src, size_t start, size_t length); + void insertManyFromNotNullable(const IColumn & src, size_t position, size_t length); void insertDefault() override { diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index 3955ed4b528..1f9067c6c81 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -146,6 +146,13 @@ public: /// Could be used to concatenate columns. virtual void insertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; + /// Appends one element from other column with the same type multiple times. + virtual void insertManyFrom(const IColumn & src, size_t position, size_t length) + { + for (size_t i = 0; i < length; ++i) + insertFrom(src, position); + } + /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). /// Is used to optimize some computations (in aggregation, for example). /// Parameter length could be ignored if column values have fixed size. @@ -157,6 +164,13 @@ public: /// For example, ColumnNullable(Nested) absolutely ignores values of nested column if it is marked as NULL. virtual void insertDefault() = 0; + /// Appends "default value" multiple times. + virtual void insertManyDefaults(size_t length) + { + for (size_t i = 0; i < length; ++i) + insertDefault(); + } + /** Removes last n elements. * Is used to support exception-safety of several operations. * For example, sometimes insertion should be reverted if we catch an exception during operation processing. diff --git a/dbms/src/Interpreters/MergeJoin.cpp b/dbms/src/Interpreters/MergeJoin.cpp index 9cc02d9be7a..584562fe363 100644 --- a/dbms/src/Interpreters/MergeJoin.cpp +++ b/dbms/src/Interpreters/MergeJoin.cpp @@ -151,9 +151,9 @@ void copyRightRange(const Block & right_block, const Block & right_columns_to_ad auto * dst_nullable = typeid_cast(dst_column.get()); if (dst_nullable && !isColumnNullable(*src_column)) - dst_nullable->insertRangeFromNotNullable(*src_column, row_position, rows_to_add); + dst_nullable->insertManyFromNotNullable(*src_column, row_position, rows_to_add); else - dst_column->insertRangeFrom(*src_column, row_position, rows_to_add); + dst_column->insertManyFrom(*src_column, row_position, rows_to_add); } } @@ -179,8 +179,7 @@ void joinEquals(const Block & left_block, const Block & right_block, const Block void appendNulls(MutableColumns & right_columns, size_t rows_to_add) { for (auto & column : right_columns) - for (size_t i = 0; i < rows_to_add; ++i) - column->insertDefault(); + column->insertManyDefaults(rows_to_add); } void joinInequalsLeft(const Block & left_block, MutableColumns & left_columns, MutableColumns & right_columns, diff --git a/dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.reference b/dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.reference new file mode 100644 index 00000000000..5f51e52afa8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.reference @@ -0,0 +1,9 @@ +1 0 0 +1 0 1 +1 1 0 +1 1 1 +- +1 0 0 +1 0 1 +1 1 0 +1 1 1 diff --git a/dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql b/dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql new file mode 100644 index 00000000000..0a00628d3b5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01010_pm_join_all_join_bug.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS ints; +CREATE TABLE ints (i64 Int64, i32 Int32) ENGINE = Memory; + +SET partial_merge_join = 1; + +INSERT INTO ints SELECT 1 AS i64, number AS i32 FROM numbers(2); + +SELECT * FROM ints l LEFT JOIN ints r USING i64 ORDER BY l.i32, r.i32; +SELECT '-'; +SELECT * FROM ints l INNER JOIN ints r USING i64 ORDER BY l.i32, r.i32; + +DROP TABLE ints; From b8b6197288fe9b8196e711ae7e51b0e183e37764 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 18 Sep 2019 22:12:53 +0300 Subject: [PATCH 049/108] partial_merge_join settings affects LEFT and INNER JOINs only --- dbms/src/Core/Settings.h | 2 +- dbms/src/Interpreters/AnalyzedJoin.cpp | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index d1f5994df16..cacaf883fb7 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -288,7 +288,7 @@ struct Settings : public SettingsCollection M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).") \ M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.") \ M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.") \ - M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join if possible.") \ + M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.") \ \ M(SettingUInt64, max_rows_to_transfer, 0, "Maximum size (in rows) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ M(SettingUInt64, max_bytes_to_transfer, 0, "Maximum size (in uncompressed bytes) of the transmitted external table obtained when the GLOBAL IN/JOIN section is executed.") \ diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 5da4a7219d0..53f763d54dd 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -262,7 +262,10 @@ NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpressio JoinPtr makeJoin(std::shared_ptr table_join, const Block & right_sample_block) { - if (table_join->partial_merge_join) + bool is_left_or_inner = isLeft(table_join->kind()) || isInner(table_join->kind()); + bool is_asof = (table_join->strictness() == ASTTableJoin::Strictness::Asof); + + if (table_join->partial_merge_join && !is_asof && is_left_or_inner) return std::make_shared(table_join, right_sample_block); return std::make_shared(table_join, right_sample_block); } From db470be07706157056fc3e3d05e013cd49d39423 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 18 Sep 2019 23:07:23 +0300 Subject: [PATCH 050/108] Update merge_tree_huge_pk.xml --- dbms/tests/performance/merge_tree_huge_pk.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/performance/merge_tree_huge_pk.xml b/dbms/tests/performance/merge_tree_huge_pk.xml index 351cc17cb6c..e39ff7501f7 100644 --- a/dbms/tests/performance/merge_tree_huge_pk.xml +++ b/dbms/tests/performance/merge_tree_huge_pk.xml @@ -203,4 +203,4 @@ 10]]> DROP TABLE IF EXISTS huge_pk - \ No newline at end of file + From 0dc3866c36d2306a53acb9286916c422ea2c8c22 Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Tue, 17 Sep 2019 14:34:08 +0800 Subject: [PATCH 051/108] added bitmapSubsetLimit --- .../AggregateFunctionGroupBitmapData.h | 46 +++++++++++++++++-- dbms/src/Functions/FunctionsBitmap.cpp | 1 + dbms/src/Functions/FunctionsBitmap.h | 37 +++++++++++++-- .../00829_bitmap_function.reference | 8 ++++ .../0_stateless/00829_bitmap_function.sql | 19 ++++++++ .../functions/bitmap_functions.md | 26 +++++++++++ .../functions/bitmap_functions.md | 26 +++++++++++ 7 files changed, 154 insertions(+), 9 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index 220493a918c..e1fbd092490 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -467,11 +467,10 @@ public: return count; if (isSmall()) { - std::vector ans; for (const auto & x : small) { T val = x.getValue(); - if ((UInt32)val >= range_start && (UInt32)val < range_end) + if (UInt32(val) >= range_start && UInt32(val) < range_end) { r1.add(val); count++; @@ -483,10 +482,47 @@ public: roaring_uint32_iterator_t iterator; roaring_init_iterator(rb, &iterator); roaring_move_uint32_iterator_equalorlarger(&iterator, range_start); - while (iterator.has_value) + while (iterator.has_value && UInt32(iterator.current_value) < range_end) + { + r1.add(iterator.current_value); + roaring_advance_uint32_iterator(&iterator); + count++; + } + } + return count; + } + + /** + * Return new set of the smallest `limit` values in set which is no less than `range_start`. + */ + UInt64 rb_limit(UInt32 range_start, UInt32 limit, RoaringBitmapWithSmallSet& r1) const + { + UInt64 count = 0; + if (isSmall()) + { + std::vector ans; + for (const auto & x : small) + { + T val = x.getValue(); + if (UInt32(val) >= range_start) + { + ans.push_back(val); + } + } + sort(ans.begin(), ans.end()); + if (limit > ans.size()) + limit = ans.size(); + for (size_t i=0; i= range_end) - break; r1.add(iterator.current_value); roaring_advance_uint32_iterator(&iterator); count++; diff --git a/dbms/src/Functions/FunctionsBitmap.cpp b/dbms/src/Functions/FunctionsBitmap.cpp index b24e9cdbd9f..62faf49d2b3 100644 --- a/dbms/src/Functions/FunctionsBitmap.cpp +++ b/dbms/src/Functions/FunctionsBitmap.cpp @@ -10,6 +10,7 @@ void registerFunctionsBitmap(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/FunctionsBitmap.h b/dbms/src/Functions/FunctionsBitmap.h index ad4f16b16ef..740dc5bafe4 100644 --- a/dbms/src/Functions/FunctionsBitmap.h +++ b/dbms/src/Functions/FunctionsBitmap.h @@ -34,6 +34,9 @@ namespace ErrorCodes * Return subset in specified range (not include the range_end): * bitmapSubsetInRange: bitmap,integer,integer -> bitmap * + * Return subset of the smallest `limit` values in set which is no smaller than `range_start`. + * bitmapSubsetInRange: bitmap,integer,integer -> bitmap + * * Two bitmap and calculation: * bitmapAnd: bitmap,bitmap -> bitmap * @@ -250,12 +253,13 @@ private: } }; -class FunctionBitmapSubsetInRange : public IFunction +template +class FunctionBitmapSubset : public IFunction { public: - static constexpr auto name = "bitmapSubsetInRange"; + static constexpr auto name = Impl::name; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context &) { return std::make_shared>(); } String getName() const override { return name; } @@ -357,12 +361,37 @@ private: col_to->insertDefault(); AggregateFunctionGroupBitmapData & bd2 = *reinterpret_cast *>(col_to->getData()[i]); - bd0.rbs.rb_range(range_start, range_end, bd2.rbs); + Impl::apply(bd0, range_start, range_end, bd2); } block.getByPosition(result).column = std::move(col_to); } }; +struct BitmapSubsetInRangeImpl +{ +public: + static constexpr auto name = "bitmapSubsetInRange"; + template + static void apply(const AggregateFunctionGroupBitmapData & bd0, UInt32 range_start, UInt32 range_end, AggregateFunctionGroupBitmapData & bd2) + { + bd0.rbs.rb_range(range_start, range_end, bd2.rbs); + } +}; + +struct BitmapSubsetLimitImpl +{ +public: + static constexpr auto name = "bitmapSubsetLimit"; + template + static void apply(const AggregateFunctionGroupBitmapData & bd0, UInt32 range_start, UInt32 range_end, AggregateFunctionGroupBitmapData & bd2) + { + bd0.rbs.rb_limit(range_start, range_end, bd2.rbs); + } +}; + +using FunctionBitmapSubsetInRange = FunctionBitmapSubset; +using FunctionBitmapSubsetLimit = FunctionBitmapSubset; + template class FunctionBitmapSelfCardinalityImpl : public IFunction { diff --git a/dbms/tests/queries/0_stateless/00829_bitmap_function.reference b/dbms/tests/queries/0_stateless/00829_bitmap_function.reference index 3edcd0e1214..b5b6c1e0d49 100644 --- a/dbms/tests/queries/0_stateless/00829_bitmap_function.reference +++ b/dbms/tests/queries/0_stateless/00829_bitmap_function.reference @@ -67,6 +67,14 @@ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33] [30,31,32,33,100] [100] +[] +[] +[1,5,7,9] +[] +[5,7,9] +[0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500] +[30,31,32,33,100,200,500] +[100,200,500] 4294967295 4294967295 4294967295 diff --git a/dbms/tests/queries/0_stateless/00829_bitmap_function.sql b/dbms/tests/queries/0_stateless/00829_bitmap_function.sql index 82e1030c036..f12fe4c8218 100644 --- a/dbms/tests/queries/0_stateless/00829_bitmap_function.sql +++ b/dbms/tests/queries/0_stateless/00829_bitmap_function.sql @@ -212,6 +212,25 @@ select bitmapToArray(bitmapSubsetInRange(bitmapBuild([ 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, 100,200,500]), toUInt32(100), toUInt32(200))); +-- bitmapSubsetLimit: +---- Empty +SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild(emptyArrayUInt32()), toUInt32(0), toUInt32(10))); +SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild(emptyArrayUInt16()), toUInt32(0), toUInt32(10))); +---- Small +select bitmapToArray(bitmapSubsetLimit(bitmapBuild([1,5,7,9]), toUInt32(0), toUInt32(4))); +select bitmapToArray(bitmapSubsetLimit(bitmapBuild([1,5,7,9]), toUInt32(10), toUInt32(10))); +select bitmapToArray(bitmapSubsetLimit(bitmapBuild([1,5,7,9]), toUInt32(3), toUInt32(7))); +---- Large +select bitmapToArray(bitmapSubsetLimit(bitmapBuild([ + 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, + 100,200,500]), toUInt32(0), toUInt32(100))); +select bitmapToArray(bitmapSubsetLimit(bitmapBuild([ + 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, + 100,200,500]), toUInt32(30), toUInt32(200))); +select bitmapToArray(bitmapSubsetLimit(bitmapBuild([ + 0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33, + 100,200,500]), toUInt32(100), toUInt32(200))); + -- bitmapMin: ---- Empty SELECT bitmapMin(bitmapBuild(emptyArrayUInt8())); diff --git a/docs/en/query_language/functions/bitmap_functions.md b/docs/en/query_language/functions/bitmap_functions.md index fdc2e8a7a0d..3d328359c88 100644 --- a/docs/en/query_language/functions/bitmap_functions.md +++ b/docs/en/query_language/functions/bitmap_functions.md @@ -82,6 +82,32 @@ SELECT bitmapToArray(bitmapSubsetInRange(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11, └───────────────────┘ ``` +## bitmapSubsetLimit {#bitmap_functions-bitmapsubsetlimit} + +Return subset of the smallest `limit` values in set which is no less than `range_start`. + +``` +bitmapSubsetLimit(bitmap, range_start, limit) +``` + +**Parameters** + +- `bitmap` – [Bitmap object](#bitmap_functions-bitmapbuild). +- `range_start` – range start point. Type: [UInt32](../../data_types/int_uint.md). +- `limit` – subset cardinality upper limit. Type: [UInt32](../../data_types/int_uint.md). + +**Example** + +``` sql +SELECT bitmapToArray(bitmapSubsetLimit(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res +``` + +``` +┌─res───────────────────────┐ +│ [30,31,32,33,100,200,500] │ +└───────────────────────────┘ +``` + ## bitmapContains {#bitmap_functions-bitmapcontains} Checks whether the bitmap contains an element. diff --git a/docs/zh/query_language/functions/bitmap_functions.md b/docs/zh/query_language/functions/bitmap_functions.md index b727a4aba16..1fbfd3c8fe7 100644 --- a/docs/zh/query_language/functions/bitmap_functions.md +++ b/docs/zh/query_language/functions/bitmap_functions.md @@ -77,6 +77,32 @@ SELECT bitmapToArray(bitmapSubsetInRange(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11, └───────────────────┘ ``` +## bitmapSubsetLimit + +将位图指定范围(起始点和数目上限)转换为另一个位图。 + +``` +bitmapSubsetLimit(bitmap, range_start, limit) +``` + +**参数** + +- `bitmap` – 位图对象. +- `range_start` – 范围起始点(含). +- `limit` – 子位图基数上限. + +**示例** + +``` sql +SELECT bitmapToArray(bitmapSubsetInRange(bitmapBuild([0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19,20,21,22,23,24,25,26,27,28,29,30,31,32,33,100,200,500]), toUInt32(30), toUInt32(200))) AS res +``` + +``` +┌─res───────────────────────┐ +│ [30,31,32,33,100,200,500] │ +└───────────────────────────┘ +``` + ## bitmapContains 检查位图是否包含指定元素。 From bd1ce56797fb08b524a045b9d29452aa431a3821 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 19 Sep 2019 09:34:33 +0000 Subject: [PATCH 052/108] Tests decomposition. --- .../configs/min_chunk_size.xml | 3 + .../tests/integration/test_storage_s3/test.py | 255 ++++++++++-------- .../test_storage_s3/test_server.py | 121 ++++++++- 3 files changed, 259 insertions(+), 120 deletions(-) create mode 100644 dbms/tests/integration/test_storage_s3/configs/min_chunk_size.xml diff --git a/dbms/tests/integration/test_storage_s3/configs/min_chunk_size.xml b/dbms/tests/integration/test_storage_s3/configs/min_chunk_size.xml new file mode 100644 index 00000000000..f61fcd2c5c9 --- /dev/null +++ b/dbms/tests/integration/test_storage_s3/configs/min_chunk_size.xml @@ -0,0 +1,3 @@ + + 1000000 + diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index b975c4c92d5..2013daa6ae6 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -1,19 +1,3 @@ -import pytest - -from helpers.cluster import ClickHouseCluster - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster = ClickHouseCluster(__file__) - instance = cluster.add_instance('dummy') - cluster.start() - yield cluster - - finally: - cluster.shutdown() - - import httplib import json import logging @@ -21,115 +5,152 @@ import os import time import traceback +import pytest + +from helpers.cluster import ClickHouseCluster + logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) -def test_simple(started_cluster): - instance = started_cluster.instances['dummy'] - instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), 'test_server.py'), 'test_server.py') - communication_port = 10000 - bucket = 'abc' - instance.exec_in_container(['python', 'test_server.py', str(communication_port), bucket], detach=True) - def get_data(): - conn = httplib.HTTPConnection(started_cluster.instances['dummy'].ip_address, communication_port) - conn.request("GET", "/") - r = conn.getresponse() - raw_data = r.read() - conn.close() - return json.loads(raw_data) +def get_communication_data(started_cluster): + conn = httplib.HTTPConnection(started_cluster.instances['dummy'].ip_address, started_cluster.communication_port) + conn.request("GET", "/") + r = conn.getresponse() + raw_data = r.read() + conn.close() + return json.loads(raw_data) - format = 'column1 UInt32, column2 UInt32, column3 UInt32' - values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' - other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' - for i in range(10): - try: - data = get_data() - redirecting_to_http_port = data['redirecting_to_http_port'] - preserving_data_port = data['preserving_data_port'] - redirecting_preserving_data_port = data['redirecting_preserving_data_port'] - except: - logging.error(traceback.format_exc()) - time.sleep(0.5) - else: - break - else: - assert False, 'Could not initialize mock server' - mock_host = started_cluster.instances['dummy'].ip_address +def put_communication_data(started_cluster, body): + conn = httplib.HTTPConnection(started_cluster.instances['dummy'].ip_address, started_cluster.communication_port) + conn.request("PUT", "/", body) + r = conn.getresponse() + conn.close() - def run_query(query): - logging.info('Running query "{}"...'.format(query)) - result = instance.query(query) - logging.info('Query finished') - return result - - - prepare_put_queries = [ - "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(mock_host, preserving_data_port, bucket, format, values), - ] - - queries = [ - "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(mock_host, redirecting_to_http_port, format), - ] - - put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(mock_host, preserving_data_port, bucket, format, values) - - redirect_put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(mock_host, redirecting_preserving_data_port, bucket, format, other_values) - - check_queries = [ - "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(mock_host, preserving_data_port, bucket, format), - ] - + +@pytest.fixture(scope="module") +def started_cluster(): try: - logging.info('Phase 1') - for query in prepare_put_queries: - run_query(query) - - logging.info('Phase 2') - for query in queries: - stdout = run_query(query) - assert list(map(str.split, stdout.splitlines())) == [ - ['42', '87', '44', '160776'], - ['55', '33', '81', '147015'], - ['1', '0', '9', '0'], - ] - - logging.info('Phase 3') - query = put_query - run_query(query) - data = get_data() - received_data_completed = data['received_data_completed'] - received_data = data['received_data'] - finalize_data = data['finalize_data'] - finalize_data_query = data['finalize_data_query'] - assert received_data[-1].decode() == '1,2,3\n3,2,1\n78,43,45\n' - assert received_data_completed - assert finalize_data == '1hello-etag' - assert finalize_data_query == 'uploadId=TEST' - - logging.info('Phase 4') - query = redirect_put_query - run_query(query) - - for query in check_queries: - logging.info(query) - stdout = run_query(query) - assert list(map(str.split, stdout.splitlines())) == [ - ['1', '1', '1', '1'], - ['1', '1', '1', '1'], - ['11', '11', '11', '1331'], - ] - data = get_data() - received_data = data['received_data'] - assert received_data[-1].decode() == '1,1,1\n1,1,1\n11,11,11\n' + cluster = ClickHouseCluster(__file__) + instance = cluster.add_instance('dummy', config_dir="configs", main_configs=['configs/min_chunk_size.xml']) + cluster.start() - # FIXME tests for multipart - - except: - logging.error(traceback.format_exc()) - raise + cluster.communication_port = 10000 + instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), 'test_server.py'), 'test_server.py') + cluster.bucket = 'abc' + instance.exec_in_container(['python', 'test_server.py', str(cluster.communication_port), cluster.bucket], detach=True) + cluster.mock_host = instance.ip_address - else: - logging.info('Done') + for i in range(10): + try: + data = get_communication_data(cluster) + cluster.redirecting_to_http_port = data['redirecting_to_http_port'] + cluster.preserving_data_port = data['preserving_data_port'] + cluster.multipart_preserving_data_port = data['multipart_preserving_data_port'] + cluster.redirecting_preserving_data_port = data['redirecting_preserving_data_port'] + except: + logging.error(traceback.format_exc()) + time.sleep(0.5) + else: + break + else: + assert False, 'Could not initialize mock server' + + yield cluster + + finally: + cluster.shutdown() + + +def run_query(instance, query, stdin=None): + logging.info('Running query "{}"...'.format(query)) + result = instance.query(query, stdin=stdin) + logging.info('Query finished') + return result + +def test_get_with_redirect(started_cluster): + instance = started_cluster.instances['dummy'] + format = 'column1 UInt32, column2 UInt32, column3 UInt32' + + put_communication_data(started_cluster, '=== Get with redirect test ===') + query = "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.redirecting_to_http_port, format) + stdout = run_query(instance, query) + assert list(map(str.split, stdout.splitlines())) == [ + ['42', '87', '44', '160776'], + ['55', '33', '81', '147015'], + ['1', '0', '9', '0'], + ] + +def test_put(started_cluster): + instance = started_cluster.instances['dummy'] + format = 'column1 UInt32, column2 UInt32, column3 UInt32' + + logging.info('Phase 3') + put_communication_data(started_cluster, '=== Put test ===') + values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' + put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format, values) + run_query(instance, put_query) + data = get_communication_data(started_cluster) + received_data_completed = data['received_data_completed'] + received_data = data['received_data'] + finalize_data = data['finalize_data'] + finalize_data_query = data['finalize_data_query'] + assert received_data[-1].decode() == '1,2,3\n3,2,1\n78,43,45\n' + assert received_data_completed + assert finalize_data == '1hello-etag' + assert finalize_data_query == 'uploadId=TEST' + +def test_put_csv(started_cluster): + instance = started_cluster.instances['dummy'] + format = 'column1 UInt32, column2 UInt32, column3 UInt32' + + put_communication_data(started_cluster, '=== Put test CSV ===') + put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) + csv_data = '8,9,16\n11,18,13\n22,14,2\n' + run_query(instance, put_query, stdin=csv_data) + data = get_communication_data(started_cluster) + received_data_completed = data['received_data_completed'] + received_data = data['received_data'] + finalize_data = data['finalize_data'] + finalize_data_query = data['finalize_data_query'] + assert received_data[-1].decode() == csv_data + assert received_data_completed + assert finalize_data == '1hello-etag' + assert finalize_data_query == 'uploadId=TEST' + +def test_put_with_redirect(started_cluster): + instance = started_cluster.instances['dummy'] + format = 'column1 UInt32, column2 UInt32, column3 UInt32' + + put_communication_data(started_cluster, '=== Put with redirect test ===') + other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' + query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.redirecting_preserving_data_port, started_cluster.bucket, format, other_values) + run_query(instance, query) + + query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) + stdout = run_query(instance, query) + assert list(map(str.split, stdout.splitlines())) == [ + ['1', '1', '1', '1'], + ['1', '1', '1', '1'], + ['11', '11', '11', '1331'], + ] + data = get_communication_data(started_cluster) + received_data = data['received_data'] + assert received_data[-1].decode() == '1,1,1\n1,1,1\n11,11,11\n' + +def test_multipart_put(started_cluster): + instance = started_cluster.instances['dummy'] + format = 'column1 UInt32, column2 UInt32, column3 UInt32' + + put_communication_data(started_cluster, '=== Multipart test ===') + long_data = [[i, i+1, i+2] for i in range(100000)] + long_values = ''.join([ '{},{},{}\n'.format(x,y,z) for x, y, z in long_data ]) + put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.multipart_preserving_data_port, started_cluster.bucket, format) + run_query(instance, put_query, stdin=long_values) + data = get_communication_data(started_cluster) + assert 'multipart_received_data' in data + received_data = data['multipart_received_data'] + assert received_data[-1].decode() == ''.join([ '{},{},{}\n'.format(x, y, z) for x, y, z in long_data ]) + assert 1 < data['multipart_parts'] < 10000 diff --git a/dbms/tests/integration/test_storage_s3/test_server.py b/dbms/tests/integration/test_storage_s3/test_server.py index bc22b0df085..3c10445566a 100644 --- a/dbms/tests/integration/test_storage_s3/test_server.py +++ b/dbms/tests/integration/test_storage_s3/test_server.py @@ -20,6 +20,8 @@ import socket import sys import threading import time +import uuid +import xml.etree.ElementTree logging.getLogger().setLevel(logging.INFO) @@ -43,13 +45,20 @@ def GetFreeTCPPortsAndIP(n): [ s.close() for s in sockets ] return result, addr -(redirecting_to_http_port, simple_server_port, preserving_data_port, redirecting_preserving_data_port), localhost = GetFreeTCPPortsAndIP(4) +( + redirecting_to_http_port, + simple_server_port, + preserving_data_port, + multipart_preserving_data_port, + redirecting_preserving_data_port +), localhost = GetFreeTCPPortsAndIP(5) + data = { 'redirecting_to_http_port': redirecting_to_http_port, 'preserving_data_port': preserving_data_port, + 'multipart_preserving_data_port': multipart_preserving_data_port, 'redirecting_preserving_data_port': redirecting_preserving_data_port, } -redirecting_host = localhost class SimpleHTTPServerHandler(BaseHTTPRequestHandler): @@ -113,7 +122,7 @@ class PreservingDataHandler(BaseHTTPRequestHandler): def do_POST(self): self.send_response(200) query = urlparse.urlparse(self.path).query - logging.info('POST ' + query) + logging.info('PreservingDataHandler POST ?' + query) if query == 'uploads': post_data = r''' TEST'''.encode() @@ -161,6 +170,104 @@ class PreservingDataHandler(BaseHTTPRequestHandler): self.finish() +class MultipartPreservingDataHandler(BaseHTTPRequestHandler): + protocol_version = 'HTTP/1.1' + + def parse_request(self): + result = BaseHTTPRequestHandler.parse_request(self) + # Adaptation to Python 3. + if sys.version_info.major == 2 and result == True: + expect = self.headers.get('Expect', "") + if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): + if not self.handle_expect_100(): + return False + return result + + def send_response_only(self, code, message=None): + if message is None: + if code in self.responses: + message = self.responses[code][0] + else: + message = '' + if self.request_version != 'HTTP/0.9': + self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) + + def handle_expect_100(self): + logging.info('Received Expect-100') + self.send_response_only(100) + self.end_headers() + return True + + def do_POST(self): + query = urlparse.urlparse(self.path).query + logging.info('MultipartPreservingDataHandler POST ?' + query) + if query == 'uploads': + self.send_response(200) + post_data = r''' +TEST'''.encode() + self.send_header('Content-length', str(len(post_data))) + self.send_header('Content-type', 'text/plain') + self.end_headers() + self.wfile.write(post_data) + else: + try: + assert query == 'uploadId=TEST' + logging.info('Content-Length = ' + self.headers.get('Content-Length')) + post_data = self.rfile.read(int(self.headers.get('Content-Length'))) + root = xml.etree.ElementTree.fromstring(post_data) + assert root.tag == 'CompleteMultipartUpload' + assert len(root) > 1 + content = '' + for i, part in enumerate(root): + assert part.tag == 'Part' + assert len(part) == 2 + assert part[0].tag == 'PartNumber' + assert part[1].tag == 'ETag' + assert int(part[0].text) == i + 1 + content += self.server.storage['@'+part[1].text] + data.setdefault('multipart_received_data', []).append(content) + data['multipart_parts'] = len(root) + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.end_headers() + logging.info('Sending 200') + except: + logging.error('Sending 500') + self.send_response(500) + self.finish() + + def do_PUT(self): + uid = uuid.uuid4() + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.send_header('ETag', str(uid)) + self.end_headers() + query = urlparse.urlparse(self.path).query + path = urlparse.urlparse(self.path).path + logging.info('Content-Length = ' + self.headers.get('Content-Length')) + logging.info('PUT ' + query) + assert self.headers.get('Content-Length') + assert self.headers['Expect'] == '100-continue' + put_data = self.rfile.read() + data.setdefault('received_data', []).append(put_data) + logging.info('PUT to {}'.format(path)) + self.server.storage['@'+str(uid)] = put_data + self.finish() + + def do_GET(self): + path = urlparse.urlparse(self.path).path + if path in self.server.storage: + self.send_response(200) + self.send_header('Content-type', 'text/plain') + self.send_header('Content-length', str(len(self.server.storage[path]))) + self.end_headers() + self.wfile.write(self.server.storage[path]) + else: + self.send_response(404) + self.end_headers() + self.finish() + + class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): protocol_version = 'HTTP/1.1' @@ -229,12 +336,20 @@ class CommunicationServerHandler(BaseHTTPRequestHandler): self.wfile.write(json.dumps(data)) self.finish() + def do_PUT(self): + self.send_response(200) + self.end_headers() + logging.info(self.rfile.read()) + self.finish() + servers = [] servers.append(HTTPServer((localhost, communication_port), CommunicationServerHandler)) servers.append(HTTPServer((localhost, redirecting_to_http_port), RedirectingToHTTPHandler)) servers.append(HTTPServer((localhost, preserving_data_port), PreservingDataHandler)) servers[-1].storage = {} +servers.append(HTTPServer((localhost, multipart_preserving_data_port), MultipartPreservingDataHandler)) +servers[-1].storage = {} servers.append(HTTPServer((localhost, simple_server_port), SimpleHTTPServerHandler)) servers.append(HTTPServer((localhost, redirecting_preserving_data_port), RedirectingPreservingDataHandler)) jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] From 787a646680b0af46ed15c7a5755da5f5ef478f71 Mon Sep 17 00:00:00 2001 From: Akazz Date: Thu, 19 Sep 2019 13:00:44 +0300 Subject: [PATCH 053/108] Added a test to challenge timeout mechanism in SYNC REPLICA command --- .../01013_sync_replica_timeout.reference | 1 + .../0_stateless/01013_sync_replica_timeout.sh | 22 +++++++++++++++++++ 2 files changed, 23 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference create mode 100644 dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference new file mode 100644 index 00000000000..a0aba9318ad --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference @@ -0,0 +1 @@ +OK \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh new file mode 100644 index 00000000000..49197ed1207 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh @@ -0,0 +1,22 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + + +${CLICKHOUSE_CLIENT} -n -q " + CREATE TABLE table_1013_1 (x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1013', 'r1') ORDER BY x; + CREATE TABLE table_1013_2 (x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1013', 'r2') ORDER BY x; + + SYSTEM STOP FETCHES table_1013_2; + INSERT INTO table_1013_1 VALUES (1) +" + +timeout 2s ${CLICKHOUSE_CLIENT} -n -q "SET receive_timeout=1; SYSTEM SYNC REPLICA table_1013_2" 2>&1 \ +| fgrep -q "DB::Exception: SYNC REPLICA ${CLICKHOUSE_DATABASE}.table_1013_2: command timed out!" \ + && echo 'OK' \ + || (${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE query = 'SYSTEM SYNC REPLICA table_1013_2'"; echo "Failed!") + +${CLICKHOUSE_CLIENT} -n -q " + DROP TABLE IF EXISTS table_1013_2; + DROP TABLE IF EXISTS table_1013_1;" From 283e09d2a550bb7f2012b5519b0e211548f7c1ad Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 19 Sep 2019 10:02:45 +0000 Subject: [PATCH 054/108] Removed trash logging and fixed query parameters in multipart uploads. --- dbms/src/IO/ReadBufferFromS3.cpp | 2 +- dbms/src/IO/WriteBufferFromS3.cpp | 10 +++------- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index e26f683cdd4..852e601b9ab 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -32,7 +32,7 @@ ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_, if (!credentials.getUsername().empty()) credentials.authenticate(*request); - LOG_TRACE((&Logger::get("ReadWriteBufferFromS3")), "Sending request to " << uri.toString()); + LOG_TRACE((&Logger::get("ReadBufferFromS3")), "Sending request to " << uri.toString()); session->sendRequest(*request); diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 4e574a11c0b..181fba7f63f 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -49,9 +49,6 @@ void WriteBufferFromS3::nextImpl() if (!offset()) return; - - LOG_TRACE((&Logger::get("WriteBufferFromS3")), "nextImpl(), offset() == " << offset()); - temporary_buffer->write(working_buffer.begin(), offset()); last_part_size += offset(); @@ -68,11 +65,9 @@ void WriteBufferFromS3::nextImpl() void WriteBufferFromS3::finalize() { - LOG_TRACE((&Logger::get("WriteBufferFromS3")), "finalize()"); temporary_buffer->finish(); if (!buffer_string.empty()) { - LOG_TRACE((&Logger::get("WriteBufferFromS3")), "finalize(), writing last part"); writePart(buffer_string); } @@ -101,7 +96,8 @@ void WriteBufferFromS3::initiate() HTTPSessionPtr session; std::istream * istr = nullptr; /// owned by session Poco::URI initiate_uri = uri; - initiate_uri.setRawQuery("uploads"); // FIXME find how to leave user params as is + initiate_uri.setRawQuery("uploads"); + initiate_uri.setQueryParameters(uri.getQueryParameters()); for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { @@ -225,7 +221,7 @@ void WriteBufferFromS3::complete() String data; WriteBufferFromString buffer(data); - writeString("", buffer); // FIXME move to Poco::XML maybe?? + writeString("", buffer); for (size_t i = 0; i < part_tags.size(); ++i) { writeString("", buffer); From a88d795f0fece799c401dfcf969f9656470da46a Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 19 Sep 2019 10:15:59 +0000 Subject: [PATCH 055/108] Proper passing of query parameters in initiate multipart upload. --- dbms/src/IO/WriteBufferFromS3.cpp | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 181fba7f63f..82f3290f24d 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -90,14 +90,18 @@ WriteBufferFromS3::~WriteBufferFromS3() void WriteBufferFromS3::initiate() { - // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadInitiate.html + // See https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadInitiate.html Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; std::istream * istr = nullptr; /// owned by session Poco::URI initiate_uri = uri; initiate_uri.setRawQuery("uploads"); - initiate_uri.setQueryParameters(uri.getQueryParameters()); + auto params = uri.getQueryParameters(); + for (QueryParameters::const_iterator it = params.begin(); it != params.end(); ++it) + { + initiate_uri.addQueryParameter(it->first, it->second); + } for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { @@ -148,7 +152,7 @@ void WriteBufferFromS3::initiate() void WriteBufferFromS3::writePart(const String & data) { - // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html + // See https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; @@ -211,7 +215,7 @@ void WriteBufferFromS3::writePart(const String & data) void WriteBufferFromS3::complete() { - // https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadComplete.html + // See https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadComplete.html Poco::Net::HTTPResponse response; std::unique_ptr request; HTTPSessionPtr session; From a5d6bd9e017a1f73da4ad14ed1a1eae2df68976e Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 19 Sep 2019 10:16:39 +0000 Subject: [PATCH 056/108] Proper passing of query parameters in initiate multipart upload. --- dbms/src/IO/WriteBufferFromS3.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 82f3290f24d..a40230e9878 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -98,7 +98,7 @@ void WriteBufferFromS3::initiate() Poco::URI initiate_uri = uri; initiate_uri.setRawQuery("uploads"); auto params = uri.getQueryParameters(); - for (QueryParameters::const_iterator it = params.begin(); it != params.end(); ++it) + for (auto it = params.begin(); it != params.end(); ++it) { initiate_uri.addQueryParameter(it->first, it->second); } From 12f7361095171cf152529534b466e18d8984f89c Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 19 Sep 2019 10:25:31 +0000 Subject: [PATCH 057/108] Removed unused headers from `ReadBufferFromS3.h` --- dbms/src/IO/ReadBufferFromS3.cpp | 2 ++ dbms/src/IO/ReadBufferFromS3.h | 13 +------------ 2 files changed, 3 insertions(+), 12 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index 852e601b9ab..0091930c92e 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -1,5 +1,7 @@ #include +#include + #include diff --git a/dbms/src/IO/ReadBufferFromS3.h b/dbms/src/IO/ReadBufferFromS3.h index ec53a24c5a6..ffc0c5c0ab1 100644 --- a/dbms/src/IO/ReadBufferFromS3.h +++ b/dbms/src/IO/ReadBufferFromS3.h @@ -1,23 +1,12 @@ #pragma once -#include #include -#include + #include #include #include -#include -#include -#include #include -#include -#include -#include #include -#include -#include -#include -#include namespace DB From 0ee6f6231a4f8b514d3abe40ccb8ea875fc235d7 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 19 Sep 2019 10:33:34 +0000 Subject: [PATCH 058/108] Added comments about redirecting requests and got rid of magic values. --- dbms/src/IO/ReadBufferFromS3.cpp | 4 +++- dbms/src/IO/WriteBufferFromS3.cpp | 12 +++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index 0091930c92e..7fcb7a0ca41 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -40,7 +40,9 @@ ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_, istr = &session->receiveResponse(response); - if (response.getStatus() != 307) + // Handle 307 Temporary Redirect in order to allow request redirection + // See https://docs.aws.amazon.com/AmazonS3/latest/dev/Redirects.html + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) break; auto location_iterator = response.find("Location"); diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index a40230e9878..5b6f9fdff4c 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -123,7 +123,9 @@ void WriteBufferFromS3::initiate() istr = &session->receiveResponse(response); - if (response.getStatus() != 307) + // Handle 307 Temporary Redirect in order to allow request redirection + // See https://docs.aws.amazon.com/AmazonS3/latest/dev/Redirects.html + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) break; auto location_iterator = response.find("Location"); @@ -193,7 +195,9 @@ void WriteBufferFromS3::writePart(const String & data) istr = &session->receiveResponse(response); - if (response.getStatus() != 307) + // Handle 307 Temporary Redirect in order to allow request redirection + // See https://docs.aws.amazon.com/AmazonS3/latest/dev/Redirects.html + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) break; auto location_iterator = response.find("Location"); @@ -264,7 +268,9 @@ void WriteBufferFromS3::complete() istr = &session->receiveResponse(response); - if (response.getStatus() != 307) + // Handle 307 Temporary Redirect in order to allow request redirection + // See https://docs.aws.amazon.com/AmazonS3/latest/dev/Redirects.html + if (response.getStatus() != Poco::Net::HTTPResponse::HTTP_TEMPORARY_REDIRECT) break; auto location_iterator = response.find("Location"); From 4ae3db45c6b522b516e2d380822a2df0fce7e29d Mon Sep 17 00:00:00 2001 From: Akazz Date: Thu, 19 Sep 2019 14:04:57 +0300 Subject: [PATCH 059/108] Fixed timeout reading in SYNC REPLICA --- .../Interpreters/InterpreterSystemQuery.cpp | 10 ++++++- .../Storages/StorageReplicatedMergeTree.cpp | 27 +++++++------------ 2 files changed, 18 insertions(+), 19 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 0081f41e3e0..0c0fc5c58ba 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -38,6 +38,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int CANNOT_KILL; extern const int NOT_IMPLEMENTED; + extern const int TIMEOUT_EXCEEDED; } @@ -338,7 +339,14 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) StoragePtr table = context.getTable(database_name, table_name); if (auto storage_replicated = dynamic_cast(table.get())) - storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.value.milliseconds()); + { + LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty"); + if (!storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.totalMilliseconds())) + throw Exception( + "SYNC REPLICA " + database_name + "." + table_name + ": command timed out! " + "See the 'receive_timeout' setting", ErrorCodes::TIMEOUT_EXCEEDED); + LOG_TRACE(log, "SYNC REPLICA " + database_name + "." + table_name + ": OK"); + } else throw Exception("Table " + database_name + "." + table_name + " is not replicated", ErrorCodes::BAD_ARGUMENTS); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 538930d3d2b..c55378d8526 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -5110,38 +5110,29 @@ ActionLock StorageReplicatedMergeTree::getActionLock(StorageActionBlockType acti bool StorageReplicatedMergeTree::waitForShrinkingQueueSize(size_t queue_size, UInt64 max_wait_milliseconds) { + Stopwatch watch; + /// Let's fetch new log entries firstly queue.pullLogsToQueue(getZooKeeper()); - Stopwatch watch; - Poco::Event event; - std::atomic cond_reached{false}; - - auto callback = [&event, &cond_reached, queue_size] (size_t new_queue_size) + Poco::Event target_size_event; + auto callback = [&target_size_event, queue_size] (size_t new_queue_size) { if (new_queue_size <= queue_size) - cond_reached.store(true, std::memory_order_relaxed); - - event.set(); + target_size_event.set(); }; + const auto handler = queue.addSubscriber(std::move(callback)); - auto handler = queue.addSubscriber(std::move(callback)); - - while (true) + while (!target_size_event.tryWait(50)) { - event.tryWait(50); - if (max_wait_milliseconds && watch.elapsedMilliseconds() > max_wait_milliseconds) - break; - - if (cond_reached) - break; + return false; if (partial_shutdown_called) throw Exception("Shutdown is called for table", ErrorCodes::ABORTED); } - return cond_reached.load(std::memory_order_relaxed); + return true; } From 7a0f7a57e262617603247b369a0eb07fe8f3eb15 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 19 Sep 2019 15:48:18 +0300 Subject: [PATCH 060/108] Chmod +x for test!!! --- dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh | 0 1 file changed, 0 insertions(+), 0 deletions(-) mode change 100644 => 100755 dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh old mode 100644 new mode 100755 From b30dee621bc5f40bd89024d9bb81debc0f7ea0ae Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 18 Sep 2019 21:27:18 +0300 Subject: [PATCH 061/108] Search for ld.gold (since it is more common then just "gold") --- CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 53635ed3a05..9d82da6838a 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -139,7 +139,7 @@ endif () string(REGEX MATCH "-?[0-9]+(.[0-9]+)?$" COMPILER_POSTFIX ${CMAKE_CXX_COMPILER}) find_program (LLD_PATH NAMES "lld${COMPILER_POSTFIX}" "lld") -find_program (GOLD_PATH NAMES "gold") +find_program (GOLD_PATH NAMES "ld.gold" "gold") if (COMPILER_CLANG AND LLD_PATH AND NOT LINKER_NAME) set (LINKER_NAME "lld") From b289d7e6a503f3e159c5f6ec951ec4f2fb643278 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 19 Sep 2019 17:41:49 +0300 Subject: [PATCH 062/108] Fix system contributors generating script --- dbms/src/Storages/System/StorageSystemContributors.sh | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/System/StorageSystemContributors.sh b/dbms/src/Storages/System/StorageSystemContributors.sh index aea122df0dc..c4c4eb5ad30 100755 --- a/dbms/src/Storages/System/StorageSystemContributors.sh +++ b/dbms/src/Storages/System/StorageSystemContributors.sh @@ -2,11 +2,15 @@ set -x +# doesn't actually cd to directory, but return absolute path CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# cd to directory +cd $CUR_DIR CONTRIBUTORS_FILE=${CONTRIBUTORS_FILE=$CUR_DIR/StorageSystemContributors.generated.cpp} -git shortlog --summary | perl -lnE 's/^\s+\d+\s+(.+)/ "$1",/; next unless $1; say $_' > $CONTRIBUTORS_FILE.tmp +# if you don't specify HEAD here, without terminal `git shortlog` would expect input from stdin +git shortlog HEAD --summary | perl -lnE 's/^\s+\d+\s+(.+)/ "$1",/; next unless $1; say $_' > $CONTRIBUTORS_FILE.tmp # If git history not available - dont make target file if [ ! -s $CONTRIBUTORS_FILE.tmp ]; then From 90c9d83fa5c98816938782d759dba4704b332247 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Thu, 19 Sep 2019 17:48:08 +0300 Subject: [PATCH 063/108] Added missing newline at the EOF --- .../queries/0_stateless/01013_sync_replica_timeout.reference | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference index a0aba9318ad..d86bac9de59 100644 --- a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference +++ b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference @@ -1 +1 @@ -OK \ No newline at end of file +OK From da14951ecd0ac8fc91b36bac52063a00a704a03a Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 19 Sep 2019 17:53:03 +0300 Subject: [PATCH 064/108] implement MergeJoin.joinTotals() --- dbms/src/Interpreters/Join.cpp | 24 +----------------- dbms/src/Interpreters/MergeJoin.cpp | 5 ++++ dbms/src/Interpreters/MergeJoin.h | 2 +- dbms/src/Interpreters/join_common.cpp | 25 +++++++++++++++++++ dbms/src/Interpreters/join_common.h | 1 + .../0_stateless/00055_join_two_numbers.sql | 3 ++- .../0_stateless/00056_join_number_string.sql | 3 ++- 7 files changed, 37 insertions(+), 26 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index aa585834412..d5381e1dc6d 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -959,29 +959,7 @@ void Join::joinBlock(Block & block) void Join::joinTotals(Block & block) const { - Block totals_without_keys = totals; - - if (totals_without_keys) - { - for (const auto & name : key_names_right) - totals_without_keys.erase(totals_without_keys.getPositionByName(name)); - - for (size_t i = 0; i < totals_without_keys.columns(); ++i) - block.insert(totals_without_keys.safeGetByPosition(i)); - } - else - { - /// We will join empty `totals` - from one row with the default values. - - for (size_t i = 0; i < sample_block_with_columns_to_add.columns(); ++i) - { - const auto & col = sample_block_with_columns_to_add.getByPosition(i); - block.insert({ - col.type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(), - col.type, - col.name}); - } - } + JoinCommon::joinTotals(totals, sample_block_with_columns_to_add, key_names_right, block); } diff --git a/dbms/src/Interpreters/MergeJoin.cpp b/dbms/src/Interpreters/MergeJoin.cpp index 584562fe363..29853c67434 100644 --- a/dbms/src/Interpreters/MergeJoin.cpp +++ b/dbms/src/Interpreters/MergeJoin.cpp @@ -231,6 +231,11 @@ void MergeJoin::setTotals(const Block & totals_block) mergeRightBlocks(); } +void MergeJoin::joinTotals(Block & block) const +{ + JoinCommon::joinTotals(totals, right_columns_to_add, table_join->keyNamesRight(), block); +} + void MergeJoin::mergeRightBlocks() { const size_t max_merged_block_size = 128 * 1024 * 1024; diff --git a/dbms/src/Interpreters/MergeJoin.h b/dbms/src/Interpreters/MergeJoin.h index 127fc20feca..72a431c8d59 100644 --- a/dbms/src/Interpreters/MergeJoin.h +++ b/dbms/src/Interpreters/MergeJoin.h @@ -22,7 +22,7 @@ public: bool addJoinedBlock(const Block & block) override; void joinBlock(Block &) override; - void joinTotals(Block &) const override {} + void joinTotals(Block &) const override; void setTotals(const Block &) override; size_t getTotalRowCount() const override { return right_blocks_row_count; } diff --git a/dbms/src/Interpreters/join_common.cpp b/dbms/src/Interpreters/join_common.cpp index 4dea22cb453..dad4fbc7301 100644 --- a/dbms/src/Interpreters/join_common.cpp +++ b/dbms/src/Interpreters/join_common.cpp @@ -122,5 +122,30 @@ void createMissedColumns(Block & block) } } +void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block) +{ + if (Block totals_without_keys = totals) + { + for (const auto & name : key_names_right) + totals_without_keys.erase(totals_without_keys.getPositionByName(name)); + + for (size_t i = 0; i < totals_without_keys.columns(); ++i) + block.insert(totals_without_keys.safeGetByPosition(i)); + } + else + { + /// We will join empty `totals` - from one row with the default values. + + for (size_t i = 0; i < columns_to_add.columns(); ++i) + { + const auto & col = columns_to_add.getByPosition(i); + block.insert({ + col.type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(), + col.type, + col.name}); + } + } +} + } } diff --git a/dbms/src/Interpreters/join_common.h b/dbms/src/Interpreters/join_common.h index d75a25563ad..8567ea68f01 100644 --- a/dbms/src/Interpreters/join_common.h +++ b/dbms/src/Interpreters/join_common.h @@ -26,6 +26,7 @@ ColumnRawPtrs extractKeysForJoin(const Names & key_names_right, const Block & ri void checkTypesOfKeys(const Block & block_left, const Names & key_names_left, const Block & block_right, const Names & key_names_right); void createMissedColumns(Block & block); +void joinTotals(const Block & totals, const Block & columns_to_add, const Names & key_names_right, Block & block); } diff --git a/dbms/tests/queries/0_stateless/00055_join_two_numbers.sql b/dbms/tests/queries/0_stateless/00055_join_two_numbers.sql index 76c38bac41e..572062d4537 100644 --- a/dbms/tests/queries/0_stateless/00055_join_two_numbers.sql +++ b/dbms/tests/queries/0_stateless/00055_join_two_numbers.sql @@ -6,4 +6,5 @@ ALL LEFT JOIN ( SELECT number % 2 AS k1, number % 6 AS k2, number AS right FROM system.numbers LIMIT 10 ) js2 -USING k1, k2; +USING k1, k2 +ORDER BY left, right; diff --git a/dbms/tests/queries/0_stateless/00056_join_number_string.sql b/dbms/tests/queries/0_stateless/00056_join_number_string.sql index db10ad99b27..6b8e54ce536 100644 --- a/dbms/tests/queries/0_stateless/00056_join_number_string.sql +++ b/dbms/tests/queries/0_stateless/00056_join_number_string.sql @@ -6,4 +6,5 @@ ALL LEFT JOIN ( SELECT number % 2 AS k1, toString(number % 6) AS k2, number AS right FROM system.numbers LIMIT 10 ) js2 -USING k1, k2; +USING k1, k2 +ORDER BY left, right; From 0bc345fdc0f2cc2b9d722efeebf1f9e61c368f27 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 19 Sep 2019 20:09:59 +0300 Subject: [PATCH 065/108] MergeJoin compare nullables --- dbms/src/Interpreters/MergeJoin.cpp | 137 ++++++++++++++++++++++------ 1 file changed, 107 insertions(+), 30 deletions(-) diff --git a/dbms/src/Interpreters/MergeJoin.cpp b/dbms/src/Interpreters/MergeJoin.cpp index 29853c67434..b6a02a605ef 100644 --- a/dbms/src/Interpreters/MergeJoin.cpp +++ b/dbms/src/Interpreters/MergeJoin.cpp @@ -17,6 +17,51 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } +namespace +{ + +template +int nullableCompareAt(const IColumn & left_column, const IColumn & right_column, size_t lhs_pos, size_t rhs_pos) +{ + static constexpr int null_direction_hint = 1; + + if constexpr (has_nulls) + { + auto * left_nullable = checkAndGetColumn(left_column); + auto * right_nullable = checkAndGetColumn(right_column); + + if (left_nullable && right_nullable) + { + int res = left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); + if (res) + return res; + + /// NULL != NULL case + if (left_column.isNullAt(lhs_pos)) + return null_direction_hint; + } + + if (left_nullable && !right_nullable) + { + if (left_column.isNullAt(lhs_pos)) + return null_direction_hint; + return left_nullable->getNestedColumn().compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); + } + + if (!left_nullable && right_nullable) + { + if (right_column.isNullAt(rhs_pos)) + return -null_direction_hint; + return left_column.compareAt(lhs_pos, rhs_pos, right_nullable->getNestedColumn(), null_direction_hint); + } + } + + /// !left_nullable && !right_nullable + return left_column.compareAt(lhs_pos, rhs_pos, right_column, null_direction_hint); +} + +} + struct MergeJoinEqualRange { size_t left_start = 0; @@ -42,45 +87,40 @@ public: bool atEnd() const { return impl.pos >= impl.rows; } void nextN(size_t num) { impl.pos += num; } - int compareAt(const MergeJoinCursor & rhs, size_t lhs_pos, size_t rhs_pos) const + void setCompareNullability(const MergeJoinCursor & rhs) { - int res = 0; + has_nullable_columns = false; + for (size_t i = 0; i < impl.sort_columns_size; ++i) { - res = impl.sort_columns[i]->compareAt(lhs_pos, rhs_pos, *(rhs.impl.sort_columns[i]), 1); - if (res) + bool is_left_nullable = isColumnNullable(*impl.sort_columns[i]); + bool is_right_nullable = isColumnNullable(*rhs.impl.sort_columns[i]); + + if (is_left_nullable || is_right_nullable) + { + has_nullable_columns = true; break; + } } - return res; - } - - bool sameNext(size_t lhs_pos) const - { - if (lhs_pos + 1 >= impl.rows) - return false; - - for (size_t i = 0; i < impl.sort_columns_size; ++i) - if (impl.sort_columns[i]->compareAt(lhs_pos, lhs_pos + 1, *(impl.sort_columns[i]), 1) != 0) - return false; - return true; - } - - size_t getEqualLength() - { - if (atEnd()) - return 0; - - size_t pos = impl.pos; - while (sameNext(pos)) - ++pos; - return pos - impl.pos + 1; } Range getNextEqualRange(MergeJoinCursor & rhs) + { + if (has_nullable_columns) + return getNextEqualRangeImpl(rhs); + return getNextEqualRangeImpl(rhs); + } + +private: + SortCursorImpl impl; + bool has_nullable_columns = false; + + template + Range getNextEqualRangeImpl(MergeJoinCursor & rhs) { while (!atEnd() && !rhs.atEnd()) { - int cmp = compareAt(rhs, impl.pos, rhs.impl.pos); + int cmp = compareAt(rhs, impl.pos, rhs.impl.pos); if (cmp < 0) impl.next(); if (cmp > 0) @@ -97,8 +137,43 @@ public: return Range{impl.pos, rhs.impl.pos, 0, 0}; } -private: - SortCursorImpl impl; + template + int compareAt(const MergeJoinCursor & rhs, size_t lhs_pos, size_t rhs_pos) const + { + int res = 0; + for (size_t i = 0; i < impl.sort_columns_size; ++i) + { + auto * left_column = impl.sort_columns[i]; + auto * right_column = rhs.impl.sort_columns[i]; + + res = nullableCompareAt(*left_column, *right_column, lhs_pos, rhs_pos); + if (res) + break; + } + return res; + } + + size_t getEqualLength() + { + if (atEnd()) + return 0; + + size_t pos = impl.pos; + while (sameNext(pos)) + ++pos; + return pos - impl.pos + 1; + } + + bool sameNext(size_t lhs_pos) const + { + if (lhs_pos + 1 >= impl.rows) + return false; + + for (size_t i = 0; i < impl.sort_columns_size; ++i) + if (impl.sort_columns[i]->compareAt(lhs_pos, lhs_pos + 1, *(impl.sort_columns[i]), 1) != 0) + return false; + return true; + } }; namespace @@ -323,6 +398,7 @@ void MergeJoin::leftJoin(MergeJoinCursor & left_cursor, const Block & left_block MutableColumns & left_columns, MutableColumns & right_columns, size_t & left_key_tail) { MergeJoinCursor right_cursor(right_block, right_merge_description); + left_cursor.setCompareNullability(right_cursor); while (!left_cursor.atEnd() && !right_cursor.atEnd()) { @@ -355,6 +431,7 @@ void MergeJoin::innerJoin(MergeJoinCursor & left_cursor, const Block & left_bloc MutableColumns & left_columns, MutableColumns & right_columns, size_t & left_key_tail) { MergeJoinCursor right_cursor(right_block, right_merge_description); + left_cursor.setCompareNullability(right_cursor); while (!left_cursor.atEnd() && !right_cursor.atEnd()) { From 5526f33a5ed7d8b73f49e4981ba9d42e106349ed Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Thu, 19 Sep 2019 21:01:47 +0300 Subject: [PATCH 066/108] Implement auto-labelling for the backporting script (#7001) --- utils/github/__main__.py | 34 +++++++++++---- utils/github/parser.py | 44 ++++++++++++++++++++ utils/github/query.py | 89 +++++++++++++++++++++++++++++++++------- 3 files changed, 144 insertions(+), 23 deletions(-) create mode 100644 utils/github/parser.py diff --git a/utils/github/__main__.py b/utils/github/__main__.py index dd8ed692e1d..59bf9e01450 100644 --- a/utils/github/__main__.py +++ b/utils/github/__main__.py @@ -20,13 +20,18 @@ ''' from . import local, query - -from termcolor import colored # `pip install termcolor` +from . import parser as parse_description import argparse import re import sys +try: + from termcolor import colored # `pip install termcolor` +except ImportError: + sys.exit("Package 'termcolor' not found. Try run: `pip3 install termcolor`") + + CHECK_MARK = colored('🗸', 'green') CROSS_MARK = colored('🗙', 'red') LABEL_MARK = colored('🏷', 'yellow') @@ -44,6 +49,8 @@ parser.add_argument('--token', type=str, required=True, help='token for Github access') parser.add_argument('--login', type=str, help='filter authorship by login') +parser.add_argument('--auto-label', action='store_true', dest='autolabel', + help='try to automatically parse PR description and put labels') args = parser.parse_args() @@ -83,14 +90,23 @@ def print_responsible(pull_request): bad_pull_requests = [] # collect and print if not empty need_backporting = [] for pull_request in pull_requests: - label_found = False - for label in github.get_labels(pull_request): - if label['name'].startswith('pr-'): - label_found = True - if label['color'] == 'ff0000': - need_backporting.append(pull_request) - break + def find_label(): + for label in github.get_labels(pull_request): + if label['name'].startswith('pr-'): + if label['color'] == 'ff0000': + need_backporting.append(pull_request) + return True + return False + + label_found = find_label() + + if not label_found and args.autolabel: + print(f"Trying to auto-label pull-request: {pull_request['number']}") + description = parse_description.Description(pull_request) + if description.label_name: + github.set_label(pull_request, description.label_name) + label_found = find_label() if not label_found: bad_pull_requests.append(pull_request) diff --git a/utils/github/parser.py b/utils/github/parser.py new file mode 100644 index 00000000000..77ad5a1b278 --- /dev/null +++ b/utils/github/parser.py @@ -0,0 +1,44 @@ +# -*- coding: utf-8 -*- + +class Description: + '''Parsed description representation + ''' + MAP_CATEGORY_TO_LABEL = { + 'New Feature': 'pr-feature', + 'Bug Fix': 'pr-bugfix', + 'Improvement': 'pr-improvement', + 'Performance Improvement': 'pr-performance', + # 'Backward Incompatible Change': doesn't match anything + 'Build/Testing/Packaging Improvement': 'pr-build', + # 'Other': doesn't match anything + } + + def __init__(self, pull_request): + self.label_name = str() + self.legal = False + + self._parse(pull_request['bodyText']) + + def _parse(self, text): + lines = text.splitlines() + next_category = False + category = str() + + for line in lines: + stripped = line.strip() + + if not stripped: + continue + + if next_category: + category = stripped + next_category = False + + if stripped == 'I hereby agree to the terms of the CLA available at: https://yandex.ru/legal/cla/?lang=en': + self.legal = True + + if stripped == 'Category (leave one):': + next_category = True + + if category in Description.MAP_CATEGORY_TO_LABEL: + self.label_name = Description.MAP_CATEGORY_TO_LABEL[category] diff --git a/utils/github/query.py b/utils/github/query.py index 5810e3bb40a..f2926e67330 100644 --- a/utils/github/query.py +++ b/utils/github/query.py @@ -180,10 +180,12 @@ class Query: totalCount nodes {{ ... on PullRequest {{ + id number author {{ login }} + bodyText mergedBy {{ login }} @@ -301,7 +303,58 @@ class Query: ''' return self._run(Query._DEFAULT)['repository']['defaultBranchRef']['name'] - def _run(self, query): + _GET_LABEL = ''' + repository(owner: "yandex" name: "ClickHouse") {{ + labels(first: {max_page_size} {next} query: "{name}") {{ + pageInfo {{ + hasNextPage + endCursor + }} + nodes {{ + id + name + color + }} + }} + }} + ''' + _SET_LABEL = ''' + addLabelsToLabelable(input: {{ labelableId: "{pr_id}", labelIds: "{label_id}" }}) {{ + clientMutationId + }} + ''' + def set_label(self, pull_request, label_name): + '''Set label by name to the pull request + + Args: + pull_request: JSON object returned by `get_pull_requests()` + label_name (string): label name + ''' + labels = [] + not_end = True + query = Query._GET_LABEL.format(name=label_name, + max_page_size=self._max_page_size, + next='') + + while not_end: + result = self._run(query)['repository']['labels'] + not_end = result['pageInfo']['hasNextPage'] + query = Query._GET_LABEL.format(name=label_name, + max_page_size=self._max_page_size, + next=f'after: "{result["pageInfo"]["endCursor"]}"') + + labels += [label for label in result['nodes']] + + if not labels: + return + + query = Query._SET_LABEL.format(pr_id = pull_request['id'], label_id = labels[0]['id']) + self._run(query, is_mutation=True) + + pull_request['labels']['nodes'].append(labels[0]) + + + def _run(self, query, is_mutation=False): from requests.adapters import HTTPAdapter from urllib3.util.retry import Retry @@ -325,26 +378,34 @@ class Query: return session headers = {'Authorization': f'bearer {self._token}'} - query = f''' - {{ - {query} - rateLimit {{ - cost - remaining + if is_mutation: + query = f''' + mutation {{ + {query} }} - }} - ''' + ''' + else: + query = f''' + query {{ + {query} + rateLimit {{ + cost + remaining + }} + }} + ''' request = requests_retry_session().post('https://api.github.com/graphql', json={'query': query}, headers=headers) if request.status_code == 200: result = request.json() if 'errors' in result: raise Exception(f'Errors occured: {result["errors"]}') - import inspect - caller = inspect.getouterframes(inspect.currentframe(), 2)[1][3] - if caller not in self.api_costs.keys(): - self.api_costs[caller] = 0 - self.api_costs[caller] += result['data']['rateLimit']['cost'] + if not is_mutation: + import inspect + caller = inspect.getouterframes(inspect.currentframe(), 2)[1][3] + if caller not in self.api_costs.keys(): + self.api_costs[caller] = 0 + self.api_costs[caller] += result['data']['rateLimit']['cost'] return result['data'] else: From b2a3db1048629ba5bf04103ba1c8b99130691d44 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 18 Sep 2019 00:25:19 +0300 Subject: [PATCH 067/108] Ignore indices for CREATE TABLE .. AS .. if storage do not support them --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 4 +++- .../0_stateless/01011_test_create_as_skip_indices.reference | 0 .../0_stateless/01011_test_create_as_skip_indices.sql | 5 +++++ 3 files changed, 8 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.reference create mode 100644 dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 59ae2fc2b72..f2189a59866 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -2,6 +2,7 @@ #include +#include #include #include @@ -416,7 +417,8 @@ ColumnsDescription InterpreterCreateQuery::setProperties( else if (!create.as_table.empty()) { columns = as_storage->getColumns(); - indices = as_storage->getIndices(); + if (create.storage && endsWith(create.storage->engine->name, "MergeTree")) + indices = as_storage->getIndices(); constraints = as_storage->getConstraints(); } else if (create.select) diff --git a/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.reference b/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql b/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql new file mode 100644 index 00000000000..b702fc3654c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01011_test_create_as_skip_indices.sql @@ -0,0 +1,5 @@ +SET allow_experimental_data_skipping_indices=1; +CREATE TABLE foo (key int, INDEX i1 key TYPE minmax GRANULARITY 1) Engine=MergeTree() ORDER BY key; +CREATE TABLE as_foo AS foo; +CREATE TABLE dist (key int, INDEX i1 key TYPE minmax GRANULARITY 1) Engine=Distributed(test_shard_localhost, currentDatabase(), 'foo'); -- { serverError 36 } +CREATE TABLE dist_as_foo Engine=Distributed(test_shard_localhost, currentDatabase(), 'foo') AS foo; From 2d807368e14096fde25f8864f05fbc8d41693275 Mon Sep 17 00:00:00 2001 From: Alexander Kazakov Date: Fri, 20 Sep 2019 00:20:58 +0300 Subject: [PATCH 068/108] Adjusted timings in test + more verbose diagnostics --- .../Interpreters/InterpreterSystemQuery.cpp | 3 ++ .../0_stateless/01013_sync_replica_timeout.sh | 22 -------------- ..._sync_replica_timeout_zookeeper.reference} | 0 .../01013_sync_replica_timeout_zookeeper.sh | 30 +++++++++++++++++++ 4 files changed, 33 insertions(+), 22 deletions(-) delete mode 100755 dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh rename dbms/tests/queries/0_stateless/{01013_sync_replica_timeout.reference => 01013_sync_replica_timeout_zookeeper.reference} (100%) create mode 100755 dbms/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 0c0fc5c58ba..6d4ad1219f9 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -342,9 +342,12 @@ void InterpreterSystemQuery::syncReplica(ASTSystemQuery & query) { LOG_TRACE(log, "Synchronizing entries in replica's queue with table's log and waiting for it to become empty"); if (!storage_replicated->waitForShrinkingQueueSize(0, context.getSettingsRef().receive_timeout.totalMilliseconds())) + { + LOG_ERROR(log, "SYNC REPLICA " + database_name + "." + table_name + ": Timed out!"); throw Exception( "SYNC REPLICA " + database_name + "." + table_name + ": command timed out! " "See the 'receive_timeout' setting", ErrorCodes::TIMEOUT_EXCEEDED); + } LOG_TRACE(log, "SYNC REPLICA " + database_name + "." + table_name + ": OK"); } else diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh deleted file mode 100755 index 49197ed1207..00000000000 --- a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.sh +++ /dev/null @@ -1,22 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - - -${CLICKHOUSE_CLIENT} -n -q " - CREATE TABLE table_1013_1 (x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1013', 'r1') ORDER BY x; - CREATE TABLE table_1013_2 (x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1013', 'r2') ORDER BY x; - - SYSTEM STOP FETCHES table_1013_2; - INSERT INTO table_1013_1 VALUES (1) -" - -timeout 2s ${CLICKHOUSE_CLIENT} -n -q "SET receive_timeout=1; SYSTEM SYNC REPLICA table_1013_2" 2>&1 \ -| fgrep -q "DB::Exception: SYNC REPLICA ${CLICKHOUSE_DATABASE}.table_1013_2: command timed out!" \ - && echo 'OK' \ - || (${CLICKHOUSE_CLIENT} -q "KILL QUERY WHERE query = 'SYSTEM SYNC REPLICA table_1013_2'"; echo "Failed!") - -${CLICKHOUSE_CLIENT} -n -q " - DROP TABLE IF EXISTS table_1013_2; - DROP TABLE IF EXISTS table_1013_1;" diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.reference similarity index 100% rename from dbms/tests/queries/0_stateless/01013_sync_replica_timeout.reference rename to dbms/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.reference diff --git a/dbms/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh new file mode 100755 index 00000000000..9e846b42591 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_sync_replica_timeout_zookeeper.sh @@ -0,0 +1,30 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + + +R1=table_1013_1 +R2=table_1013_2 + +${CLICKHOUSE_CLIENT} -n -q " + DROP TABLE IF EXISTS $R1; + DROP TABLE IF EXISTS $R2; + + CREATE TABLE $R1 (x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1013', 'r1') ORDER BY x; + CREATE TABLE $R2 (x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/${CLICKHOUSE_DATABASE}.table_1013', 'r2') ORDER BY x; + + SYSTEM STOP FETCHES $R2; + INSERT INTO $R1 VALUES (1) +" + +timeout 10s ${CLICKHOUSE_CLIENT} -n -q " + SET receive_timeout=1; + SYSTEM SYNC REPLICA $R2 +" 2>&1 | fgrep -q "DB::Exception: SYNC REPLICA ${CLICKHOUSE_DATABASE}.$R2: command timed out!" && echo 'OK' || echo 'Failed!' + +# By dropping tables all related SYNC REPLICA queries would be terminated as well +${CLICKHOUSE_CLIENT} -n -q " + DROP TABLE IF EXISTS $R2; + DROP TABLE IF EXISTS $R1; +" From e07ff38b848366258e3b92ad9d6ffb9453c5da6c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 20 Sep 2019 12:12:21 +0300 Subject: [PATCH 069/108] Bump urllib3 from 1.23 to 1.24.2 in /docs/tools (#7003) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.23 to 1.24.2. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/master/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.23...1.24.2) Signed-off-by: dependabot[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 6ac9a4a034c..440789f907c 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -32,4 +32,4 @@ sphinxcontrib-websupport==1.0.1 tornado==5.1 typing==3.6.2 Unidecode==1.0.23 -urllib3==1.23 +urllib3==1.24.2 From ef75a45fefef671b7cc1b206bd7f5ba01dbb0834 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 20 Sep 2019 12:12:35 +0300 Subject: [PATCH 070/108] Bump pyyaml from 3.12 to 5.1 in /docs/tools (#7004) Bumps [pyyaml](https://github.com/yaml/pyyaml) from 3.12 to 5.1. - [Release notes](https://github.com/yaml/pyyaml/releases) - [Changelog](https://github.com/yaml/pyyaml/blob/master/CHANGES) - [Commits](https://github.com/yaml/pyyaml/compare/3.12...5.1) Signed-off-by: dependabot[bot] --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index 440789f907c..b216433f772 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -21,7 +21,7 @@ mkdocs==1.0.4 Pygments==2.2.0 python-slugify==1.2.6 pytz==2017.3 -PyYAML==3.12 +PyYAML==5.1 recommonmark==0.4.0 requests==2.21.0 singledispatch==3.4.0.3 From cffc2549222decbfd37f3542d4365a760cfbb742 Mon Sep 17 00:00:00 2001 From: Ivan <5627721+abyss7@users.noreply.github.com> Date: Fri, 20 Sep 2019 15:12:32 +0300 Subject: [PATCH 071/108] Always read and insert Kafka messages as a whole (#6950) * Always read and insert Kafka messages as a whole. --- dbms/src/DataStreams/SquashingTransform.cpp | 2 +- dbms/src/Formats/FormatFactory.cpp | 8 +- dbms/src/Formats/FormatFactory.h | 3 - dbms/src/Formats/NativeFormat.cpp | 1 - .../Formats/tests/tab_separated_streams.cpp | 2 +- .../Processors/Formats/IRowInputFormat.cpp | 39 +------- dbms/src/Processors/Formats/IRowInputFormat.h | 3 - .../Storages/Kafka/KafkaBlockInputStream.cpp | 94 ++++++++++++++----- .../Kafka/ReadBufferFromKafkaConsumer.cpp | 13 +-- .../Kafka/ReadBufferFromKafkaConsumer.h | 6 +- dbms/src/Storages/Kafka/StorageKafka.cpp | 2 +- 11 files changed, 81 insertions(+), 92 deletions(-) diff --git a/dbms/src/DataStreams/SquashingTransform.cpp b/dbms/src/DataStreams/SquashingTransform.cpp index 00e3a51582c..22f35b1fe8f 100644 --- a/dbms/src/DataStreams/SquashingTransform.cpp +++ b/dbms/src/DataStreams/SquashingTransform.cpp @@ -16,7 +16,7 @@ SquashingTransform::Result SquashingTransform::add(MutableColumns && columns) if (columns.empty()) return Result(std::move(accumulated_columns)); - /// Just read block is alredy enough. + /// Just read block is already enough. if (isEnoughSize(columns)) { /// If no accumulated data, return just read block. diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index c1798b2d84f..b3253d3cbf1 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -83,7 +83,6 @@ BlockInputStreamPtr FormatFactory::getInput( const Block & sample, const Context & context, UInt64 max_block_size, - UInt64 rows_portion_size, ReadCallback callback) const { if (name == "Native") @@ -98,11 +97,10 @@ BlockInputStreamPtr FormatFactory::getInput( const Settings & settings = context.getSettingsRef(); FormatSettings format_settings = getInputFormatSetting(settings); - return input_getter( - buf, sample, context, max_block_size, rows_portion_size, callback ? callback : ReadCallback(), format_settings); + return input_getter(buf, sample, context, max_block_size, callback ? callback : ReadCallback(), format_settings); } - auto format = getInputFormat(name, buf, sample, context, max_block_size, rows_portion_size, std::move(callback)); + auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback)); return std::make_shared(std::move(format)); } @@ -150,7 +148,6 @@ InputFormatPtr FormatFactory::getInputFormat( const Block & sample, const Context & context, UInt64 max_block_size, - UInt64 rows_portion_size, ReadCallback callback) const { const auto & input_getter = getCreators(name).input_processor_creator; @@ -164,7 +161,6 @@ InputFormatPtr FormatFactory::getInputFormat( params.max_block_size = max_block_size; params.allow_errors_num = format_settings.input_allow_errors_num; params.allow_errors_ratio = format_settings.input_allow_errors_ratio; - params.rows_portion_size = rows_portion_size; params.callback = std::move(callback); params.max_execution_time = settings.max_execution_time; params.timeout_overflow_mode = settings.timeout_overflow_mode; diff --git a/dbms/src/Formats/FormatFactory.h b/dbms/src/Formats/FormatFactory.h index 04e9ce22e09..1c6fbc1b97e 100644 --- a/dbms/src/Formats/FormatFactory.h +++ b/dbms/src/Formats/FormatFactory.h @@ -51,7 +51,6 @@ private: const Block & sample, const Context & context, UInt64 max_block_size, - UInt64 rows_portion_size, ReadCallback callback, const FormatSettings & settings)>; @@ -96,7 +95,6 @@ public: const Block & sample, const Context & context, UInt64 max_block_size, - UInt64 rows_portion_size = 0, ReadCallback callback = {}) const; BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf, @@ -108,7 +106,6 @@ public: const Block & sample, const Context & context, UInt64 max_block_size, - UInt64 rows_portion_size = 0, ReadCallback callback = {}) const; OutputFormatPtr getOutputFormat( diff --git a/dbms/src/Formats/NativeFormat.cpp b/dbms/src/Formats/NativeFormat.cpp index f9063e0eced..11835c01123 100644 --- a/dbms/src/Formats/NativeFormat.cpp +++ b/dbms/src/Formats/NativeFormat.cpp @@ -13,7 +13,6 @@ void registerInputFormatNative(FormatFactory & factory) const Block & sample, const Context &, UInt64 /* max_block_size */, - UInt64 /* min_read_rows */, FormatFactory::ReadCallback /* callback */, const FormatSettings &) { diff --git a/dbms/src/Formats/tests/tab_separated_streams.cpp b/dbms/src/Formats/tests/tab_separated_streams.cpp index f05a83bc751..671043b9aac 100644 --- a/dbms/src/Formats/tests/tab_separated_streams.cpp +++ b/dbms/src/Formats/tests/tab_separated_streams.cpp @@ -39,7 +39,7 @@ try FormatSettings format_settings; - RowInputFormatParams params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, 0, []{}}; + RowInputFormatParams params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, []{}}; InputFormatPtr input_format = std::make_shared(sample, in_buf, params, false, false, format_settings); BlockInputStreamPtr block_input = std::make_shared(std::move(input_format)); diff --git a/dbms/src/Processors/Formats/IRowInputFormat.cpp b/dbms/src/Processors/Formats/IRowInputFormat.cpp index b45c714ea07..a63fd80a087 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/IRowInputFormat.cpp @@ -20,8 +20,10 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; } +namespace +{ -static bool isParseError(int code) +bool isParseError(int code) { return code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED || code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING @@ -33,34 +35,8 @@ static bool isParseError(int code) || code == ErrorCodes::TOO_LARGE_STRING_SIZE; } - -static bool handleOverflowMode(OverflowMode mode, const String & message, int code) -{ - switch (mode) - { - case OverflowMode::THROW: - throw Exception(message, code); - case OverflowMode::BREAK: - return false; - default: - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); - } } - -static bool checkTimeLimit(const IRowInputFormat::Params & params, const Stopwatch & stopwatch) -{ - if (params.max_execution_time != 0 - && stopwatch.elapsed() > static_cast(params.max_execution_time.totalMicroseconds()) * 1000) - return handleOverflowMode(params.timeout_overflow_mode, - "Timeout exceeded: elapsed " + toString(stopwatch.elapsedSeconds()) - + " seconds, maximum: " + toString(params.max_execution_time.totalMicroseconds() / 1000000.0), - ErrorCodes::TIMEOUT_EXCEEDED); - - return true; -} - - Chunk IRowInputFormat::generate() { if (total_rows == 0) @@ -76,15 +52,8 @@ Chunk IRowInputFormat::generate() try { - for (size_t rows = 0, batch = 0; rows < params.max_block_size; ++rows, ++batch) + for (size_t rows = 0; rows < params.max_block_size; ++rows) { - if (params.rows_portion_size && batch == params.rows_portion_size) - { - batch = 0; - if (!checkTimeLimit(params, total_stopwatch) || isCancelled()) - break; - } - try { ++total_rows; diff --git a/dbms/src/Processors/Formats/IRowInputFormat.h b/dbms/src/Processors/Formats/IRowInputFormat.h index 72a6c813701..ff6c60b3999 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.h +++ b/dbms/src/Processors/Formats/IRowInputFormat.h @@ -27,8 +27,6 @@ struct RowInputFormatParams UInt64 allow_errors_num; Float64 allow_errors_ratio; - UInt64 rows_portion_size; - using ReadCallback = std::function; ReadCallback callback; @@ -85,4 +83,3 @@ private: }; } - diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp index 29adb061e29..adfa5300435 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -49,10 +49,13 @@ void KafkaBlockInputStream::readPrefixImpl() buffer->subscribe(storage.getTopics()); - const auto & limits_ = getLimits(); - const size_t poll_timeout = buffer->pollTimeout(); - size_t rows_portion_size = poll_timeout ? std::min(max_block_size, limits_.max_execution_time.totalMilliseconds() / poll_timeout) : max_block_size; - rows_portion_size = std::max(rows_portion_size, 1ul); + broken = true; +} + +Block KafkaBlockInputStream::readImpl() +{ + if (!buffer) + return Block(); auto non_virtual_header = storage.getSampleBlockNonMaterialized(); /// FIXME: add materialized columns support auto read_callback = [this] @@ -67,33 +70,72 @@ void KafkaBlockInputStream::readPrefixImpl() virtual_columns[4]->insert(std::chrono::duration_cast(timestamp->get_timestamp()).count()); // "timestamp" }; - auto child = FormatFactory::instance().getInput( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size, rows_portion_size, read_callback); - child->setLimits(limits_); - addChild(child); + auto merge_blocks = [] (Block & block1, Block && block2) + { + if (!block1) + { + // Need to make sure that resulting block has the same structure + block1 = std::move(block2); + return; + } - broken = true; -} + if (!block2) + return; -Block KafkaBlockInputStream::readImpl() -{ - if (!buffer) + auto columns1 = block1.mutateColumns(); + auto columns2 = block2.mutateColumns(); + for (size_t i = 0, s = columns1.size(); i < s; ++i) + columns1[i]->insertRangeFrom(*columns2[i], 0, columns2[i]->size()); + block1.setColumns(std::move(columns1)); + }; + + auto read_kafka_message = [&, this] + { + Block result; + auto child = FormatFactory::instance().getInput( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size, read_callback); + const auto virtual_header = storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp"}); + + while (auto block = child->read()) + { + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + virtual_columns = virtual_header.cloneEmptyColumns(); + + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + block.insert(column); + + /// FIXME: materialize MATERIALIZED columns here. + + merge_blocks(result, std::move(block)); + } + + return result; + }; + + Block single_block; + + UInt64 total_rows = 0; + while (total_rows < max_block_size) + { + auto new_block = read_kafka_message(); + auto new_rows = new_block.rows(); + total_rows += new_rows; + merge_blocks(single_block, std::move(new_block)); + + buffer->allowNext(); + + if (!new_rows || !checkTimeLimit()) + break; + } + + if (!single_block) return Block(); - Block block = children.back()->read(); - if (!block) - return block; - - Block virtual_block = storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp"}).cloneWithColumns(std::move(virtual_columns)); - virtual_columns = storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp"}).cloneEmptyColumns(); - - for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - block.insert(column); - - /// FIXME: materialize MATERIALIZED columns here. - return ConvertingBlockInputStream( - context, std::make_shared(block), getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name) + context, + std::make_shared(single_block), + getHeader(), + ConvertingBlockInputStream::MatchColumnsMode::Name) .read(); } diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp index 083b471d4f1..9a3bd73a6b2 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp @@ -13,7 +13,6 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( size_t max_batch_size, size_t poll_timeout_, bool intermediate_commit_, - char delimiter_, const std::atomic & stopped_) : ReadBuffer(nullptr, 0) , consumer(consumer_) @@ -21,7 +20,6 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer( , batch_size(max_batch_size) , poll_timeout(poll_timeout_) , intermediate_commit(intermediate_commit_) - , delimiter(delimiter_) , stopped(stopped_) , current(messages.begin()) { @@ -140,16 +138,9 @@ bool ReadBufferFromKafkaConsumer::nextImpl() /// NOTE: ReadBuffer was implemented with an immutable underlying contents in mind. /// If we failed to poll any message once - don't try again. /// Otherwise, the |poll_timeout| expectations get flawn. - if (stalled || stopped) + if (stalled || stopped || !allowed) return false; - if (put_delimiter) - { - BufferBase::set(&delimiter, 1, 0); - put_delimiter = false; - return true; - } - if (current == messages.end()) { if (intermediate_commit) @@ -181,7 +172,7 @@ bool ReadBufferFromKafkaConsumer::nextImpl() // XXX: very fishy place with const casting. auto new_position = reinterpret_cast(const_cast(current->get_payload().get_data())); BufferBase::set(new_position, current->get_payload().get_size(), 0); - put_delimiter = (delimiter != 0); + allowed = false; /// Since we can poll more messages than we already processed - commit only processed messages. consumer->store_offset(*current); diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h index d5d847dd153..8c2fcd3c7bb 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.h @@ -25,10 +25,10 @@ public: size_t max_batch_size, size_t poll_timeout_, bool intermediate_commit_, - char delimiter_, const std::atomic & stopped_); ~ReadBufferFromKafkaConsumer() override; + void allowNext() { allowed = true; } // Allow to read next message. void commit(); // Commit all processed messages. void subscribe(const Names & topics); // Subscribe internal consumer to topics. void unsubscribe(); // Unsubscribe internal consumer in case of failure. @@ -51,9 +51,7 @@ private: const size_t poll_timeout = 0; bool stalled = false; bool intermediate_commit = true; - - char delimiter; - bool put_delimiter = false; + bool allowed = true; const std::atomic & stopped; diff --git a/dbms/src/Storages/Kafka/StorageKafka.cpp b/dbms/src/Storages/Kafka/StorageKafka.cpp index ed067993a18..c27fdaf4fe7 100644 --- a/dbms/src/Storages/Kafka/StorageKafka.cpp +++ b/dbms/src/Storages/Kafka/StorageKafka.cpp @@ -278,7 +278,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer() size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds(); /// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage. - return std::make_shared(consumer, log, batch_size, poll_timeout, intermediate_commit, row_delimiter, stream_cancelled); + return std::make_shared(consumer, log, batch_size, poll_timeout, intermediate_commit, stream_cancelled); } From bf2654e0930f465b929fd28bfd43be5307d8b235 Mon Sep 17 00:00:00 2001 From: sfod Date: Fri, 20 Sep 2019 17:39:26 +0300 Subject: [PATCH 072/108] Use sql file to test changes --- .../01013_totals_without_aggregation.reference | 10 +++++++--- .../01013_totals_without_aggregation.sh | 15 --------------- .../01013_totals_without_aggregation.sql | 6 ++++++ 3 files changed, 13 insertions(+), 18 deletions(-) delete mode 100755 dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh create mode 100755 dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sql diff --git a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference index 7614df8ec46..6dddf22a467 100644 --- a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference +++ b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.reference @@ -1,3 +1,7 @@ -ok -ok -ok +11 + +11 +12 +12 +13 +13 diff --git a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh deleted file mode 100755 index c159a73388d..00000000000 --- a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sh +++ /dev/null @@ -1,15 +0,0 @@ -#!/usr/bin/env bash - -CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) -. $CURDIR/../shell_config.sh - -EXCEPTION_SUCCESS_TEXT=ok - -# Must throw an exception -EXCEPTION_TEXT="WITH TOTALS, ROLLUP or CUBE are not supported without aggregation" -$CLICKHOUSE_CLIENT --query="SELECT 1 AS id, 'hello' AS s WITH TOTALS" 2>&1 \ - | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" -$CLICKHOUSE_CLIENT --query="SELECT 1 AS id, 'hello' AS s WITH ROLLUP" 2>&1 \ - | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" -$CLICKHOUSE_CLIENT --query="SELECT 1 AS id, 'hello' AS s WITH CUBE" 2>&1 \ - | grep -q "$EXCEPTION_TEXT" && echo "$EXCEPTION_SUCCESS_TEXT" || echo "Did not throw an exception" diff --git a/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sql b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sql new file mode 100755 index 00000000000..bed393b63d3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_totals_without_aggregation.sql @@ -0,0 +1,6 @@ +SELECT 11 AS n GROUP BY n WITH TOTALS; +SELECT 12 AS n GROUP BY n WITH ROLLUP; +SELECT 13 AS n GROUP BY n WITH CUBE; +SELECT 1 AS n WITH TOTALS; -- { serverError 49 } +SELECT 1 AS n WITH ROLLUP; -- { serverError 49 } +SELECT 1 AS n WITH CUBE; -- { serverError 49 } From bc9bcfdb7ee79de92e16e47dc26196b6f93ad061 Mon Sep 17 00:00:00 2001 From: sfod Date: Fri, 20 Sep 2019 17:42:30 +0300 Subject: [PATCH 073/108] Fix tests which use WITH TOTALS without aggregation --- .../00378_json_quote_64bit_integers.reference | 20 +++++++++---------- .../00378_json_quote_64bit_integers.sql | 12 +++++------ .../00937_template_output_format.reference | 4 ++-- .../00937_template_output_format.sql | 2 +- 4 files changed, 19 insertions(+), 19 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference b/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference index 22395188fe6..49c937e09df 100644 --- a/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference +++ b/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.reference @@ -48,10 +48,10 @@ { "i0": "0", "u0": "0", - "ip": "0", - "in": "0", - "up": "0", - "arr": [], + "ip": "9223372036854775807", + "in": "-9223372036854775808", + "up": "18446744073709551615", + "arr": ["0"], "tuple": ["0","0"] }, @@ -119,7 +119,7 @@ ["0", "0", "9223372036854775807", "-9223372036854775808", "18446744073709551615", ["0"], ["0","0"]] ], - "totals": ["0","0","0","0","0",[],["0","0"]], + "totals": ["0","0","9223372036854775807","-9223372036854775808","18446744073709551615",["0"],["0","0"]], "extremes": { @@ -180,10 +180,10 @@ { "i0": 0, "u0": 0, - "ip": 0, - "in": 0, - "up": 0, - "arr": [], + "ip": 9223372036854775807, + "in": -9223372036854775808, + "up": 18446744073709551615, + "arr": [0], "tuple": [0,0] }, @@ -251,7 +251,7 @@ [0, 0, 9223372036854775807, -9223372036854775808, 18446744073709551615, [0], [0,0]] ], - "totals": [0,0,0,0,0,[],[0,0]], + "totals": [0,0,9223372036854775807,-9223372036854775808,18446744073709551615,[0],[0,0]], "extremes": { diff --git a/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql b/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql index 261a044c711..2d99202a8ac 100644 --- a/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql +++ b/dbms/tests/queries/0_stateless/00378_json_quote_64bit_integers.sql @@ -2,11 +2,11 @@ SET output_format_write_statistics = 0; SET extremes = 1; SET output_format_json_quote_64bit_integers = 1; -SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple WITH TOTALS FORMAT JSON; -SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple WITH TOTALS FORMAT JSONCompact; -SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple WITH TOTALS FORMAT JSONEachRow; +SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSON; +SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSONCompact; +SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSONEachRow; SET output_format_json_quote_64bit_integers = 0; -SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple WITH TOTALS FORMAT JSON; -SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple WITH TOTALS FORMAT JSONCompact; -SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple WITH TOTALS FORMAT JSONEachRow; \ No newline at end of file +SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSON; +SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSONCompact; +SELECT toInt64(0) as i0, toUInt64(0) as u0, toInt64(9223372036854775807) as ip, toInt64(-9223372036854775808) as in, toUInt64(18446744073709551615) as up, [toInt64(0)] as arr, (toUInt64(0), toUInt64(0)) as tuple GROUP BY i0, u0, ip, in, up, arr, tuple WITH TOTALS FORMAT JSONEachRow; diff --git a/dbms/tests/queries/0_stateless/00937_template_output_format.reference b/dbms/tests/queries/0_stateless/00937_template_output_format.reference index c4cfb4ed3a4..c7b81d262ae 100644 --- a/dbms/tests/queries/0_stateless/00937_template_output_format.reference +++ b/dbms/tests/queries/0_stateless/00937_template_output_format.reference @@ -2,8 +2,8 @@ n: "123", s1: qwe,rty, s2: 'as"df\'gh', s3: "", s4: "zx cv bn m", d: 2016-01-01, n: 123 ; n: "456", s1: as"df\'gh, s2: '', s3: "zx\ncv\tbn m", s4: "qwe,rty", d: 2016-01-02, n: 456 ; -n: "9876543210", s1: , s2: 'zx\ncv\tbn m', s3: "qwe,rty", s4: "as""df'gh", d: 2016-01-03, n: 9876543210 ; -n: "789", s1: zx\ncv\tbn m, s2: 'qwe,rty', s3: "as\"df'gh", s4: "", d: 2016-01-04, n: 789 +n: "789", s1: zx\ncv\tbn m, s2: 'qwe,rty', s3: "as\"df'gh", s4: "", d: 2016-01-04, n: 789 ; +n: "9876543210", s1: , s2: 'zx\ncv\tbn m', s3: "qwe,rty", s4: "as""df'gh", d: 2016-01-03, n: 9876543210 ------ n: "0", s1: , s2: '', s3: "", s4: "", d: 0000-00-00, n: 0 ------ diff --git a/dbms/tests/queries/0_stateless/00937_template_output_format.sql b/dbms/tests/queries/0_stateless/00937_template_output_format.sql index 7a981c641da..40312272ccb 100644 --- a/dbms/tests/queries/0_stateless/00937_template_output_format.sql +++ b/dbms/tests/queries/0_stateless/00937_template_output_format.sql @@ -3,7 +3,7 @@ CREATE TABLE template (s1 String, s2 String, `s 3` String, "s 4" String, n UInt6 INSERT INTO template VALUES ('qwe,rty', 'as"df''gh', '', 'zx\ncv\tbn m', 123, '2016-01-01'),('as"df''gh', '', 'zx\ncv\tbn m', 'qwe,rty', 456, '2016-01-02'),('', 'zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', 9876543210, '2016-01-03'),('zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', '', 789, '2016-01-04'); -SELECT * FROM template WITH TOTALS LIMIT 4 FORMAT Template SETTINGS +SELECT * FROM template GROUP BY s1, s2, `s 3`, "s 4", n, d WITH TOTALS ORDER BY n LIMIT 4 FORMAT Template SETTINGS extremes = 1, format_schema = '{prefix} \n${data:None}\n------\n${totals:}\n------\n${min}\n------\n${max}\n${rows:Escaped} rows\nbefore limit ${rows_before_limit:XML}\nread ${rows_read:Escaped} $$ suffix $$', format_schema_rows = 'n:\t${n:JSON}, s1:\t${s1:Escaped}, s2:\t${s2:Quoted}, s3:\t${`s 3`:JSON}, s4:\t${"s 4":CSV}, d:\t${d:Escaped}, n:\t${n:Raw}\t', From b64bab29ed1a329f6f7414f742c2a3a94f4bcf77 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 20 Sep 2019 22:45:25 +0800 Subject: [PATCH 074/108] Outline methods to get needed instantiations. --- dbms/src/DataTypes/DataTypeLowCardinality.cpp | 68 +++++++++++++++ dbms/src/DataTypes/DataTypeLowCardinality.h | 83 ++++--------------- 2 files changed, 82 insertions(+), 69 deletions(-) diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index b0f9d312773..57ff63483c1 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -742,6 +742,74 @@ void DataTypeLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) dictionary_type->deserializeBinary(field, istr); } +void DataTypeLowCardinality::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const +{ + serializeImpl(column, row_num, &IDataType::serializeBinary, ostr); +} +void DataTypeLowCardinality::deserializeBinary(IColumn & column, ReadBuffer & istr) const +{ + deserializeImpl(column, &IDataType::deserializeBinary, istr); +} + +void DataTypeLowCardinality::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &IDataType::serializeAsTextEscaped, ostr, settings); +} + +void DataTypeLowCardinality::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings); +} + +void DataTypeLowCardinality::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &IDataType::serializeAsTextQuoted, ostr, settings); +} + +void DataTypeLowCardinality::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeImpl(column, &IDataType::deserializeAsTextQuoted, istr, settings); +} + +void DataTypeLowCardinality::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings); +} + +void DataTypeLowCardinality::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &IDataType::serializeAsTextCSV, ostr, settings); +} + +void DataTypeLowCardinality::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeImpl(column, &IDataType::deserializeAsTextCSV, istr, settings); +} + +void DataTypeLowCardinality::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &IDataType::serializeAsText, ostr, settings); +} + +void DataTypeLowCardinality::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &IDataType::serializeAsTextJSON, ostr, settings); +} +void DataTypeLowCardinality::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const +{ + deserializeImpl(column, &IDataType::deserializeAsTextJSON, istr, settings); +} + +void DataTypeLowCardinality::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + serializeImpl(column, row_num, &IDataType::serializeAsTextXML, ostr, settings); +} + +void DataTypeLowCardinality::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const +{ + serializeImpl(column, row_num, &IDataType::serializeProtobuf, protobuf, value_index); +} + void DataTypeLowCardinality::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const { if (allow_add_row) diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.h b/dbms/src/DataTypes/DataTypeLowCardinality.h index 8e6e12fadba..638af7bb968 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.h +++ b/dbms/src/DataTypes/DataTypeLowCardinality.h @@ -51,75 +51,20 @@ public: void serializeBinary(const Field & field, WriteBuffer & ostr) const override; void deserializeBinary(Field & field, ReadBuffer & istr) const override; - - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override - { - serializeImpl(column, row_num, &IDataType::serializeBinary, ostr); - } - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override - { - deserializeImpl(column, &IDataType::deserializeBinary, istr); - } - - void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, &IDataType::serializeAsTextEscaped, ostr, settings); - } - - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings); - } - - void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, &IDataType::serializeAsTextQuoted, ostr, settings); - } - - void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, &IDataType::deserializeAsTextQuoted, istr, settings); - } - - void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, &IDataType::deserializeAsTextEscaped, istr, settings); - } - - void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, &IDataType::serializeAsTextCSV, ostr, settings); - } - - void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, &IDataType::deserializeAsTextCSV, istr, settings); - } - - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, &IDataType::serializeAsText, ostr, settings); - } - - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, &IDataType::serializeAsTextJSON, ostr, settings); - } - void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, &IDataType::deserializeAsTextJSON, istr, settings); - } - - void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, &IDataType::serializeAsTextXML, ostr, settings); - } - - void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override - { - serializeImpl(column, row_num, &IDataType::serializeProtobuf, protobuf, value_index); - } - + void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; + void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; + void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override; + void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override; + void serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const override; void deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const override; MutableColumnPtr createColumn() const override; From 7726130303494380f6b28db6c1f87a7fa0c54578 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 20 Sep 2019 19:01:19 +0300 Subject: [PATCH 075/108] fix multiple joins aliasing for order by and group by --- .../JoinToSubqueryTransformVisitor.cpp | 21 ++++++----- .../00847_multiple_join_same_column.sql | 28 +++++++++++---- .../00882_multiple_join_no_alias.reference | 8 +++++ .../00882_multiple_join_no_alias.sql | 35 +++++++++++++++++++ 4 files changed, 77 insertions(+), 15 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.reference create mode 100644 dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.sql diff --git a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp index c6e72b4d252..b60e6533921 100644 --- a/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp +++ b/dbms/src/Interpreters/JoinToSubqueryTransformVisitor.cpp @@ -194,14 +194,14 @@ struct ColumnAliasesMatcher } }; - static bool needChildVisit(ASTPtr & node, const ASTPtr &) + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { if (node->as()) return false; return true; } - static void visit(ASTPtr & ast, Data & data) + static void visit(const ASTPtr & ast, Data & data) { if (auto * t = ast->as()) visit(*t, ast, data); @@ -210,8 +210,9 @@ struct ColumnAliasesMatcher throw Exception("Multiple JOIN do not support asterisks for complex queries yet", ErrorCodes::NOT_IMPLEMENTED); } - static void visit(ASTIdentifier & node, ASTPtr &, Data & data) + static void visit(const ASTIdentifier & const_node, const ASTPtr &, Data & data) { + ASTIdentifier & node = const_cast(const_node); /// we know it's not const if (node.isShort()) return; @@ -375,7 +376,7 @@ using RewriteVisitor = InDepthNodeVisitor; using SetSubqueryAliasMatcher = OneTypeMatcher; using SetSubqueryAliasVisitor = InDepthNodeVisitor; using ExtractAsterisksVisitor = ExtractAsterisksMatcher::Visitor; -using ColumnAliasesVisitor = InDepthNodeVisitor; +using ColumnAliasesVisitor = ConstInDepthNodeVisitor; using AppendSemanticMatcher = OneTypeMatcher; using AppendSemanticVisitor = InDepthNodeVisitor; @@ -403,15 +404,19 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast if (select.select()) { aliases_data.public_names = true; - ColumnAliasesVisitor(aliases_data).visit(select.refSelect()); + ColumnAliasesVisitor(aliases_data).visit(select.select()); aliases_data.public_names = false; } if (select.where()) - ColumnAliasesVisitor(aliases_data).visit(select.refWhere()); + ColumnAliasesVisitor(aliases_data).visit(select.where()); if (select.prewhere()) - ColumnAliasesVisitor(aliases_data).visit(select.refPrewhere()); + ColumnAliasesVisitor(aliases_data).visit(select.prewhere()); + if (select.orderBy()) + ColumnAliasesVisitor(aliases_data).visit(select.orderBy()); + if (select.groupBy()) + ColumnAliasesVisitor(aliases_data).visit(select.groupBy()); if (select.having()) - ColumnAliasesVisitor(aliases_data).visit(select.refHaving()); + ColumnAliasesVisitor(aliases_data).visit(select.having()); /// JOIN sections for (auto & child : select.tables()->children) diff --git a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql index d444655a6ce..44b3fe202d3 100644 --- a/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql +++ b/dbms/tests/queries/0_stateless/00847_multiple_join_same_column.sql @@ -16,30 +16,44 @@ left join y on (y.a = s.a and y.b = s.b) format Vertical; select t.a, s.b, s.a, s.b, y.a, y.b from t left join s on (t.a = s.a and s.b = t.b) -left join y on (y.a = s.a and y.b = s.b) format PrettyCompactNoEscapes; +left join y on (y.a = s.a and y.b = s.b) +order by t.a +format PrettyCompactNoEscapes; select t.a as t_a from t -left join s on s.a = t_a format PrettyCompactNoEscapes; +left join s on s.a = t_a +order by t.a +format PrettyCompactNoEscapes; select t.a, s.a as s_a from t left join s on s.a = t.a -left join y on y.b = s.b format PrettyCompactNoEscapes; +left join y on y.b = s.b +order by t.a +format PrettyCompactNoEscapes; select t.a, t.a, t.b as t_b from t left join s on t.a = s.a -left join y on y.b = s.b format PrettyCompactNoEscapes; +left join y on y.b = s.b +order by t.a +format PrettyCompactNoEscapes; select s.a, s.a, s.b as s_b, s.b from t left join s on s.a = t.a -left join y on s.b = y.b format PrettyCompactNoEscapes; +left join y on s.b = y.b +order by t.a +format PrettyCompactNoEscapes; select y.a, y.a, y.b as y_b, y.b from t left join s on s.a = t.a -left join y on y.b = s.b format PrettyCompactNoEscapes; +left join y on y.b = s.b +order by t.a +format PrettyCompactNoEscapes; select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t left join s on t.a = s.a -left join y on y.b = s.b format PrettyCompactNoEscapes; +left join y on y.b = s.b +order by t.a +format PrettyCompactNoEscapes; drop table t; drop table s; diff --git a/dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.reference b/dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.reference new file mode 100644 index 00000000000..a3723bc9976 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.reference @@ -0,0 +1,8 @@ +1 1 1 1 +0 0 0 0 +0 +1 +1 1 1 1 1 1 +2 2 0 0 0 0 +2 2 0 +1 1 1 diff --git a/dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.sql b/dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.sql new file mode 100644 index 00000000000..bd3a2a19913 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00882_multiple_join_no_alias.sql @@ -0,0 +1,35 @@ +drop table if exists t; +drop table if exists s; +drop table if exists y; + +create table t(a Int64, b Int64) engine = Memory; +create table s(a Int64, b Int64) engine = Memory; +create table y(a Int64, b Int64) engine = Memory; + +insert into t values (1,1), (2,2); +insert into s values (1,1); +insert into y values (1,1); + +select s.a, s.a, s.b as s_b, s.b from t +left join s on s.a = t.a +left join y on s.b = y.b +order by t.a; + +select max(s.a) from t +left join s on s.a = t.a +left join y on s.b = y.b +group by t.a; + +select t.a, t.a as t_a, s.a, s.a as s_a, y.a, y.a as y_a from t +left join s on t.a = s.a +left join y on y.b = s.b +order by t.a; + +select t.a, t.a as t_a, max(s.a) from t +left join s on t.a = s.a +left join y on y.b = s.b +group by t.a; + +drop table t; +drop table s; +drop table y; From 9eb0f8c536b6b1fc6c5ef69b843ea6e5529ef0be Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Sep 2019 19:13:40 +0300 Subject: [PATCH 076/108] Update CHANGELOG.md --- CHANGELOG.md | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 5d1d83f4c9c..d1d88b22e89 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,13 @@ +## ClickHouse release 19.14.6.12, 2019-09-19 + +### Bug Fix +* Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/yandex/ClickHouse/pull/6928) ([proller](https://github.com/proller)) +* Fixed subquery name in queries with `ARRAY JOIN` and `GLOBAL IN subquery` with alias. Use subquery alias for external table name if it is specified. [#6934](https://github.com/yandex/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) +* Fixed UBSan and MemSan failure in function `groupUniqArray` with emtpy array argument. It was caused by placing of empty `PaddedPODArray` into hash table zero cell because constructor for zero cell value was not called. [#6937](https://github.com/yandex/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) + +### Build/Testing/Packaging Improvement +* Fix [flapping](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) test `00715_fetch_merged_or_mutated_part_zookeeper` by rewriting it to a shell scripts because it needs to wait for mutations to apply. [#6977](https://github.com/yandex/ClickHouse/pull/6977) ([Alexander Kazakov](https://github.com/Akazz)) + ## ClickHouse release 19.14.3.3, 2019-09-10 ### New Feature From 26e2a574b47d34c74ba3bdc3fea55311a3bef974 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Sep 2019 19:21:48 +0300 Subject: [PATCH 077/108] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d1d88b22e89..bc2370049ec 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,10 +3,10 @@ ### Bug Fix * Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/yandex/ClickHouse/pull/6928) ([proller](https://github.com/proller)) * Fixed subquery name in queries with `ARRAY JOIN` and `GLOBAL IN subquery` with alias. Use subquery alias for external table name if it is specified. [#6934](https://github.com/yandex/ClickHouse/pull/6934) ([Ivan](https://github.com/abyss7)) -* Fixed UBSan and MemSan failure in function `groupUniqArray` with emtpy array argument. It was caused by placing of empty `PaddedPODArray` into hash table zero cell because constructor for zero cell value was not called. [#6937](https://github.com/yandex/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) ### Build/Testing/Packaging Improvement * Fix [flapping](https://clickhouse-test-reports.s3.yandex.net/6944/aab95fd5175a513413c7395a73a82044bdafb906/functional_stateless_tests_(debug).html) test `00715_fetch_merged_or_mutated_part_zookeeper` by rewriting it to a shell scripts because it needs to wait for mutations to apply. [#6977](https://github.com/yandex/ClickHouse/pull/6977) ([Alexander Kazakov](https://github.com/Akazz)) +* Fixed UBSan and MemSan failure in function `groupUniqArray` with emtpy array argument. It was caused by placing of empty `PaddedPODArray` into hash table zero cell because constructor for zero cell value was not called. [#6937](https://github.com/yandex/ClickHouse/pull/6937) ([Amos Bird](https://github.com/amosbird)) ## ClickHouse release 19.14.3.3, 2019-09-10 From 8783798b99d0a72f766fed10b4b25b343056da91 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 20 Sep 2019 19:29:06 +0300 Subject: [PATCH 078/108] Move so libraries to output directory in binary build image --- docker/packager/binary/build.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index c31fb592782..c5b2e260e45 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -12,3 +12,5 @@ ninja ccache --show-stats ||: mv ./dbms/programs/clickhouse* /output mv ./dbms/unit_tests_dbms /output +find . -name '*.so' -print -exec mv '{}' /output \; +find . -name '*.so.*' -print -exec mv '{}' /output \; From 589a3c97438dd5e0c44552cbe223d88f841c0bb0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Sep 2019 20:34:40 +0300 Subject: [PATCH 079/108] Update CHANGELOG.md --- CHANGELOG.md | 47 ++++++++++++++++++++++++++++------------------- 1 file changed, 28 insertions(+), 19 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bc2370049ec..ffe6a0e08e8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -41,6 +41,7 @@ * Implementation of `LIVE VIEW` tables that were originally proposed in [#2898](https://github.com/yandex/ClickHouse/pull/2898), prepared in [#3925](https://github.com/yandex/ClickHouse/issues/3925), and then updated in [#5541](https://github.com/yandex/ClickHouse/issues/5541). See [#5541](https://github.com/yandex/ClickHouse/issues/5541) for detailed description. [#5541](https://github.com/yandex/ClickHouse/issues/5541) ([vzakaznikov](https://github.com/vzakaznikov)) [#6425](https://github.com/yandex/ClickHouse/pull/6425) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) [#6656](https://github.com/yandex/ClickHouse/pull/6656) ([vzakaznikov](https://github.com/vzakaznikov)) Note that `LIVE VIEW` feature may be removed in next versions. ### Bug Fix +* This release also contains all bug fixes from 19.13 and 19.11. * Fix segmentation fault when the table has skip indices and vertical merge happens. [#6723](https://github.com/yandex/ClickHouse/pull/6723) ([alesapin](https://github.com/alesapin)) * Fix per-column TTL with non-trivial column defaults. Previously in case of force TTL merge with `OPTIMIZE ... FINAL` query, expired values was replaced by type defaults instead of user-specified column defaults. [#6796](https://github.com/yandex/ClickHouse/pull/6796) ([Anton Popov](https://github.com/CurtizJ)) * Fix Kafka messages duplication problem on normal server restart. [#6597](https://github.com/yandex/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) @@ -55,13 +56,10 @@ * Fixed possible incomplete result returned by `SELECT` query with `WHERE` condition on primary key contained conversion to Float type. It was caused by incorrect checking of monotonicity in `toFloat` function. [#6248](https://github.com/yandex/ClickHouse/issues/6248) [#6374](https://github.com/yandex/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) * Check `max_expanded_ast_elements` setting for mutations. Clear mutations after `TRUNCATE TABLE`. [#6205](https://github.com/yandex/ClickHouse/pull/6205) ([Winter Zhang](https://github.com/zhang2014)) * Fix excessive CPU usage while executing `JSONExtractRaw` function over a boolean value. [#6208](https://github.com/yandex/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fixed an issue when long `ALTER UPDATE` or `ALTER DELETE` may prevent regular merges to run. Prevent mutations from executing if there is no enough free threads available. [#6502](https://github.com/yandex/ClickHouse/issues/6502) [#6617](https://github.com/yandex/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) * Fix JOIN results for key columns when used with `join_use_nulls`. Attach Nulls instead of columns defaults. [#6249](https://github.com/yandex/ClickHouse/pull/6249) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix `JSONExtract` function while extracting a `Tuple` from JSON. [#6718](https://github.com/yandex/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) * Fix for data race in StorageMerge [#6717](https://github.com/yandex/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix for skip indices with vertical merge and alter. Fix for `Bad size of marks file` exception. [#6594](https://github.com/yandex/ClickHouse/issues/6594) [#6713](https://github.com/yandex/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) * Fix rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows) [#6746](https://github.com/yandex/ClickHouse/issues/6746) [#6780](https://github.com/yandex/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) -* Fixed wrong behaviour of `nullIf` function for constant arguments. [#6518](https://github.com/yandex/ClickHouse/pull/6518) ([Guillaume Tassery](https://github.com/YiuRULE)) [#6580](https://github.com/yandex/ClickHouse/pull/6580) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [#6257](https://github.com/yandex/ClickHouse/issues/6257). [#6281](https://github.com/yandex/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/yandex/ClickHouse/issues/6224) [#6282](https://github.com/yandex/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) * Fix wrong behavior and possible segfaults in `topK` and `topKWeighted` aggregated functions. [#6404](https://github.com/yandex/ClickHouse/pull/6404) ([Anton Popov](https://github.com/CurtizJ)) @@ -73,42 +71,30 @@ * Resolve a bug with `nullIf` function when we send a `NULL` argument on the second argument. [#6446](https://github.com/yandex/ClickHouse/pull/6446) ([Guillaume Tassery](https://github.com/YiuRULE)) * Fix rare bug with wrong memory allocation/deallocation in complex key cache dictionaries with string fields which leads to infinite memory consumption (looks like memory leak). Bug reproduces when string size was a power of two starting from eight (8, 16, 32, etc). [#6447](https://github.com/yandex/ClickHouse/pull/6447) ([alesapin](https://github.com/alesapin)) * Fixed Gorilla encoding on small sequences which caused exception `Cannot write after end of buffer`. [#6398](https://github.com/yandex/ClickHouse/issues/6398) [#6444](https://github.com/yandex/ClickHouse/pull/6444) ([Vasily Nemkov](https://github.com/Enmk)) -* Fixed error with processing "timezone" in server configuration file. [#6709](https://github.com/yandex/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Allow to use not nullable types in JOINs with `join_use_nulls` enabled. [#6705](https://github.com/yandex/ClickHouse/pull/6705) ([Artem Zuikov](https://github.com/4ertus2)) * Disable `Poco::AbstractConfiguration` substitutions in query in `clickhouse-client`. [#6706](https://github.com/yandex/ClickHouse/pull/6706) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and prewhere. [#6167](https://github.com/yandex/ClickHouse/issues/6167) ([Lixiang Qian](https://github.com/fancyqlx)) [#6823](https://github.com/yandex/ClickHouse/pull/6823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Avoid deadlock in `REPLACE PARTITION`. [#6677](https://github.com/yandex/ClickHouse/pull/6677) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Query transformation for `MySQL`, `ODBC`, `JDBC` table functions now works properly for `SELECT WHERE` queries with multiple `AND` expressions. [#6381](https://github.com/yandex/ClickHouse/issues/6381) [#6676](https://github.com/yandex/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) -* Fixed bug in function `arrayEnumerateUniqRanked`. [#6779](https://github.com/yandex/ClickHouse/pull/6779) ([proller](https://github.com/proller)) * Fixed parsing of `AggregateFunction` values embedded in query. [6575](https://github.com/yandex/ClickHouse/issues/6575) [#6773](https://github.com/yandex/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) * Using `arrayReduce` for constant arguments may lead to segfault. [#6242](https://github.com/yandex/ClickHouse/issues/6242) [#6326](https://github.com/yandex/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix inconsistent parts which can appear if replica was restored after `DROP PARTITION`. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) -* Fix crash when casting types to `Decimal` that do not support it. Throw exception instead. [#6297](https://github.com/yandex/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) * Fixed hang in `JSONExtractRaw` function. [#6195](https://github.com/yandex/ClickHouse/issues/6195) [#6198](https://github.com/yandex/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed crash when using `IN` clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) * Fixes the regression while pushing to materialized view. [#6415](https://github.com/yandex/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) * Fixed possible inconsistent state of table while executing `DROP` query for replicated table while zookeeper is not accessible. [#6045](https://github.com/yandex/ClickHouse/issues/6045) [#6413](https://github.com/yandex/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) * Fix bug with incorrect skip indices serialization and aggregation with adaptive granularity. [#6594](https://github.com/yandex/ClickHouse/issues/6594). [#6748](https://github.com/yandex/ClickHouse/pull/6748) ([alesapin](https://github.com/alesapin)) * Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [#6225](https://github.com/yandex/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) -* Improve error handling in cache dictionaries. [#6737](https://github.com/yandex/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) -* Parquet: Fix reading boolean columns. [#6579](https://github.com/yandex/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix bug with writing secondary indices marks with adaptive granularity. [#6126](https://github.com/yandex/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) * Fix initialization order while server startup. Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [#6080](https://github.com/yandex/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) -* Fixed crash in `extractAll()` function. [#6644](https://github.com/yandex/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) * Fixed wrong behaviour of `trim` functions family. [#6647](https://github.com/yandex/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Clearing the data buffer from the previous read operation that was completed with an error. [#6026](https://github.com/yandex/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) * Fix bug with enabling adaptive granularity when creating new replica for Replicated*MergeTree table. [#6394](https://github.com/yandex/ClickHouse/issues/6394) [#6452](https://github.com/yandex/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) * Fixed possible crash during server startup in case of exception happened in `libunwind` during exception at access to uninitialised `ThreadStatus` structure. [#6456](https://github.com/yandex/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) -* Fixed data race in `system.parts` table and `ALTER` query. [#6245](https://github.com/yandex/ClickHouse/issues/6245). [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix crash in `yandexConsistentHash` function. Found by fuzz test. [#6304](https://github.com/yandex/ClickHouse/issues/6304) [#6305](https://github.com/yandex/ClickHouse/pull/6305) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed the possibility of hanging queries when server is overloaded and global thread pool becomes near full. This have higher chance to happen on clusters with large number of shards (hundreds), because distributed queries allocate a thread per connection to each shard. For example, this issue may reproduce if a cluster of 330 shards is processing 30 concurrent distributed queries. This issue affects all versions starting from 19.2. [#6301](https://github.com/yandex/ClickHouse/pull/6301) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed logic of `arrayEnumerateUniqRanked` function. [#6423](https://github.com/yandex/ClickHouse/pull/6423) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix segfault when decoding symbol table. [#6603](https://github.com/yandex/ClickHouse/pull/6603) ([Amos Bird](https://github.com/amosbird)) -* Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and prewhere. [#6167](https://github.com/yandex/ClickHouse/pull/6167) ([Lixiang Qian](https://github.com/fancyqlx)) * Fixed irrelevant exception in cast of `LowCardinality(Nullable)` to not-Nullable column in case if it doesn't contain Nulls (e.g. in query like `SELECT CAST(CAST('Hello' AS LowCardinality(Nullable(String))) AS String)`. [#6094](https://github.com/yandex/ClickHouse/issues/6094) [#6119](https://github.com/yandex/ClickHouse/pull/6119) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Removed extra quoting of description in `system.settings` table. [#6696](https://github.com/yandex/ClickHouse/issues/6696) [#6699](https://github.com/yandex/ClickHouse/pull/6699) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Avoid possible deadlock in `TRUNCATE` of Replicated table. [#6695](https://github.com/yandex/ClickHouse/pull/6695) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix case with same column names in `GLOBAL JOIN ON` section. [#6181](https://github.com/yandex/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) * Fix reading in order of sorting key. [#6189](https://github.com/yandex/ClickHouse/pull/6189) ([Anton Popov](https://github.com/CurtizJ)) * Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) * Fixed the case when server may close listening sockets but not shutdown and continue serving remaining queries. You may end up with two running clickhouse-server processes. Sometimes, the server may return an error `bad_function_call` for remaining queries. [#6231](https://github.com/yandex/ClickHouse/pull/6231) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -124,7 +110,7 @@ * Fixed error while parsing of columns list from string if type contained a comma (this issue was relevant for `File`, `URL`, `HDFS` storages) [#6217](https://github.com/yandex/ClickHouse/issues/6217). [#6209](https://github.com/yandex/ClickHouse/pull/6209) ([dimarub2000](https://github.com/dimarub2000)) ### Security Fix -* Fix two vulnerabilities in codecs in decompression phase (malicious user can fabricate compressed data that will lead to buffer overflow in decompression). [#6670](https://github.com/yandex/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) +* This release also contains all bug security fixes from 19.13 and 19.11. * If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/yandex/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser. Fixed the possibility of stack overflow in Merge and Distributed tables, materialized views and conditions for row-level security that involve subqueries. [#6433](https://github.com/yandex/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) @@ -240,7 +226,6 @@ * `odbc-bridge.cpp` defines `main()` so it should not be included in `clickhouse-lib`. [#6538](https://github.com/yandex/ClickHouse/pull/6538) ([Orivej Desh](https://github.com/orivej)) * Test for crash in `FULL|RIGHT JOIN` with nulls in right table's keys. [#6362](https://github.com/yandex/ClickHouse/pull/6362) ([Artem Zuikov](https://github.com/4ertus2)) * Added a test for the limit on expansion of aliases just in case. [#6442](https://github.com/yandex/ClickHouse/pull/6442) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Added previous declaration checks for MySQL 8 integration. [#6569](https://github.com/yandex/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) * Switched from `boost::filesystem` to `std::filesystem` where appropriate. [#6253](https://github.com/yandex/ClickHouse/pull/6253) [#6385](https://github.com/yandex/ClickHouse/pull/6385) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Added RPM packages to website. [#6251](https://github.com/yandex/ClickHouse/pull/6251) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Add a test for fixed `Unknown identifier` exception in `IN` section. [#6708](https://github.com/yandex/ClickHouse/pull/6708) ([Artem Zuikov](https://github.com/4ertus2)) @@ -268,7 +253,6 @@ * Fixed tests affected by slow stack traces printing. [#6315](https://github.com/yandex/ClickHouse/pull/6315) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Add a test case for crash in `groupUniqArray` fixed in [#6029](https://github.com/yandex/ClickHouse/pull/6029). [#4402](https://github.com/yandex/ClickHouse/issues/4402) [#6129](https://github.com/yandex/ClickHouse/pull/6129) ([akuzm](https://github.com/akuzm)) * Fixed indices mutations tests. [#6645](https://github.com/yandex/ClickHouse/pull/6645) ([Nikita Vasilev](https://github.com/nikvas0)) -* Attempt to fix performance test. [#6392](https://github.com/yandex/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) * In performance test, do not read query log for queries we didn't run. [#6427](https://github.com/yandex/ClickHouse/pull/6427) ([akuzm](https://github.com/akuzm)) * Materialized view now could be created with any low cardinality types regardless to the setting about suspicious low cardinality types. [#6428](https://github.com/yandex/ClickHouse/pull/6428) ([Olga Khvostikova](https://github.com/stavrolia)) * Updated tests for `send_logs_level` setting. [#6207](https://github.com/yandex/ClickHouse/pull/6207) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -278,12 +262,37 @@ * Fixes for Mac OS build (incomplete). [#6390](https://github.com/yandex/ClickHouse/pull/6390) ([alexey-milovidov](https://github.com/alexey-milovidov)) [#6429](https://github.com/yandex/ClickHouse/pull/6429) ([alex-zaitsev](https://github.com/alex-zaitsev)) * Fix "splitted" build. [#6618](https://github.com/yandex/ClickHouse/pull/6618) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Other build fixes: [#6186](https://github.com/yandex/ClickHouse/pull/6186) ([Amos Bird](https://github.com/amosbird)) [#6486](https://github.com/yandex/ClickHouse/pull/6486) [#6348](https://github.com/yandex/ClickHouse/pull/6348) ([vxider](https://github.com/Vxider)) [#6744](https://github.com/yandex/ClickHouse/pull/6744) ([Ivan](https://github.com/abyss7)) [#6016](https://github.com/yandex/ClickHouse/pull/6016) [#6421](https://github.com/yandex/ClickHouse/pull/6421) [#6491](https://github.com/yandex/ClickHouse/pull/6491) ([proller](https://github.com/proller)) -* Fix kafka tests. [#6805](https://github.com/yandex/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) ### Backward Incompatible Change * Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/yandex/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_get_any_from_right_table` setting to enable them. [#5126](https://github.com/yandex/ClickHouse/issues/5126) [#6351](https://github.com/yandex/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) +## ClickHouse release 19.13.4.32, 2019-09-10 + +### Bug Fix +* Fixed an issue when long `ALTER UPDATE` or `ALTER DELETE` may prevent regular merges to run. Prevent mutations from executing if there is no enough free threads available. [#6502](https://github.com/yandex/ClickHouse/issues/6502) [#6617](https://github.com/yandex/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) +* Fixed data race in `system.parts` table and `ALTER` query. [#6245](https://github.com/yandex/ClickHouse/issues/6245) [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix Kafka messages duplication problem on normal server restart. [#6597](https://github.com/yandex/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) +* Do store offsets for Kafka messages manually to be able to commit them all at once for all partitions. Fixes potential duplication in "one consumer - many partitions" scenario. [#6872](https://github.com/yandex/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) +* Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and prewhere. [#6167](https://github.com/yandex/ClickHouse/issues/6167) ([Lixiang Qian](https://github.com/fancyqlx)) [#6823](https://github.com/yandex/ClickHouse/pull/6823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) +* Fixed crash when using `IN` clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) +* Fixed bug in function `arrayEnumerateUniqRanked`. [#6779](https://github.com/yandex/ClickHouse/pull/6779) ([proller](https://github.com/proller)) +* Fix `JSONExtract` function while extracting a `Tuple` from JSON. [#6718](https://github.com/yandex/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix case with same column names in `GLOBAL JOIN ON` section. [#6181](https://github.com/yandex/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) +* Fix crash when casting types to `Decimal` that do not support it. Throw exception instead. [#6297](https://github.com/yandex/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) +* Fixed wrong behaviour of `nullIf` function for constant arguments. [#6518](https://github.com/yandex/ClickHouse/pull/6518) ([Guillaume Tassery](https://github.com/YiuRULE)) [#6580](https://github.com/yandex/ClickHouse/pull/6580) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed crash in `extractAll()` function. [#6644](https://github.com/yandex/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) +* Query transformation for `MySQL`, `ODBC`, `JDBC` table functions now works properly for `SELECT WHERE` queries with multiple `AND` expressions. [#6381](https://github.com/yandex/ClickHouse/issues/6381) [#6676](https://github.com/yandex/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) +* Added previous declaration checks for MySQL 8 integration. [#6569](https://github.com/yandex/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) +* Parquet: Fix reading boolean columns. [#6579](https://github.com/yandex/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed error with processing "timezone" in server configuration file. [#6709](https://github.com/yandex/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Improve error handling in cache dictionaries. [#6737](https://github.com/yandex/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fix kafka tests. [#6805](https://github.com/yandex/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) +* Fixed performance test. [#6392](https://github.com/yandex/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) + +### Security Fix +* Fix two vulnerabilities in codecs in decompression phase (malicious user can fabricate compressed data that will lead to buffer overflow in decompression). [#6670](https://github.com/yandex/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) + ## ClickHouse release 19.13.3.26, 2019-08-22 From 5b189d3828baff564738762c59c39e99d0a50874 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Sep 2019 21:48:19 +0300 Subject: [PATCH 080/108] Update CHANGELOG.md --- CHANGELOG.md | 23 ++++++++++++++--------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index ffe6a0e08e8..11ca2fd1301 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,17 @@ +## ClickHouse release 19.13.5.44, 2019-09-20 + +### Bug Fix +* This release also contains all bug fixes from 19.14.6.12. +* Fixed possible inconsistent state of table while executing `DROP` query for replicated table while zookeeper is not accessible. [#6045](https://github.com/yandex/ClickHouse/issues/6045) [#6413](https://github.com/yandex/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) +* Fix for data race in StorageMerge [#6717](https://github.com/yandex/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix bug introduced in query profiler which leads to endless recv from socket. [#6386](https://github.com/yandex/ClickHouse/pull/6386) ([alesapin](https://github.com/alesapin)) +* Fix excessive CPU usage while executing `JSONExtractRaw` function over a boolean value. [#6208](https://github.com/yandex/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fixes the regression while pushing to materialized view. [#6415](https://github.com/yandex/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) +* Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request. This issue was found by [Nikita Tikhomirov](https://github.com/NSTikhomirov). [#6466](https://github.com/yandex/ClickHouse/pull/6466) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix useless `AST` check in Set index. [#6510](https://github.com/yandex/ClickHouse/issues/6510) [#6651](https://github.com/yandex/ClickHouse/pull/6651) ([Nikita Vasilev](https://github.com/nikvas0)) +* Fixed parsing of `AggregateFunction` values embedded in query. [#6575](https://github.com/yandex/ClickHouse/issues/6575) [#6773](https://github.com/yandex/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) +* Fixed wrong behaviour of `trim` functions family. [#6647](https://github.com/yandex/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) + ## ClickHouse release 19.14.6.12, 2019-09-19 ### Bug Fix @@ -49,15 +63,12 @@ * Fix `Key expression contains comparison between inconvertible types` exception in `bitmapContains` function. [#6136](https://github.com/yandex/ClickHouse/issues/6136) [#6146](https://github.com/yandex/ClickHouse/issues/6146) [#6156](https://github.com/yandex/ClickHouse/pull/6156) ([dimarub2000](https://github.com/dimarub2000)) * Fix segfault with enabled `optimize_skip_unused_shards` and missing sharding key. [#6384](https://github.com/yandex/ClickHouse/pull/6384) ([Anton Popov](https://github.com/CurtizJ)) * Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/yandex/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix bug introduced in query profiler which leads to endless recv from socket. [#6386](https://github.com/yandex/ClickHouse/pull/6386) ([alesapin](https://github.com/alesapin)) * Removed extra verbose logging in MySQL interface [#6389](https://github.com/yandex/ClickHouse/pull/6389) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Return ability to parse boolean settings from 'true' and 'false' in configuration file. [#6278](https://github.com/yandex/ClickHouse/pull/6278) ([alesapin](https://github.com/alesapin)) * Fix crash in `quantile` and `median` function over `Nullable(Decimal128)`. [#6378](https://github.com/yandex/ClickHouse/pull/6378) ([Artem Zuikov](https://github.com/4ertus2)) * Fixed possible incomplete result returned by `SELECT` query with `WHERE` condition on primary key contained conversion to Float type. It was caused by incorrect checking of monotonicity in `toFloat` function. [#6248](https://github.com/yandex/ClickHouse/issues/6248) [#6374](https://github.com/yandex/ClickHouse/pull/6374) ([dimarub2000](https://github.com/dimarub2000)) * Check `max_expanded_ast_elements` setting for mutations. Clear mutations after `TRUNCATE TABLE`. [#6205](https://github.com/yandex/ClickHouse/pull/6205) ([Winter Zhang](https://github.com/zhang2014)) -* Fix excessive CPU usage while executing `JSONExtractRaw` function over a boolean value. [#6208](https://github.com/yandex/ClickHouse/pull/6208) ([Vitaly Baranov](https://github.com/vitlibar)) * Fix JOIN results for key columns when used with `join_use_nulls`. Attach Nulls instead of columns defaults. [#6249](https://github.com/yandex/ClickHouse/pull/6249) ([Artem Zuikov](https://github.com/4ertus2)) -* Fix for data race in StorageMerge [#6717](https://github.com/yandex/ClickHouse/pull/6717) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix for skip indices with vertical merge and alter. Fix for `Bad size of marks file` exception. [#6594](https://github.com/yandex/ClickHouse/issues/6594) [#6713](https://github.com/yandex/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) * Fix rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows) [#6746](https://github.com/yandex/ClickHouse/issues/6746) [#6780](https://github.com/yandex/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) * Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [#6257](https://github.com/yandex/ClickHouse/issues/6257). [#6281](https://github.com/yandex/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) @@ -74,17 +85,13 @@ * Allow to use not nullable types in JOINs with `join_use_nulls` enabled. [#6705](https://github.com/yandex/ClickHouse/pull/6705) ([Artem Zuikov](https://github.com/4ertus2)) * Disable `Poco::AbstractConfiguration` substitutions in query in `clickhouse-client`. [#6706](https://github.com/yandex/ClickHouse/pull/6706) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Avoid deadlock in `REPLACE PARTITION`. [#6677](https://github.com/yandex/ClickHouse/pull/6677) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed parsing of `AggregateFunction` values embedded in query. [6575](https://github.com/yandex/ClickHouse/issues/6575) [#6773](https://github.com/yandex/ClickHouse/pull/6773) ([Zhichang Yu](https://github.com/yuzhichang)) * Using `arrayReduce` for constant arguments may lead to segfault. [#6242](https://github.com/yandex/ClickHouse/issues/6242) [#6326](https://github.com/yandex/ClickHouse/pull/6326) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix inconsistent parts which can appear if replica was restored after `DROP PARTITION`. [#6522](https://github.com/yandex/ClickHouse/issues/6522) [#6523](https://github.com/yandex/ClickHouse/pull/6523) ([tavplubix](https://github.com/tavplubix)) * Fixed hang in `JSONExtractRaw` function. [#6195](https://github.com/yandex/ClickHouse/issues/6195) [#6198](https://github.com/yandex/ClickHouse/pull/6198) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixes the regression while pushing to materialized view. [#6415](https://github.com/yandex/ClickHouse/pull/6415) ([Ivan](https://github.com/abyss7)) -* Fixed possible inconsistent state of table while executing `DROP` query for replicated table while zookeeper is not accessible. [#6045](https://github.com/yandex/ClickHouse/issues/6045) [#6413](https://github.com/yandex/ClickHouse/pull/6413) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) * Fix bug with incorrect skip indices serialization and aggregation with adaptive granularity. [#6594](https://github.com/yandex/ClickHouse/issues/6594). [#6748](https://github.com/yandex/ClickHouse/pull/6748) ([alesapin](https://github.com/alesapin)) * Fix `WITH ROLLUP` and `WITH CUBE` modifiers of `GROUP BY` with two-level aggregation. [#6225](https://github.com/yandex/ClickHouse/pull/6225) ([Anton Popov](https://github.com/CurtizJ)) * Fix bug with writing secondary indices marks with adaptive granularity. [#6126](https://github.com/yandex/ClickHouse/pull/6126) ([alesapin](https://github.com/alesapin)) * Fix initialization order while server startup. Since `StorageMergeTree::background_task_handle` is initialized in `startup()` the `MergeTreeBlockOutputStream::write()` may try to use it before initialization. Just check if it is initialized. [#6080](https://github.com/yandex/ClickHouse/pull/6080) ([Ivan](https://github.com/abyss7)) -* Fixed wrong behaviour of `trim` functions family. [#6647](https://github.com/yandex/ClickHouse/pull/6647) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Clearing the data buffer from the previous read operation that was completed with an error. [#6026](https://github.com/yandex/ClickHouse/pull/6026) ([Nikolay](https://github.com/bopohaa)) * Fix bug with enabling adaptive granularity when creating new replica for Replicated*MergeTree table. [#6394](https://github.com/yandex/ClickHouse/issues/6394) [#6452](https://github.com/yandex/ClickHouse/pull/6452) ([alesapin](https://github.com/alesapin)) * Fixed possible crash during server startup in case of exception happened in `libunwind` during exception at access to uninitialised `ThreadStatus` structure. [#6456](https://github.com/yandex/ClickHouse/pull/6456) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)) @@ -98,11 +105,9 @@ * Fix reading in order of sorting key. [#6189](https://github.com/yandex/ClickHouse/pull/6189) ([Anton Popov](https://github.com/CurtizJ)) * Fix `ALTER TABLE ... UPDATE` query for tables with `enable_mixed_granularity_parts=1`. [#6543](https://github.com/yandex/ClickHouse/pull/6543) ([alesapin](https://github.com/alesapin)) * Fixed the case when server may close listening sockets but not shutdown and continue serving remaining queries. You may end up with two running clickhouse-server processes. Sometimes, the server may return an error `bad_function_call` for remaining queries. [#6231](https://github.com/yandex/ClickHouse/pull/6231) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Table function `url` had the vulnerability allowed the attacker to inject arbitrary HTTP headers in the request. This issue was found by [Nikita Tikhomirov](https://github.com/NSTikhomirov). [#6466](https://github.com/yandex/ClickHouse/pull/6466) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix bug opened by [#4405](https://github.com/yandex/ClickHouse/pull/4405) (since 19.4.0). Reproduces in queries to Distributed tables over MergeTree tables when we doesn't query any columns (`SELECT 1`). [#6236](https://github.com/yandex/ClickHouse/pull/6236) ([alesapin](https://github.com/alesapin)) * Fixed overflow in integer division of signed type to unsigned type. The behaviour was exactly as in C or C++ language (integer promotion rules) that may be surprising. Please note that the overflow is still possible when dividing large signed number to large unsigned number or vice-versa (but that case is less usual). The issue existed in all server versions. [#6214](https://github.com/yandex/ClickHouse/issues/6214) [#6233](https://github.com/yandex/ClickHouse/pull/6233) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Limit maximum sleep time for throttling when `max_execution_speed` or `max_execution_speed_bytes` is set. Fixed false errors like `Estimated query execution time (inf seconds) is too long`. [#5547](https://github.com/yandex/ClickHouse/issues/5547) [#6232](https://github.com/yandex/ClickHouse/pull/6232) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix useless `AST` check in Set index. [#6510](https://github.com/yandex/ClickHouse/issues/6510) [#6651](https://github.com/yandex/ClickHouse/pull/6651) ([Nikita Vasilev](https://github.com/nikvas0)) * Fixed issues about using `MATERIALIZED` columns and aliases in `MaterializedView`. [#448](https://github.com/yandex/ClickHouse/issues/448) [#3484](https://github.com/yandex/ClickHouse/issues/3484) [#3450](https://github.com/yandex/ClickHouse/issues/3450) [#2878](https://github.com/yandex/ClickHouse/issues/2878) [#2285](https://github.com/yandex/ClickHouse/issues/2285) [#3796](https://github.com/yandex/ClickHouse/pull/3796) ([Amos Bird](https://github.com/amosbird)) [#6316](https://github.com/yandex/ClickHouse/pull/6316) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fix `FormatFactory` behaviour for input streams which are not implemented as processor. [#6495](https://github.com/yandex/ClickHouse/pull/6495) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Fixed typo. [#6631](https://github.com/yandex/ClickHouse/pull/6631) ([Alex Ryndin](https://github.com/alexryndin)) From d0ef26e1b832e1a2450c2702e5be5afc9c1e78d1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 20 Sep 2019 22:57:09 +0300 Subject: [PATCH 081/108] Better shared binary build --- docker/packager/binary/Dockerfile | 3 ++- docker/packager/binary/build.sh | 3 +++ 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docker/packager/binary/Dockerfile b/docker/packager/binary/Dockerfile index 78f113d94b4..82fa93ec570 100644 --- a/docker/packager/binary/Dockerfile +++ b/docker/packager/binary/Dockerfile @@ -56,7 +56,8 @@ RUN apt-get update -y \ tzdata \ gperf \ cmake \ - gdb + gdb \ + rename COPY build.sh / CMD ["/bin/bash", "/build.sh"] diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index c5b2e260e45..9d994af0986 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -14,3 +14,6 @@ mv ./dbms/programs/clickhouse* /output mv ./dbms/unit_tests_dbms /output find . -name '*.so' -print -exec mv '{}' /output \; find . -name '*.so.*' -print -exec mv '{}' /output \; +tar -czvf shared_build.tgz /output +rm -r /output/* +mv shared_build.tgz /output From f2f9f585e42557c7b9e41fc182692bd44176d79f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 20 Sep 2019 23:15:42 +0300 Subject: [PATCH 082/108] Better check in shared binary build --- docker/packager/binary/build.sh | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index 9d994af0986..ed30feb8cb7 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -14,6 +14,11 @@ mv ./dbms/programs/clickhouse* /output mv ./dbms/unit_tests_dbms /output find . -name '*.so' -print -exec mv '{}' /output \; find . -name '*.so.*' -print -exec mv '{}' /output \; -tar -czvf shared_build.tgz /output -rm -r /output/* -mv shared_build.tgz /output + +count=`ls -1 /output/*.so 2>/dev/null | wc -l` +if [ $count != 0 ] +then + tar -czvf shared_build.tgz /output + rm -r /output/* + mv shared_build.tgz /output +fi From b1f19a50d4eee10386a8db78cdca57a808555618 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 20 Sep 2019 23:16:55 +0300 Subject: [PATCH 083/108] Update CHANGELOG.md --- CHANGELOG.md | 42 +++++++++++++++++++++++++++++------------- 1 file changed, 29 insertions(+), 13 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 11ca2fd1301..c385831af85 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -72,7 +72,6 @@ * Fix for skip indices with vertical merge and alter. Fix for `Bad size of marks file` exception. [#6594](https://github.com/yandex/ClickHouse/issues/6594) [#6713](https://github.com/yandex/ClickHouse/pull/6713) ([alesapin](https://github.com/alesapin)) * Fix rare crash in `ALTER MODIFY COLUMN` and vertical merge when one of merged/altered parts is empty (0 rows) [#6746](https://github.com/yandex/ClickHouse/issues/6746) [#6780](https://github.com/yandex/ClickHouse/pull/6780) ([alesapin](https://github.com/alesapin)) * Fixed bug in conversion of `LowCardinality` types in `AggregateFunctionFactory`. This fixes [#6257](https://github.com/yandex/ClickHouse/issues/6257). [#6281](https://github.com/yandex/ClickHouse/pull/6281) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) -* Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/yandex/ClickHouse/issues/6224) [#6282](https://github.com/yandex/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) * Fix wrong behavior and possible segfaults in `topK` and `topKWeighted` aggregated functions. [#6404](https://github.com/yandex/ClickHouse/pull/6404) ([Anton Popov](https://github.com/CurtizJ)) * Fixed unsafe code around `getIdentifier` function. [#6401](https://github.com/yandex/ClickHouse/issues/6401) [#6409](https://github.com/yandex/ClickHouse/pull/6409) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed bug in MySQL wire protocol (is used while connecting to ClickHouse form MySQL client). Caused by heap buffer overflow in `PacketPayloadWriteBuffer`. [#6212](https://github.com/yandex/ClickHouse/pull/6212) ([Yuriy Baranov](https://github.com/yurriy)) @@ -116,7 +115,6 @@ ### Security Fix * This release also contains all bug security fixes from 19.13 and 19.11. -* If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/yandex/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed the possibility of a fabricated query to cause server crash due to stack overflow in SQL parser. Fixed the possibility of stack overflow in Merge and Distributed tables, materialized views and conditions for row-level security that involve subqueries. [#6433](https://github.com/yandex/ClickHouse/pull/6433) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### Improvement @@ -272,32 +270,46 @@ * Removed rarely used table function `catBoostPool` and storage `CatBoostPool`. If you have used this table function, please write email to `clickhouse-feedback@yandex-team.com`. Note that CatBoost integration remains and will be supported. [#6279](https://github.com/yandex/ClickHouse/pull/6279) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Disable `ANY RIGHT JOIN` and `ANY FULL JOIN` by default. Set `any_join_get_any_from_right_table` setting to enable them. [#5126](https://github.com/yandex/ClickHouse/issues/5126) [#6351](https://github.com/yandex/ClickHouse/pull/6351) ([Artem Zuikov](https://github.com/4ertus2)) +## ClickHouse release 19.11.11.57, 2019-09-13 +* Fix logical error causing segfaults when selecting from Kafka empty topic. [#6902](https://github.com/yandex/ClickHouse/issues/6902) [#6909](https://github.com/yandex/ClickHouse/pull/6909) ([Ivan](https://github.com/abyss7)) +* Fix for function `АrrayEnumerateUniqRanked` with empty arrays in params. [#6928](https://github.com/yandex/ClickHouse/pull/6928) ([proller](https://github.com/proller)) + ## ClickHouse release 19.13.4.32, 2019-09-10 ### Bug Fix -* Fixed an issue when long `ALTER UPDATE` or `ALTER DELETE` may prevent regular merges to run. Prevent mutations from executing if there is no enough free threads available. [#6502](https://github.com/yandex/ClickHouse/issues/6502) [#6617](https://github.com/yandex/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) +* This release also contains all bug security fixes from 19.11.9.52 and 19.11.10.54. * Fixed data race in `system.parts` table and `ALTER` query. [#6245](https://github.com/yandex/ClickHouse/issues/6245) [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fix Kafka messages duplication problem on normal server restart. [#6597](https://github.com/yandex/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) -* Do store offsets for Kafka messages manually to be able to commit them all at once for all partitions. Fixes potential duplication in "one consumer - many partitions" scenario. [#6872](https://github.com/yandex/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) * Fixed mismatched header in streams happened in case of reading from empty distributed table with sample and prewhere. [#6167](https://github.com/yandex/ClickHouse/issues/6167) ([Lixiang Qian](https://github.com/fancyqlx)) [#6823](https://github.com/yandex/ClickHouse/pull/6823) ([Nikolai Kochetov](https://github.com/KochetovNicolai)) * Fixed crash when using `IN` clause with a subquery with a tuple. [#6125](https://github.com/yandex/ClickHouse/issues/6125) [#6550](https://github.com/yandex/ClickHouse/pull/6550) ([tavplubix](https://github.com/tavplubix)) -* Fixed bug in function `arrayEnumerateUniqRanked`. [#6779](https://github.com/yandex/ClickHouse/pull/6779) ([proller](https://github.com/proller)) -* Fix `JSONExtract` function while extracting a `Tuple` from JSON. [#6718](https://github.com/yandex/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) * Fix case with same column names in `GLOBAL JOIN ON` section. [#6181](https://github.com/yandex/ClickHouse/pull/6181) ([Artem Zuikov](https://github.com/4ertus2)) * Fix crash when casting types to `Decimal` that do not support it. Throw exception instead. [#6297](https://github.com/yandex/ClickHouse/pull/6297) ([Artem Zuikov](https://github.com/4ertus2)) -* Fixed wrong behaviour of `nullIf` function for constant arguments. [#6518](https://github.com/yandex/ClickHouse/pull/6518) ([Guillaume Tassery](https://github.com/YiuRULE)) [#6580](https://github.com/yandex/ClickHouse/pull/6580) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed crash in `extractAll()` function. [#6644](https://github.com/yandex/ClickHouse/pull/6644) ([Artem Zuikov](https://github.com/4ertus2)) * Query transformation for `MySQL`, `ODBC`, `JDBC` table functions now works properly for `SELECT WHERE` queries with multiple `AND` expressions. [#6381](https://github.com/yandex/ClickHouse/issues/6381) [#6676](https://github.com/yandex/ClickHouse/pull/6676) ([dimarub2000](https://github.com/dimarub2000)) * Added previous declaration checks for MySQL 8 integration. [#6569](https://github.com/yandex/ClickHouse/pull/6569) ([Rafael David Tinoco](https://github.com/rafaeldtinoco)) -* Parquet: Fix reading boolean columns. [#6579](https://github.com/yandex/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Fixed error with processing "timezone" in server configuration file. [#6709](https://github.com/yandex/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) -* Improve error handling in cache dictionaries. [#6737](https://github.com/yandex/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) -* Fix kafka tests. [#6805](https://github.com/yandex/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) -* Fixed performance test. [#6392](https://github.com/yandex/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) ### Security Fix * Fix two vulnerabilities in codecs in decompression phase (malicious user can fabricate compressed data that will lead to buffer overflow in decompression). [#6670](https://github.com/yandex/ClickHouse/pull/6670) ([Artem Zuikov](https://github.com/4ertus2)) +## ClickHouse release 19.11.10.54, 2019-09-10 + +### Bug Fix +* Do store offsets for Kafka messages manually to be able to commit them all at once for all partitions. Fixes potential duplication in "one consumer - many partitions" scenario. [#6872](https://github.com/yandex/ClickHouse/pull/6872) ([Ivan](https://github.com/abyss7)) + +## ClickHouse release 19.11.9.52, 2019-09-6 +* Improve error handling in cache dictionaries. [#6737](https://github.com/yandex/ClickHouse/pull/6737) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fixed bug in function `arrayEnumerateUniqRanked`. [#6779](https://github.com/yandex/ClickHouse/pull/6779) ([proller](https://github.com/proller)) +* Fix `JSONExtract` function while extracting a `Tuple` from JSON. [#6718](https://github.com/yandex/ClickHouse/pull/6718) ([Vitaly Baranov](https://github.com/vitlibar)) +* Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/yandex/ClickHouse/issues/6224) [#6282](https://github.com/yandex/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) +* Fixed performance test. [#6392](https://github.com/yandex/ClickHouse/pull/6392) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Parquet: Fix reading boolean columns. [#6579](https://github.com/yandex/ClickHouse/pull/6579) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed wrong behaviour of `nullIf` function for constant arguments. [#6518](https://github.com/yandex/ClickHouse/pull/6518) ([Guillaume Tassery](https://github.com/YiuRULE)) [#6580](https://github.com/yandex/ClickHouse/pull/6580) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix Kafka messages duplication problem on normal server restart. [#6597](https://github.com/yandex/ClickHouse/pull/6597) ([Ivan](https://github.com/abyss7)) +* Fixed an issue when long `ALTER UPDATE` or `ALTER DELETE` may prevent regular merges to run. Prevent mutations from executing if there is no enough free threads available. [#6502](https://github.com/yandex/ClickHouse/issues/6502) [#6617](https://github.com/yandex/ClickHouse/pull/6617) ([tavplubix](https://github.com/tavplubix)) +* Fixed error with processing "timezone" in server configuration file. [#6709](https://github.com/yandex/ClickHouse/pull/6709) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fix kafka tests. [#6805](https://github.com/yandex/ClickHouse/pull/6805) ([Ivan](https://github.com/abyss7)) + +### Security Fix +* If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/yandex/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) ## ClickHouse release 19.13.3.26, 2019-08-22 @@ -308,6 +320,10 @@ * Fixed issue with parsing CSV [#6426](https://github.com/yandex/ClickHouse/issues/6426) [#6559](https://github.com/yandex/ClickHouse/pull/6559) ([tavplubix](https://github.com/tavplubix)) * Fixed data race in system.parts table and ALTER query. This fixes [#6245](https://github.com/yandex/ClickHouse/issues/6245). [#6513](https://github.com/yandex/ClickHouse/pull/6513) ([alexey-milovidov](https://github.com/alexey-milovidov)) * Fixed wrong code in mutations that may lead to memory corruption. Fixed segfault with read of address `0x14c0` that may happed due to concurrent `DROP TABLE` and `SELECT` from `system.parts` or `system.parts_columns`. Fixed race condition in preparation of mutation queries. Fixed deadlock caused by `OPTIMIZE` of Replicated tables and concurrent modification operations like ALTERs. [#6514](https://github.com/yandex/ClickHouse/pull/6514) ([alexey-milovidov](https://github.com/alexey-milovidov)) +* Fixed possible data loss after `ALTER DELETE` query on table with skipping index. [#6224](https://github.com/yandex/ClickHouse/issues/6224) [#6282](https://github.com/yandex/ClickHouse/pull/6282) ([Nikita Vasilev](https://github.com/nikvas0)) + +### Security Fix +* If the attacker has write access to ZooKeeper and is able to run custom server available from the network where ClickHouse run, it can create custom-built malicious server that will act as ClickHouse replica and register it in ZooKeeper. When another replica will fetch data part from malicious replica, it can force clickhouse-server to write to arbitrary path on filesystem. Found by Eldar Zaitov, information security team at Yandex. [#6247](https://github.com/yandex/ClickHouse/pull/6247) ([alexey-milovidov](https://github.com/alexey-milovidov)) ## ClickHouse release 19.13.2.19, 2019-08-14 From 42b739550b51bc1e1b4439994486edee14557f56 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 20 Sep 2019 23:35:50 +0300 Subject: [PATCH 084/108] Renamed "storage_policy_name" to "storage_policy" --- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 2 +- .../Storages/MergeTree/MergeTreeSettings.h | 4 +-- .../integration/test_multiple_disks/test.py | 36 +++++++++---------- docs/ru/operations/table_engines/mergetree.md | 12 +++---- 4 files changed, 27 insertions(+), 27 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 3538ad4c843..d68d96675af 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -126,7 +126,7 @@ MergeTreeData::MergeTreeData( , log_name(database_name + "." + table_name) , log(&Logger::get(log_name)) , storage_settings(std::move(storage_settings_)) - , storage_policy(context_.getStoragePolicy(getSettings()->storage_policy_name)) + , storage_policy(context_.getStoragePolicy(getSettings()->storage_policy)) , data_parts_by_info(data_parts_indexes.get()) , data_parts_by_state_and_info(data_parts_indexes.get()) , parts_mover(this) diff --git a/dbms/src/Storages/MergeTree/MergeTreeSettings.h b/dbms/src/Storages/MergeTree/MergeTreeSettings.h index 994d9675941..3652718451f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSettings.h @@ -88,7 +88,7 @@ struct MergeTreeSettings : public SettingsCollection M(SettingMaxThreads, max_part_loading_threads, 0, "The number of theads to load data parts at startup.") \ M(SettingMaxThreads, max_part_removal_threads, 0, "The number of theads for concurrent removal of inactive data parts. One is usually enough, but in 'Google Compute Environment SSD Persistent Disks' file removal (unlink) operation is extraordinarily slow and you probably have to increase this number (recommended is up to 16).") \ M(SettingUInt64, concurrent_part_removal_threshold, 100, "Activate concurrent part removal (see 'max_part_removal_threads') only if the number of inactive data parts is at least this.") \ - M(SettingString, storage_policy_name, "default", "Name of storage disk policy") + M(SettingString, storage_policy, "default", "Name of storage disk policy") DECLARE_SETTINGS_COLLECTION(LIST_OF_MERGE_TREE_SETTINGS) @@ -104,7 +104,7 @@ struct MergeTreeSettings : public SettingsCollection /// We check settings after storage creation static bool isReadonlySetting(const String & name) { - return name == "index_granularity" || name == "index_granularity_bytes" || name == "storage_policy_name"; + return name == "index_granularity" || name == "index_granularity_bytes" || name == "storage_policy"; } }; diff --git a/dbms/tests/integration/test_multiple_disks/test.py b/dbms/tests/integration/test_multiple_disks/test.py index afad78cdc8c..4ee337229c9 100644 --- a/dbms/tests/integration/test_multiple_disks/test.py +++ b/dbms/tests/integration/test_multiple_disks/test.py @@ -146,7 +146,7 @@ def test_query_parser(start_cluster): d UInt64 ) ENGINE = MergeTree() ORDER BY d - SETTINGS storage_policy_name='very_exciting_policy' + SETTINGS storage_policy='very_exciting_policy' """) with pytest.raises(QueryRuntimeException): @@ -155,7 +155,7 @@ def test_query_parser(start_cluster): d UInt64 ) ENGINE = MergeTree() ORDER BY d - SETTINGS storage_policy_name='jbod1' + SETTINGS storage_policy='jbod1' """) @@ -164,7 +164,7 @@ def test_query_parser(start_cluster): d UInt64 ) ENGINE = MergeTree() ORDER BY d - SETTINGS storage_policy_name='default' + SETTINGS storage_policy='default' """) node1.query("INSERT INTO table_with_normal_policy VALUES (5)") @@ -182,7 +182,7 @@ def test_query_parser(start_cluster): node1.query("ALTER TABLE table_with_normal_policy MOVE PARTITION 'yyyy' TO DISK 'jbod1'") with pytest.raises(QueryRuntimeException): - node1.query("ALTER TABLE table_with_normal_policy MODIFY SETTING storage_policy_name='moving_jbod_with_external'") + node1.query("ALTER TABLE table_with_normal_policy MODIFY SETTING storage_policy='moving_jbod_with_external'") finally: node1.query("DROP TABLE IF EXISTS table_with_normal_policy") @@ -204,7 +204,7 @@ def test_round_robin(start_cluster, name, engine): d UInt64 ) ENGINE = {engine} ORDER BY d - SETTINGS storage_policy_name='jbods_with_external' + SETTINGS storage_policy='jbods_with_external' """.format(name=name, engine=engine)) # first should go to the jbod1 @@ -239,7 +239,7 @@ def test_max_data_part_size(start_cluster, name, engine): s1 String ) ENGINE = {engine} ORDER BY tuple() - SETTINGS storage_policy_name='jbods_with_external' + SETTINGS storage_policy='jbods_with_external' """.format(name=name, engine=engine)) data = [] # 10MB in total for i in range(10): @@ -263,7 +263,7 @@ def test_jbod_overflow(start_cluster, name, engine): s1 String ) ENGINE = {engine} ORDER BY tuple() - SETTINGS storage_policy_name='small_jbod_with_external' + SETTINGS storage_policy='small_jbod_with_external' """.format(name=name, engine=engine)) node1.query("SYSTEM STOP MERGES") @@ -313,7 +313,7 @@ def test_background_move(start_cluster, name, engine): s1 String ) ENGINE = {engine} ORDER BY tuple() - SETTINGS storage_policy_name='moving_jbod_with_external' + SETTINGS storage_policy='moving_jbod_with_external' """.format(name=name, engine=engine)) for i in range(5): @@ -357,7 +357,7 @@ def test_start_stop_moves(start_cluster, name, engine): s1 String ) ENGINE = {engine} ORDER BY tuple() - SETTINGS storage_policy_name='moving_jbod_with_external' + SETTINGS storage_policy='moving_jbod_with_external' """.format(name=name, engine=engine)) node1.query("INSERT INTO {} VALUES ('HELLO')".format(name)) @@ -452,7 +452,7 @@ def test_alter_move(start_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy_name='jbods_with_external' + SETTINGS storage_policy='jbods_with_external' """.format(name=name, engine=engine)) node1.query("SYSTEM STOP MERGES {}".format(name)) # to avoid conflicts @@ -540,7 +540,7 @@ def test_concurrent_alter_move(start_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy_name='jbods_with_external' + SETTINGS storage_policy='jbods_with_external' """.format(name=name, engine=engine)) def insert(num): @@ -591,7 +591,7 @@ def test_concurrent_alter_move_and_drop(start_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy_name='jbods_with_external' + SETTINGS storage_policy='jbods_with_external' """.format(name=name, engine=engine)) def insert(num): @@ -640,7 +640,7 @@ def test_mutate_to_another_disk(start_cluster, name, engine): s1 String ) ENGINE = {engine} ORDER BY tuple() - SETTINGS storage_policy_name='moving_jbod_with_external' + SETTINGS storage_policy='moving_jbod_with_external' """.format(name=name, engine=engine)) for i in range(5): @@ -687,7 +687,7 @@ def test_concurrent_alter_modify(start_cluster, name, engine): ) ENGINE = {engine} ORDER BY tuple() PARTITION BY toYYYYMM(EventDate) - SETTINGS storage_policy_name='jbods_with_external' + SETTINGS storage_policy='jbods_with_external' """.format(name=name, engine=engine)) def insert(num): @@ -733,7 +733,7 @@ def test_simple_replication_and_moves(start_cluster): s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_moves', '{}') ORDER BY tuple() - SETTINGS storage_policy_name='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2 + SETTINGS storage_policy='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2 """.format(i + 1)) def insert(num): @@ -796,7 +796,7 @@ def test_download_appropriate_disk(start_cluster): s1 String ) ENGINE = ReplicatedMergeTree('/clickhouse/replicated_table_for_download', '{}') ORDER BY tuple() - SETTINGS storage_policy_name='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2 + SETTINGS storage_policy='moving_jbod_with_external', old_parts_lifetime=1, cleanup_delay_period=1, cleanup_delay_period_random_add=2 """.format(i + 1)) data = [] @@ -827,7 +827,7 @@ def test_rename(start_cluster): s String ) ENGINE = MergeTree ORDER BY tuple() - SETTINGS storage_policy_name='small_jbod_with_external' + SETTINGS storage_policy='small_jbod_with_external' """) for _ in range(5): @@ -867,7 +867,7 @@ def test_freeze(start_cluster): ) ENGINE = MergeTree ORDER BY tuple() PARTITION BY toYYYYMM(d) - SETTINGS storage_policy_name='small_jbod_with_external' + SETTINGS storage_policy='small_jbod_with_external' """) for _ in range(5): diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index 543c5177bd3..c1fb3dc9edd 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -332,7 +332,7 @@ TTL date_time + INTERVAL 15 HOUR Создание таблицы с TTL ```sql -CREATE TABLE example_table +CREATE TABLE example_table ( d DateTime, a Int TTL d + INTERVAL 1 MONTH, @@ -367,7 +367,7 @@ ALTER TABLE example_table Примеры: ```sql -CREATE TABLE example_table +CREATE TABLE example_table ( d DateTime, a Int @@ -378,7 +378,7 @@ ORDER BY d TTL d + INTERVAL 1 MONTH; ``` -Изменение TTL +Изменение TTL ```sql ALTER TABLE example_table @@ -488,10 +488,10 @@ CREATE TABLE table_with_non_default_policy ( OrderID UInt64, BannerID UInt64, SearchPhrase String -) ENGINE = MergeTree() +) ENGINE = MergeTree ORDER BY (OrderID, BannerID) PARTITION BY toYYYYMM(EventDate) -SETTINGS storage_policy_name='moving_from_ssd_to_hdd' +SETTINGS storage_policy = 'moving_from_ssd_to_hdd' ``` По умолчанию используется политика хранения `default` в которой есть один том и один диск, указанный в ``. В данный момент менять политику хранения после создания таблицы нельзя. @@ -502,7 +502,7 @@ SETTINGS storage_policy_name='moving_from_ssd_to_hdd' * В результате вставки (запрос `INSERT`). * В фоновых операциях слияний и [мутаций](../../query_language/alter.md#alter-mutations). -* При скачивании данных с другой реплики. +* При скачивании данных с другой реплики. * В результате заморозки партиций [ALTER TABLE ... FREEZE PARTITION](../../query_language/alter.md#alter_freeze-partition). Во всех случаях, кроме мутаций и заморозки партиций, при записи куска выбирается том и диск в соответствии с указанной конфигурацией хранилища: From 1dc58dcb15507e4b5427310b06b221610b4bf828 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 20 Sep 2019 23:53:14 +0300 Subject: [PATCH 085/108] Update InterpreterCreateQuery.cpp --- dbms/src/Interpreters/InterpreterCreateQuery.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index f2189a59866..9d7e052c826 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -417,8 +417,12 @@ ColumnsDescription InterpreterCreateQuery::setProperties( else if (!create.as_table.empty()) { columns = as_storage->getColumns(); + + /// Secondary indices make sense only for MergeTree family of storage engines. + /// We should not copy them for other storages. if (create.storage && endsWith(create.storage->engine->name, "MergeTree")) indices = as_storage->getIndices(); + constraints = as_storage->getConstraints(); } else if (create.select) From b22b65dd299f77f882e8c98f7342b03671bd678f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Sep 2019 02:22:25 +0300 Subject: [PATCH 086/108] Avoid SIGSEGV on batch send failure (file with index XX is absent) In case of the following error: Failed to send batch: file with index 23742 is absent NULL dereference will occur for the "remote". --- dbms/src/Storages/Distributed/DirectoryMonitor.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index df19d76bbb3..183897c7574 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -392,7 +392,8 @@ struct StorageDistributedDirectoryMonitor::Batch remote->writePrepared(in); } - remote->writeSuffix(); + if (remote) + remote->writeSuffix(); } catch (const Exception & e) { From 420089c301abe7db167a05eefd80560ea4ad0e4b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 21 Sep 2019 02:22:40 +0300 Subject: [PATCH 087/108] Add new dictionary layout (sparse_hashed) that is more memory efficient With this new layout, sparsehash will be used over default HashMap, sparsehash is more memory efficient but it is also slower. So in a nutshell: - HashMap uses ~2x more memory then sparse_hash_map - HashMap ~2-2.5x faster then sparse_hash_map (tested on lots of input, and the most close to production was dictionary with 600KK hashes and UInt16 as value) TODO: - fix allocated memory calculation - getBufferSizeInBytes/getBufferSizeInCells interface - benchmarks v0: replace HashMap with google::sparse_hash_map v2: use google::sparse_hash_map only when isset to true v3: replace attributes with different layout v4: use ch hash over std::hash --- dbms/src/Dictionaries/CMakeLists.txt | 2 + dbms/src/Dictionaries/HashedDictionary.cpp | 124 ++++++++++++++---- dbms/src/Dictionaries/HashedDictionary.h | 34 ++++- dbms/src/Functions/CMakeLists.txt | 2 +- dbms/tests/config/ints_dictionary.xml | 63 +++++++++ .../0_stateless/00950_dict_get.reference | 3 + .../queries/0_stateless/00950_dict_get.sql | 28 ++++ .../dicts/external_dicts_dict_layout.md | 13 ++ 8 files changed, 242 insertions(+), 27 deletions(-) diff --git a/dbms/src/Dictionaries/CMakeLists.txt b/dbms/src/Dictionaries/CMakeLists.txt index 4d066d1f59b..d6f8fc57ff6 100644 --- a/dbms/src/Dictionaries/CMakeLists.txt +++ b/dbms/src/Dictionaries/CMakeLists.txt @@ -40,3 +40,5 @@ if(USE_POCO_MONGODB) endif() add_subdirectory(Embedded) + +target_include_directories(clickhouse_dictionaries SYSTEM PRIVATE ${SPARSEHASH_INCLUDE_DIR}) diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 9b853ac2df5..4d195b5139a 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -3,6 +3,23 @@ #include "DictionaryBlockInputStream.h" #include "DictionaryFactory.h" +namespace +{ + +/// google::sparse_hash_map +template auto first(const T &lhs) -> decltype(lhs.first) +{ return lhs.first; } +template auto second(const T &lhs) -> decltype(lhs.second) +{ return lhs.second; } + +/// HashMap +template auto first(const T &lhs) -> decltype(lhs.getFirst()) +{ return lhs.getFirst(); } +template auto second(const T &lhs) -> decltype(lhs.getSecond()) +{ return lhs.getSecond(); } + +} + namespace DB { namespace ErrorCodes @@ -21,12 +38,14 @@ HashedDictionary::HashedDictionary( DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, + bool sparse_, BlockPtr saved_block_) : name{name_} , dict_struct(dict_struct_) , source_ptr{std::move(source_ptr_)} , dict_lifetime(dict_lifetime_) , require_nonempty(require_nonempty_) + , sparse(sparse_) , saved_block{std::move(saved_block_)} { createAttributes(); @@ -57,11 +76,10 @@ static inline HashedDictionary::Key getAt(const HashedDictionary::Key & value, c return value; } -template -void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const +template +void HashedDictionary::isInAttrImpl(const AttrType & attr, const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const { const auto null_value = std::get(hierarchical_attribute->null_values); - const auto & attr = *std::get>(hierarchical_attribute->maps); const auto rows = out.size(); for (const auto row : ext::range(0, rows)) @@ -73,7 +91,7 @@ void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType { auto it = attr.find(id); if (it != std::end(attr)) - id = it->getSecond(); + id = second(*it); else break; } @@ -83,6 +101,13 @@ void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType query_count.fetch_add(rows, std::memory_order_relaxed); } +template +void HashedDictionary::isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const +{ + if (!sparse) + return isInAttrImpl(*std::get>(hierarchical_attribute->maps), child_ids, ancestor_ids, out); + return isInAttrImpl(*std::get>(hierarchical_attribute->sparse_maps), child_ids, ancestor_ids, out); +} void HashedDictionary::isInVectorVector( const PaddedPODArray & child_ids, const PaddedPODArray & ancestor_ids, PaddedPODArray & out) const @@ -407,9 +432,22 @@ void HashedDictionary::loadData() template void HashedDictionary::addAttributeSize(const Attribute & attribute) { - const auto & map_ref = std::get>(attribute.maps); - bytes_allocated += sizeof(CollectionType) + map_ref->getBufferSizeInBytes(); - bucket_count = map_ref->getBufferSizeInCells(); + if (!sparse) + { + const auto & map_ref = std::get>(attribute.maps); + bytes_allocated += sizeof(CollectionType) + map_ref->getBufferSizeInBytes(); + bucket_count = map_ref->getBufferSizeInCells(); + } + else + { + const auto & map_ref = std::get>(attribute.sparse_maps); + bucket_count = map_ref->bucket_count(); + + /** TODO: more accurate calculation */ + bytes_allocated += sizeof(CollectionType); + bytes_allocated += bucket_count; + bytes_allocated += map_ref->size() * sizeof(Key) * sizeof(T); + } } void HashedDictionary::calculateBytesAllocated() @@ -479,12 +517,15 @@ template void HashedDictionary::createAttributeImpl(Attribute & attribute, const Field & null_value) { attribute.null_values = T(null_value.get>()); - attribute.maps = std::make_unique>(); + if (!sparse) + attribute.maps = std::make_unique>(); + else + attribute.sparse_maps = std::make_unique>(); } HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value) { - Attribute attr{type, {}, {}, {}}; + Attribute attr{type, {}, {}, {}, {}}; switch (type) { @@ -535,7 +576,10 @@ HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const Attr case AttributeUnderlyingType::utString: { attr.null_values = null_value.get(); - attr.maps = std::make_unique>(); + if (!sparse) + attr.maps = std::make_unique>(); + else + attr.sparse_maps = std::make_unique>(); attr.string_arena = std::make_unique(); break; } @@ -545,28 +589,43 @@ HashedDictionary::Attribute HashedDictionary::createAttributeWithType(const Attr } -template -void HashedDictionary::getItemsImpl( - const Attribute & attribute, const PaddedPODArray & ids, ValueSetter && set_value, DefaultGetter && get_default) const +template +void HashedDictionary::getItemsAttrImpl( + const AttrType & attr, const PaddedPODArray & ids, ValueSetter && set_value, DefaultGetter && get_default) const { - const auto & attr = *std::get>(attribute.maps); const auto rows = ext::size(ids); for (const auto i : ext::range(0, rows)) { const auto it = attr.find(ids[i]); - set_value(i, it != attr.end() ? static_cast(it->getSecond()) : get_default(i)); + set_value(i, it != attr.end() ? static_cast(second(*it)) : get_default(i)); } query_count.fetch_add(rows, std::memory_order_relaxed); } +template +void HashedDictionary::getItemsImpl( + const Attribute & attribute, const PaddedPODArray & ids, ValueSetter && set_value, DefaultGetter && get_default) const +{ + if (!sparse) + return getItemsAttrImpl(*std::get>(attribute.maps), ids, set_value, get_default); + return getItemsAttrImpl(*std::get>(attribute.sparse_maps), ids, set_value, get_default); +} template bool HashedDictionary::setAttributeValueImpl(Attribute & attribute, const Key id, const T value) { - auto & map = *std::get>(attribute.maps); - return map.insert({id, value}).second; + if (!sparse) + { + auto & map = *std::get>(attribute.maps); + return map.insert({id, value}).second; + } + else + { + auto & map = *std::get>(attribute.sparse_maps); + return map.insert({id, value}).second; + } } bool HashedDictionary::setAttributeValue(Attribute & attribute, const Key id, const Field & value) @@ -605,10 +664,18 @@ bool HashedDictionary::setAttributeValue(Attribute & attribute, const Key id, co case AttributeUnderlyingType::utString: { - auto & map = *std::get>(attribute.maps); const auto & string = value.get(); const auto string_in_arena = attribute.string_arena->insert(string.data(), string.size()); - return map.insert({id, StringRef{string_in_arena, string.size()}}).second; + if (!sparse) + { + auto & map = *std::get>(attribute.maps); + return map.insert({id, StringRef{string_in_arena, string.size()}}).second; + } + else + { + auto & map = *std::get>(attribute.sparse_maps); + return map.insert({id, StringRef{string_in_arena, string.size()}}).second; + } } } @@ -636,18 +703,23 @@ void HashedDictionary::has(const Attribute & attribute, const PaddedPODArray -PaddedPODArray HashedDictionary::getIds(const Attribute & attribute) const +template +PaddedPODArray HashedDictionary::getIdsAttrImpl(const AttrType & attr) const { - const HashMap & attr = *std::get>(attribute.maps); - PaddedPODArray ids; ids.reserve(attr.size()); for (const auto & value : attr) - ids.push_back(value.getFirst()); + ids.push_back(first(value)); return ids; } +template +PaddedPODArray HashedDictionary::getIds(const Attribute & attribute) const +{ + if (!sparse) + return getIdsAttrImpl(*std::get>(attribute.maps)); + return getIdsAttrImpl(*std::get>(attribute.sparse_maps)); +} PaddedPODArray HashedDictionary::getIds() const { @@ -714,9 +786,11 @@ void registerDictionaryHashed(DictionaryFactory & factory) ErrorCodes::BAD_ARGUMENTS}; const DictionaryLifetime dict_lifetime{config, config_prefix + ".lifetime"}; const bool require_nonempty = config.getBool(config_prefix + ".require_nonempty", false); - return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty); + const bool sparse = name == "sparse_hashed"; + return std::make_unique(name, dict_struct, std::move(source_ptr), dict_lifetime, require_nonempty, sparse); }; factory.registerLayout("hashed", create_layout); + factory.registerLayout("sparse_hashed", create_layout); } } diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index d1aa5a38d97..5cd31ba3e80 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -7,6 +7,7 @@ #include #include #include +#include #include #include "DictionaryStructure.h" #include "IDictionary.h" @@ -26,6 +27,7 @@ public: DictionarySourcePtr source_ptr_, const DictionaryLifetime dict_lifetime_, bool require_nonempty_, + bool sparse_, BlockPtr saved_block_ = nullptr); std::string getName() const override { return name; } @@ -46,7 +48,7 @@ public: std::shared_ptr clone() const override { - return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); + return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block); } const IDictionarySource * getSource() const override { return source_ptr.get(); } @@ -149,6 +151,11 @@ private: template using CollectionPtrType = std::unique_ptr>; + template + using SparseCollectionType = google::sparse_hash_map>; + template + using SparseCollectionPtrType = std::unique_ptr>; + struct Attribute final { AttributeUnderlyingType type; @@ -186,6 +193,23 @@ private: CollectionPtrType, CollectionPtrType> maps; + std::variant< + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType, + SparseCollectionPtrType> + sparse_maps; std::unique_ptr string_arena; }; @@ -207,6 +231,9 @@ private: Attribute createAttributeWithType(const AttributeUnderlyingType type, const Field & null_value); + template + void getItemsAttrImpl( + const AttrType & attr, const PaddedPODArray & ids, ValueSetter && set_value, DefaultGetter && get_default) const; template void getItemsImpl( const Attribute & attribute, const PaddedPODArray & ids, ValueSetter && set_value, DefaultGetter && get_default) const; @@ -221,11 +248,15 @@ private: template void has(const Attribute & attribute, const PaddedPODArray & ids, PaddedPODArray & out) const; + template + PaddedPODArray getIdsAttrImpl(const AttrType & attr) const; template PaddedPODArray getIds(const Attribute & attribute) const; PaddedPODArray getIds() const; + template + void isInAttrImpl(const AttrType & attr, const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; template void isInImpl(const ChildType & child_ids, const AncestorType & ancestor_ids, PaddedPODArray & out) const; @@ -234,6 +265,7 @@ private: const DictionarySourcePtr source_ptr; const DictionaryLifetime dict_lifetime; const bool require_nonempty; + const bool sparse; std::map attribute_index_by_name; std::vector attributes; diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index f495d6d8665..d3d8eb34d36 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -33,7 +33,7 @@ if (OPENSSL_CRYPTO_LIBRARY) endif() target_include_directories(clickhouse_functions PRIVATE ${CMAKE_CURRENT_BINARY_DIR}/include) -target_include_directories(clickhouse_functions SYSTEM PRIVATE ${DIVIDE_INCLUDE_DIR} ${METROHASH_INCLUDE_DIR}) +target_include_directories(clickhouse_functions SYSTEM PRIVATE ${DIVIDE_INCLUDE_DIR} ${METROHASH_INCLUDE_DIR} ${SPARSEHASH_INCLUDE_DIR}) if (CONSISTENT_HASHING_INCLUDE_DIR) target_include_directories (clickhouse_functions PRIVATE ${CONSISTENT_HASHING_INCLUDE_DIR}) diff --git a/dbms/tests/config/ints_dictionary.xml b/dbms/tests/config/ints_dictionary.xml index c70f820ce36..a406c8553c0 100644 --- a/dbms/tests/config/ints_dictionary.xml +++ b/dbms/tests/config/ints_dictionary.xml @@ -125,6 +125,69 @@ + + hashed_sparse_ints + + + localhost + 9000 + default + + test_00950 + ints
+
+ + 0 + + + + + + key + + + i8 + Int8 + 0 + + + i16 + Int16 + 0 + + + i32 + Int32 + 0 + + + i64 + Int64 + 0 + + + u8 + UInt8 + 0 + + + u16 + UInt16 + 0 + + + u32 + UInt32 + 0 + + + u64 + UInt64 + 0 + + +
+ cache_ints diff --git a/dbms/tests/queries/0_stateless/00950_dict_get.reference b/dbms/tests/queries/0_stateless/00950_dict_get.reference index fabfda8425f..3010a2989c0 100644 --- a/dbms/tests/queries/0_stateless/00950_dict_get.reference +++ b/dbms/tests/queries/0_stateless/00950_dict_get.reference @@ -4,6 +4,9 @@ dictGetOrDefault flat_ints 0 42 42 42 42 42 42 42 42 dictGet hashed_ints 1 1 1 1 1 1 1 1 1 dictGetOrDefault hashed_ints 1 1 1 1 1 1 1 1 1 dictGetOrDefault hashed_ints 0 42 42 42 42 42 42 42 42 +dictGet hashed_sparse_ints 1 1 1 1 1 1 1 1 1 +dictGetOrDefault hashed_sparse_ints 1 1 1 1 1 1 1 1 1 +dictGetOrDefault hashed_sparse_ints 0 42 42 42 42 42 42 42 42 dictGet cache_ints 1 1 1 1 1 1 1 1 1 dictGetOrDefault cache_ints 1 1 1 1 1 1 1 1 1 dictGetOrDefault cache_ints 0 42 42 42 42 42 42 42 42 diff --git a/dbms/tests/queries/0_stateless/00950_dict_get.sql b/dbms/tests/queries/0_stateless/00950_dict_get.sql index 20e5e376855..2483a21c0d3 100644 --- a/dbms/tests/queries/0_stateless/00950_dict_get.sql +++ b/dbms/tests/queries/0_stateless/00950_dict_get.sql @@ -69,6 +69,34 @@ select 'dictGetOrDefault', 'hashed_ints' as dict_name, toUInt64(0) as k, dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)), dictGetOrDefault(dict_name, 'u64', k, toUInt64(42)); +select 'dictGet', 'hashed_sparse_ints' as dict_name, toUInt64(1) as k, + dictGet(dict_name, 'i8', k), + dictGet(dict_name, 'i16', k), + dictGet(dict_name, 'i32', k), + dictGet(dict_name, 'i64', k), + dictGet(dict_name, 'u8', k), + dictGet(dict_name, 'u16', k), + dictGet(dict_name, 'u32', k), + dictGet(dict_name, 'u64', k); +select 'dictGetOrDefault', 'hashed_sparse_ints' as dict_name, toUInt64(1) as k, + dictGetOrDefault(dict_name, 'i8', k, toInt8(42)), + dictGetOrDefault(dict_name, 'i16', k, toInt16(42)), + dictGetOrDefault(dict_name, 'i32', k, toInt32(42)), + dictGetOrDefault(dict_name, 'i64', k, toInt64(42)), + dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)), + dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)), + dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)), + dictGetOrDefault(dict_name, 'u64', k, toUInt64(42)); +select 'dictGetOrDefault', 'hashed_sparse_ints' as dict_name, toUInt64(0) as k, + dictGetOrDefault(dict_name, 'i8', k, toInt8(42)), + dictGetOrDefault(dict_name, 'i16', k, toInt16(42)), + dictGetOrDefault(dict_name, 'i32', k, toInt32(42)), + dictGetOrDefault(dict_name, 'i64', k, toInt64(42)), + dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)), + dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)), + dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)), + dictGetOrDefault(dict_name, 'u64', k, toUInt64(42)); + select 'dictGet', 'cache_ints' as dict_name, toUInt64(1) as k, dictGet(dict_name, 'i8', k), dictGet(dict_name, 'i16', k), diff --git a/docs/en/query_language/dicts/external_dicts_dict_layout.md b/docs/en/query_language/dicts/external_dicts_dict_layout.md index c3096544d25..a9a80dbe761 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_layout.md +++ b/docs/en/query_language/dicts/external_dicts_dict_layout.md @@ -39,6 +39,7 @@ The configuration looks like this: - [flat](#flat) - [hashed](#dicts-external_dicts_dict_layout-hashed) +- [sparse_hashed](#dicts-external_dicts_dict_layout-sparse_hashed) - [cache](#cache) - [range_hashed](#range-hashed) - [complex_key_hashed](#complex-key-hashed) @@ -77,6 +78,18 @@ Configuration example: ``` +### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} + +Similar to `hashed`, but uses less memory in favor more CPU usage. + +Configuration example: + +```xml + + + +``` + ### complex_key_hashed From 59763cbb3af1bb28db49d27c8bf011570cb66489 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 18 Sep 2019 02:16:49 +0300 Subject: [PATCH 088/108] Add ability to send events with absolute values to graphite sends only deltas for events, while sometimes it is more convenient to get absolute values (for example to calculate rate). This patch introduces another directive -- that will handle this. v0: use as bool/string v2: use true over absolute --- dbms/programs/server/MetricsTransmitter.cpp | 11 +++++++++++ dbms/programs/server/MetricsTransmitter.h | 5 ++++- dbms/programs/server/config.xml | 2 ++ docs/en/operations/server_settings/settings.md | 4 +++- docs/ru/operations/server_settings/settings.md | 4 +++- 5 files changed, 23 insertions(+), 3 deletions(-) diff --git a/dbms/programs/server/MetricsTransmitter.cpp b/dbms/programs/server/MetricsTransmitter.cpp index 8419d3e1b8c..73413cad1c0 100644 --- a/dbms/programs/server/MetricsTransmitter.cpp +++ b/dbms/programs/server/MetricsTransmitter.cpp @@ -21,6 +21,7 @@ MetricsTransmitter::MetricsTransmitter( { interval_seconds = config.getInt(config_name + ".interval", 60); send_events = config.getBool(config_name + ".events", true); + send_events_absolute = config.getBool(config_name + ".events_absolute", false); send_metrics = config.getBool(config_name + ".metrics", true); send_asynchronous_metrics = config.getBool(config_name + ".asynchronous_metrics", true); } @@ -95,6 +96,16 @@ void MetricsTransmitter::transmit(std::vector & prev_count } } + if (send_events_absolute) + { + for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) + { + const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); + std::string key{ProfileEvents::getName(static_cast(i))}; + key_vals.emplace_back(profile_events_absolute_path_prefix + key, counter); + } + } + if (send_metrics) { for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i) diff --git a/dbms/programs/server/MetricsTransmitter.h b/dbms/programs/server/MetricsTransmitter.h index b9c7fd7f179..1d5795c24db 100644 --- a/dbms/programs/server/MetricsTransmitter.h +++ b/dbms/programs/server/MetricsTransmitter.h @@ -24,7 +24,8 @@ class AsynchronousMetrics; /** Automatically sends - * - difference of ProfileEvents; + * - values deltas of ProfileEvents; + * - absolute values of ProfileEvents; * - values of CurrentMetrics; * - values of AsynchronousMetrics; * to Graphite at beginning of every minute. @@ -44,6 +45,7 @@ private: std::string config_name; UInt32 interval_seconds; bool send_events; + bool send_events_absolute; bool send_metrics; bool send_asynchronous_metrics; @@ -53,6 +55,7 @@ private: ThreadFromGlobalPool thread{&MetricsTransmitter::run, this}; static inline constexpr auto profile_events_path_prefix = "ClickHouse.ProfileEvents."; + static inline constexpr auto profile_events_absolute_path_prefix = "ClickHouse.ProfileEventsAbsolute."; static inline constexpr auto current_metrics_path_prefix = "ClickHouse.Metrics."; static inline constexpr auto asynchronous_metrics_path_prefix = "ClickHouse.AsynchronousMetrics."; }; diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 34fe98b0e31..7263992b3d3 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -258,6 +258,7 @@ true true + false true @@ -269,6 +270,7 @@ true true + false false --> diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 49c8bf6fbc1..baf9013fbbf 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -141,7 +141,8 @@ Settings: - timeout – The timeout for sending data, in seconds. - root_path – Prefix for keys. - metrics – Sending data from a :ref:`system_tables-system.metrics` table. -- events – Sending data from a :ref:`system_tables-system.events` table. +- events – Sending deltas data from a :ref:`system_tables-system.events` table +- events_absolute – Sending absolute data from a :ref:`system_tables-system.events` table - asynchronous_metrics – Sending data from a :ref:`system_tables-system.asynchronous_metrics` table. You can configure multiple `` clauses. For instance, you can use this for sending different data at different intervals. @@ -157,6 +158,7 @@ You can configure multiple `` clauses. For instance, you can use this one_min true true + false true ``` diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 1aacd525d24..2f6362b7635 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -140,7 +140,8 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat - timeout - Таймаут отправки данных в секундах. - root_path - Префикс для ключей. - metrics - Отправка данных из таблицы :ref:`system_tables-system.metrics`. -- events - Отправка данных из таблицы :ref:`system_tables-system.events`. +- events - Отправка дельты данных из таблицы :ref:`system_tables-system.events` +- events_absolute - Отправка абсолютных данных из таблицы :ref:`system_tables-system.events` - asynchronous_metrics - Отправка данных из таблицы :ref:`system_tables-system.asynchronous_metrics`. Можно определить несколько секций ``, например, для передачи различных данных с различной частотой. @@ -156,6 +157,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat one_min true true + false true ``` From ab7ecd84a1738170657c1fcebd6379d60ede7a5a Mon Sep 17 00:00:00 2001 From: Zhichang Yu Date: Sat, 21 Sep 2019 23:34:44 +0800 Subject: [PATCH 089/108] fix bitmapMin and bitmapMax doc --- dbms/src/Functions/FunctionsBitmap.h | 2 +- docs/en/query_language/functions/bitmap_functions.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsBitmap.h b/dbms/src/Functions/FunctionsBitmap.h index ad4f16b16ef..06d45cdd54f 100644 --- a/dbms/src/Functions/FunctionsBitmap.h +++ b/dbms/src/Functions/FunctionsBitmap.h @@ -49,7 +49,7 @@ namespace ErrorCodes * Retrun bitmap cardinality: * bitmapCardinality: bitmap -> integer * - * Retrun smallest value in the set: + * Retrun the smallest value in the set: * bitmapMin: bitmap -> integer * * Retrun the greatest value in the set: diff --git a/docs/en/query_language/functions/bitmap_functions.md b/docs/en/query_language/functions/bitmap_functions.md index fdc2e8a7a0d..29cf67d70f0 100644 --- a/docs/en/query_language/functions/bitmap_functions.md +++ b/docs/en/query_language/functions/bitmap_functions.md @@ -294,7 +294,7 @@ SELECT bitmapCardinality(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMin -Retrun smallest value of type UInt64 in the set, UINT32_MAX if the set is empty. +Retrun the smallest value of type UInt64 in the set, UINT32_MAX if the set is empty. ``` @@ -319,7 +319,7 @@ SELECT bitmapMin(bitmapBuild([1, 2, 3, 4, 5])) AS res ## bitmapMax -Retrun smallest value of type UInt64 in the set, 0 if the set is empty. +Retrun the greatest value of type UInt64 in the set, 0 if the set is empty. ``` From 1fbd2fb6a81a61319b35725df56e74881f65c08e Mon Sep 17 00:00:00 2001 From: Silviu Caragea Date: Sat, 21 Sep 2019 21:30:01 +0300 Subject: [PATCH 090/108] Fix osx build --- dbms/src/Common/PoolWithFailoverBase.h | 2 +- dbms/src/Common/QueryProfiler.cpp | 6 ++++++ dbms/src/Common/StackTrace.cpp | 1 + dbms/src/Common/TraceCollector.cpp | 2 +- dbms/src/Common/checkStackSize.cpp | 18 ++++++++++++++++++ dbms/src/Common/new_delete.cpp | 4 ++++ dbms/src/Interpreters/MetricLog.cpp | 2 +- libs/libcommon/src/sleep.cpp | 18 ++++++++++++++++++ 8 files changed, 50 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/PoolWithFailoverBase.h b/dbms/src/Common/PoolWithFailoverBase.h index 989831ce2b0..6f1c88b53da 100644 --- a/dbms/src/Common/PoolWithFailoverBase.h +++ b/dbms/src/Common/PoolWithFailoverBase.h @@ -199,7 +199,7 @@ PoolWithFailoverBase::getMany( for (const ShuffledPool & pool: shuffled_pools) { auto & pool_state = shared_pool_states[pool.index]; - pool_state.error_count = std::min(max_error_cap, pool_state.error_count + pool.error_count); + pool_state.error_count = std::min(max_error_cap, static_cast(pool_state.error_count + pool.error_count)); } }); diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index a0b75c567a9..d4e0af90ceb 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -30,10 +30,13 @@ namespace /// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler. constexpr size_t QUERY_ID_MAX_LEN = 1024; +# if !defined(__APPLE__) thread_local size_t write_trace_iteration = 0; +#endif void writeTraceInfo(TimerType timer_type, int /* sig */, siginfo_t * info, void * context) { +# if !defined(__APPLE__) /// Quickly drop if signal handler is called too frequently. /// Otherwise we may end up infinitelly processing signals instead of doing any useful work. ++write_trace_iteration; @@ -50,6 +53,9 @@ namespace return; } } +#else + UNUSED(info); +#endif constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag 8 * sizeof(char) + // maximum VarUInt length for string size diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index 9981d0941aa..9694e33a2dd 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Common/TraceCollector.cpp b/dbms/src/Common/TraceCollector.cpp index 9451c3f88e5..6a160326a18 100644 --- a/dbms/src/Common/TraceCollector.cpp +++ b/dbms/src/Common/TraceCollector.cpp @@ -46,7 +46,7 @@ TraceCollector::TraceCollector(std::shared_ptr & trace_log_) if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETFL, flags | O_NONBLOCK)) throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL); -#if !defined(__FreeBSD__) +#if !defined(__FreeBSD__) && !defined(__APPLE__) /** Increase pipe size to avoid slowdown during fine-grained trace collection. */ int pipe_size = fcntl(trace_pipe.fds_rw[1], F_GETPIPE_SZ); diff --git a/dbms/src/Common/checkStackSize.cpp b/dbms/src/Common/checkStackSize.cpp index e7f91bc3330..7459277b563 100644 --- a/dbms/src/Common/checkStackSize.cpp +++ b/dbms/src/Common/checkStackSize.cpp @@ -27,14 +27,32 @@ void checkStackSize() if (!stack_address) { +#if defined(__APPLE__) + // pthread_get_stacksize_np() returns a value too low for the main thread on + // OSX 10.9, http://mail.openjdk.java.net/pipermail/hotspot-dev/2013-October/011369.html + // + // Multiple workarounds possible, adopt the one made by https://github.com/robovm/robovm/issues/274 + // https://developer.apple.com/library/mac/documentation/Cocoa/Conceptual/Multithreading/CreatingThreads/CreatingThreads.html + // Stack size for the main thread is 8MB on OSX excluding the guard page size. + pthread_t thread = pthread_self(); + max_stack_size = pthread_main_np() ? (8 * 1024 * 1024) : pthread_get_stacksize_np(thread); + stack_address = pthread_get_stackaddr_np(thread); +#else pthread_attr_t attr; +#if defined(__FreeBSD__) + pthread_attr_init(&attr); + if (0 != pthread_attr_get_np(pthread_self(), &attr)) + throwFromErrno("Cannot pthread_attr_get_np", ErrorCodes::CANNOT_PTHREAD_ATTR); +#else if (0 != pthread_getattr_np(pthread_self(), &attr)) throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); +#endif SCOPE_EXIT({ pthread_attr_destroy(&attr); }); if (0 != pthread_attr_getstack(&attr, &stack_address, &max_stack_size)) throwFromErrno("Cannot pthread_getattr_np", ErrorCodes::CANNOT_PTHREAD_ATTR); +#endif } const void * frame_address = __builtin_frame_address(0); diff --git a/dbms/src/Common/new_delete.cpp b/dbms/src/Common/new_delete.cpp index f2a85163035..5e32b910b19 100644 --- a/dbms/src/Common/new_delete.cpp +++ b/dbms/src/Common/new_delete.cpp @@ -1,4 +1,8 @@ +#if defined(__MACH__) +#include +#else #include +#endif #include #include diff --git a/dbms/src/Interpreters/MetricLog.cpp b/dbms/src/Interpreters/MetricLog.cpp index 59a500010dc..5622e0c65b0 100644 --- a/dbms/src/Interpreters/MetricLog.cpp +++ b/dbms/src/Interpreters/MetricLog.cpp @@ -103,7 +103,7 @@ void MetricLog::metricThreadFunction() for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); - UInt64 & old_value = prev_profile_events[i]; + auto & old_value = prev_profile_events[i]; elem.profile_events[i] = new_value - old_value; old_value = new_value; } diff --git a/libs/libcommon/src/sleep.cpp b/libs/libcommon/src/sleep.cpp index 710b387d62e..a11140d0cb5 100644 --- a/libs/libcommon/src/sleep.cpp +++ b/libs/libcommon/src/sleep.cpp @@ -3,6 +3,11 @@ #include #include +#if defined(__APPLE__) +#include +#include +#endif + /** * Sleep with nanoseconds precision. Tolerant to signal interruptions * @@ -14,6 +19,18 @@ */ void sleepForNanoseconds(uint64_t nanoseconds) { +# if defined(__APPLE__) + //https://developer.apple.com/library/archive/technotes/tn2169/_index.html + //https://dshil.github.io/blog/missed-os-x-clock-guide/ + static mach_timebase_info_data_t timebase_info = {0}; + if(timebase_info.denom == 0) + mach_timebase_info(&timebase_info); + + uint64_t time_to_wait = nanoseconds * timebase_info.denom / timebase_info.numer; + uint64_t now = mach_absolute_time(); + + while(mach_wait_until(now + time_to_wait) != KERN_SUCCESS); +#else constexpr auto clock_type = CLOCK_MONOTONIC; struct timespec current_time; @@ -29,6 +46,7 @@ void sleepForNanoseconds(uint64_t nanoseconds) finish_time.tv_sec += (nanoseconds / resolution) + extra_second; while (clock_nanosleep(clock_type, TIMER_ABSTIME, &finish_time, nullptr) == EINTR); +#endif } void sleepForMicroseconds(uint64_t microseconds) From ab02aad198a1d4cbf163d6fe5d98e83e914b288a Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sat, 21 Sep 2019 23:44:56 +0300 Subject: [PATCH 091/108] Update sleep.cpp --- libs/libcommon/src/sleep.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/libs/libcommon/src/sleep.cpp b/libs/libcommon/src/sleep.cpp index a11140d0cb5..cfe57151bb1 100644 --- a/libs/libcommon/src/sleep.cpp +++ b/libs/libcommon/src/sleep.cpp @@ -23,13 +23,13 @@ void sleepForNanoseconds(uint64_t nanoseconds) //https://developer.apple.com/library/archive/technotes/tn2169/_index.html //https://dshil.github.io/blog/missed-os-x-clock-guide/ static mach_timebase_info_data_t timebase_info = {0}; - if(timebase_info.denom == 0) + if (timebase_info.denom == 0) mach_timebase_info(&timebase_info); uint64_t time_to_wait = nanoseconds * timebase_info.denom / timebase_info.numer; uint64_t now = mach_absolute_time(); - while(mach_wait_until(now + time_to_wait) != KERN_SUCCESS); + while (mach_wait_until(now + time_to_wait) != KERN_SUCCESS); #else constexpr auto clock_type = CLOCK_MONOTONIC; From 5a14d97587aa8d16bca3e7aa25c16d8afac75f91 Mon Sep 17 00:00:00 2001 From: Silviu Caragea Date: Sun, 22 Sep 2019 01:04:17 +0300 Subject: [PATCH 092/108] Fix linking when __ELF__ is not defined --- dbms/src/Functions/registerFunctionsIntrospection.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/registerFunctionsIntrospection.cpp b/dbms/src/Functions/registerFunctionsIntrospection.cpp index 700a568d822..706175c777f 100644 --- a/dbms/src/Functions/registerFunctionsIntrospection.cpp +++ b/dbms/src/Functions/registerFunctionsIntrospection.cpp @@ -3,16 +3,20 @@ namespace DB class FunctionFactory; +#ifdef __ELF__ void registerFunctionAddressToSymbol(FunctionFactory & factory); -void registerFunctionDemangle(FunctionFactory & factory); void registerFunctionAddressToLine(FunctionFactory & factory); +#endif +void registerFunctionDemangle(FunctionFactory & factory); void registerFunctionTrap(FunctionFactory & factory); void registerFunctionsIntrospection(FunctionFactory & factory) { +#ifdef __ELF__ registerFunctionAddressToSymbol(factory); - registerFunctionDemangle(factory); registerFunctionAddressToLine(factory); +#endif + registerFunctionDemangle(factory); registerFunctionTrap(factory); } From d446205eece30eaf840729d001bfcd05aa775fd4 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 22 Sep 2019 04:53:19 +0300 Subject: [PATCH 093/108] Update QueryProfiler.cpp --- dbms/src/Common/QueryProfiler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index d4e0af90ceb..3060a282cd3 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -54,7 +54,7 @@ namespace } } #else - UNUSED(info); + UNUSED(info); #endif constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag From 6020ba187e78d6a66de8432f9fe549571e5d8a9b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 22 Sep 2019 04:58:21 +0300 Subject: [PATCH 094/108] Update AggregateFunctionGroupBitmapData.h --- .../AggregateFunctionGroupBitmapData.h | 32 +++++++++++-------- 1 file changed, 18 insertions(+), 14 deletions(-) diff --git a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h index e1fbd092490..9a74af530d3 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionGroupBitmapData.h @@ -63,7 +63,12 @@ public: roaring_bitmap_add(rb, value); } - UInt64 size() const { return isSmall() ? small.size() : roaring_bitmap_get_cardinality(rb); } + UInt64 size() const + { + return isSmall() + ? small.size() + : roaring_bitmap_get_cardinality(rb); + } void merge(const RoaringBitmapWithSmallSet & r1) { @@ -91,7 +96,7 @@ public: std::string s; readStringBinary(s,in); rb = roaring_bitmap_portable_deserialize(s.c_str()); - for (const auto & x : small) //merge from small + for (const auto & x : small) // merge from small roaring_bitmap_add(rb, x.getValue()); } else @@ -245,13 +250,13 @@ public: { for (const auto & x : small) if (r1.small.find(x.getValue()) != r1.small.end()) - retSize++; + ++retSize; } else if (isSmall() && r1.isLarge()) { for (const auto & x : small) if (roaring_bitmap_contains(r1.rb, x.getValue())) - retSize++; + ++retSize; } else { @@ -391,8 +396,7 @@ public: */ UInt8 rb_contains(const UInt32 x) const { - return isSmall() ? small.find(x) != small.end() : - roaring_bitmap_contains(rb, x); + return isSmall() ? small.find(x) != small.end() : roaring_bitmap_contains(rb, x); } /** @@ -460,7 +464,7 @@ public: /** * Return new set with specified range (not include the range_end) */ - UInt64 rb_range(UInt32 range_start, UInt32 range_end, RoaringBitmapWithSmallSet& r1) const + UInt64 rb_range(UInt32 range_start, UInt32 range_end, RoaringBitmapWithSmallSet & r1) const { UInt64 count = 0; if (range_start >= range_end) @@ -473,7 +477,7 @@ public: if (UInt32(val) >= range_start && UInt32(val) < range_end) { r1.add(val); - count++; + ++count; } } } @@ -486,7 +490,7 @@ public: { r1.add(iterator.current_value); roaring_advance_uint32_iterator(&iterator); - count++; + ++count; } } return count; @@ -495,7 +499,7 @@ public: /** * Return new set of the smallest `limit` values in set which is no less than `range_start`. */ - UInt64 rb_limit(UInt32 range_start, UInt32 limit, RoaringBitmapWithSmallSet& r1) const + UInt64 rb_limit(UInt32 range_start, UInt32 limit, RoaringBitmapWithSmallSet & r1) const { UInt64 count = 0; if (isSmall()) @@ -512,7 +516,7 @@ public: sort(ans.begin(), ans.end()); if (limit > ans.size()) limit = ans.size(); - for (size_t i=0; i> 16) == 0) From 9afea6f0b388652f44c2572738a0c1f332358088 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 22 Sep 2019 05:04:49 +0300 Subject: [PATCH 095/108] Update HashedDictionary.cpp --- dbms/src/Dictionaries/HashedDictionary.cpp | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 4d195b5139a..9aa111946ee 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -6,17 +6,15 @@ namespace { +/// NOTE: Trailing return type is explicitly specified for SFINAE. + /// google::sparse_hash_map -template auto first(const T &lhs) -> decltype(lhs.first) -{ return lhs.first; } -template auto second(const T &lhs) -> decltype(lhs.second) -{ return lhs.second; } +template auto first(const T & value) -> decltype(value.first) { return lhs.first; } +template auto second(const T & value) -> decltype(value.second) { return lhs.second; } /// HashMap -template auto first(const T &lhs) -> decltype(lhs.getFirst()) -{ return lhs.getFirst(); } -template auto second(const T &lhs) -> decltype(lhs.getSecond()) -{ return lhs.getSecond(); } +template auto first(const T & value) -> decltype(value.getFirst()) { return lhs.getFirst(); } +template auto second(const T & value) -> decltype(value.getSecond()) { return lhs.getSecond(); } } From ab9a7be45ad647e7be832a61f31f03ae90ff9a40 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 22 Sep 2019 05:09:40 +0300 Subject: [PATCH 096/108] Comments are the must. --- dbms/src/Dictionaries/HashedDictionary.h | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index 5cd31ba3e80..fac7bddbb6e 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -13,6 +13,10 @@ #include "IDictionary.h" #include "IDictionarySource.h" +/** This dictionary stores all content in a hash table in memory + * (a separate Key -> Value map for each attribute) + * Two variants of hash table is supported: a fast HashMap and memory efficient sparse_hash_map. + */ namespace DB { From a12cffee91e3c8a423846769be519fce210a29ef Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 22 Sep 2019 05:11:08 +0300 Subject: [PATCH 097/108] Update HashedDictionary.h --- dbms/src/Dictionaries/HashedDictionary.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index fac7bddbb6e..4f9cded40fc 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -15,7 +15,7 @@ /** This dictionary stores all content in a hash table in memory * (a separate Key -> Value map for each attribute) - * Two variants of hash table is supported: a fast HashMap and memory efficient sparse_hash_map. + * Two variants of hash table are supported: a fast HashMap and memory efficient sparse_hash_map. */ namespace DB From 52f14c1ead5d220411291d3ba146b513572a6a0b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 22 Sep 2019 10:39:50 +0300 Subject: [PATCH 098/108] Update HashedDictionary.cpp (fixed rename in return type for SFINAE) --- dbms/src/Dictionaries/HashedDictionary.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Dictionaries/HashedDictionary.cpp b/dbms/src/Dictionaries/HashedDictionary.cpp index 9aa111946ee..7946c87dff8 100644 --- a/dbms/src/Dictionaries/HashedDictionary.cpp +++ b/dbms/src/Dictionaries/HashedDictionary.cpp @@ -9,12 +9,12 @@ namespace /// NOTE: Trailing return type is explicitly specified for SFINAE. /// google::sparse_hash_map -template auto first(const T & value) -> decltype(value.first) { return lhs.first; } -template auto second(const T & value) -> decltype(value.second) { return lhs.second; } +template auto first(const T & value) -> decltype(value.first) { return value.first; } +template auto second(const T & value) -> decltype(value.second) { return value.second; } /// HashMap -template auto first(const T & value) -> decltype(value.getFirst()) { return lhs.getFirst(); } -template auto second(const T & value) -> decltype(value.getSecond()) { return lhs.getSecond(); } +template auto first(const T & value) -> decltype(value.getFirst()) { return value.getFirst(); } +template auto second(const T & value) -> decltype(value.getSecond()) { return value.getSecond(); } } From b174833bdf27e6480bd82e72da453323f3a46a4c Mon Sep 17 00:00:00 2001 From: Eldar Zaitov Date: Sun, 22 Sep 2019 13:34:09 +0300 Subject: [PATCH 099/108] Initial fuzzing support with libfuzzer --- CMakeLists.txt | 8 +++++ cmake/sanitize.cmake | 11 +++++++ dbms/src/Compression/tests/CMakeLists.txt | 6 ++++ .../tests/compressed_buffer_fuzz.cpp | 32 +++++++++++++++++++ release | 3 ++ 5 files changed, 60 insertions(+) create mode 100644 dbms/src/Compression/tests/compressed_buffer_fuzz.cpp diff --git a/CMakeLists.txt b/CMakeLists.txt index 9d82da6838a..448639c17d7 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -91,6 +91,14 @@ if (USE_STATIC_LIBRARIES) list(REVERSE CMAKE_FIND_LIBRARY_SUFFIXES) endif () +option (ENABLE_FUZZING "Enables fuzzing instrumentation" OFF) + +if (ENABLE_FUZZING) + message (STATUS "Fuzzing instrumentation enabled") + set (WITH_COVERAGE ON) + set (SANITIZE "libfuzzer") +endif() + include (cmake/sanitize.cmake) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 196a66e6845..381c186212b 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -42,6 +42,17 @@ if (SANITIZE) if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan") endif () + + elseif (SANITIZE STREQUAL "libfuzzer") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") + endif() + if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan") + endif () + set (LIBFUZZER_CMAKE_CXX_FLAGS "-fsanitize=fuzzer,address,signed-integer-overflow -fsanitize-address-use-after-scope") else () message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () diff --git a/dbms/src/Compression/tests/CMakeLists.txt b/dbms/src/Compression/tests/CMakeLists.txt index 3cfc0ccb7dc..6b13d4eb5cd 100644 --- a/dbms/src/Compression/tests/CMakeLists.txt +++ b/dbms/src/Compression/tests/CMakeLists.txt @@ -3,3 +3,9 @@ target_link_libraries (compressed_buffer PRIVATE dbms) add_executable (cached_compressed_read_buffer cached_compressed_read_buffer.cpp) target_link_libraries (cached_compressed_read_buffer PRIVATE dbms) + +if (ENABLE_FUZZING) + add_executable (compressed_buffer_fuzz compressed_buffer_fuzz.cpp) + target_link_libraries (compressed_buffer_fuzz PRIVATE dbms) + set_target_properties(compressed_buffer_fuzz PROPERTIES LINK_FLAGS ${LIBFUZZER_CMAKE_CXX_FLAGS}) +endif () diff --git a/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp b/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp new file mode 100644 index 00000000000..ff4907dcb2d --- /dev/null +++ b/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp @@ -0,0 +1,32 @@ +#include + +#include +#include +#include +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +extern "C" int LLVMFuzzerTestOneInput(const uint8_t *data, size_t size) { + try { + std::string Str(reinterpret_cast(data), size); + + DB::ReadBufferFromString from(Str); + DB::CompressedReadBuffer in{from}; + } + catch (const DB::Exception & e) + { + std::cerr << e.what() << ", " << e.displayText() << std::endl; + return 1; + } + + return 0; +} diff --git a/release b/release index 270c16f4c36..b5e3f05cd95 100755 --- a/release +++ b/release @@ -87,6 +87,9 @@ then elif [[ "$SANITIZER" == "thread" ]]; then VERSION_POSTFIX+="+tsan" elif [[ "$SANITIZER" == "memory" ]]; then VERSION_POSTFIX+="+msan" elif [[ "$SANITIZER" == "undefined" ]]; then VERSION_POSTFIX+="+ubsan" + elif [[ "$SANITIZER" == "libfuzzer" ]]; then + VERSION_POSTFIX+="+libfuzzer" + MALLOC_OPTS="-DENABLE_TCMALLOC=0 -DENABLE_JEMALLOC=0" else echo "Unknown value of SANITIZER variable: $SANITIZER" exit 3 From 64d49e4565d7beb57c61eb295a5eb6cb81f5b4cd Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sun, 22 Sep 2019 10:42:47 +0000 Subject: [PATCH 100/108] Minor review fixes. --- dbms/src/IO/ReadBufferFromS3.cpp | 4 +- dbms/src/IO/WriteBufferFromS3.cpp | 61 +++--- .../tests/integration/test_storage_s3/test.py | 125 ++++++------ .../test_storage_s3/test_server.py | 188 +++++++++--------- 4 files changed, 192 insertions(+), 186 deletions(-) diff --git a/dbms/src/IO/ReadBufferFromS3.cpp b/dbms/src/IO/ReadBufferFromS3.cpp index 7fcb7a0ca41..ae09f0fb189 100644 --- a/dbms/src/IO/ReadBufferFromS3.cpp +++ b/dbms/src/IO/ReadBufferFromS3.cpp @@ -5,11 +5,11 @@ #include -#define DEFAULT_S3_MAX_FOLLOW_GET_REDIRECT 2 - namespace DB { +const int DEFAULT_S3_MAX_FOLLOW_GET_REDIRECT = 2; + ReadBufferFromS3::ReadBufferFromS3(Poco::URI uri_, const ConnectionTimeouts & timeouts, const Poco::Net::HTTPBasicCredentials & credentials, diff --git a/dbms/src/IO/WriteBufferFromS3.cpp b/dbms/src/IO/WriteBufferFromS3.cpp index 5b6f9fdff4c..1ef6f3b19a0 100644 --- a/dbms/src/IO/WriteBufferFromS3.cpp +++ b/dbms/src/IO/WriteBufferFromS3.cpp @@ -11,12 +11,13 @@ #include -#define DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT 2 -#define S3_SOFT_MAX_PARTS 10000 - namespace DB { +const int DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT = 2; +const int S3_WARN_MAX_PARTS = 10000; + + namespace ErrorCodes { extern const int INCORRECT_DATA; @@ -92,34 +93,33 @@ void WriteBufferFromS3::initiate() { // See https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadInitiate.html Poco::Net::HTTPResponse response; - std::unique_ptr request; + std::unique_ptr request_ptr; HTTPSessionPtr session; std::istream * istr = nullptr; /// owned by session Poco::URI initiate_uri = uri; initiate_uri.setRawQuery("uploads"); - auto params = uri.getQueryParameters(); - for (auto it = params.begin(); it != params.end(); ++it) + for (auto & param: uri.getQueryParameters()) { - initiate_uri.addQueryParameter(it->first, it->second); + initiate_uri.addQueryParameter(param.first, param.second); } for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { session = makeHTTPSession(initiate_uri, timeouts); - request = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, initiate_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - request->setHost(initiate_uri.getHost()); // use original, not resolved host name in header + request_ptr = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, initiate_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request_ptr->setHost(initiate_uri.getHost()); // use original, not resolved host name in header if (auth_request.hasCredentials()) { Poco::Net::HTTPBasicCredentials credentials(auth_request); - credentials.authenticate(*request); + credentials.authenticate(*request_ptr); } - request->setContentLength(0); + request_ptr->setContentLength(0); LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << initiate_uri.toString()); - session->sendRequest(*request); + session->sendRequest(*request_ptr); istr = &session->receiveResponse(response); @@ -134,7 +134,7 @@ void WriteBufferFromS3::initiate() initiate_uri = location_iterator->second; } - assertResponseIsOk(*request, response, *istr); + assertResponseIsOk(*request_ptr, response, *istr); Poco::XML::InputSource src(*istr); Poco::XML::DOMParser parser; @@ -156,37 +156,38 @@ void WriteBufferFromS3::writePart(const String & data) { // See https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadUploadPart.html Poco::Net::HTTPResponse response; - std::unique_ptr request; + std::unique_ptr request_ptr; HTTPSessionPtr session; std::istream * istr = nullptr; /// owned by session Poco::URI part_uri = uri; part_uri.addQueryParameter("partNumber", std::to_string(part_tags.size() + 1)); part_uri.addQueryParameter("uploadId", upload_id); - if (part_tags.size() == S3_SOFT_MAX_PARTS) + if (part_tags.size() == S3_WARN_MAX_PARTS) { + // Don't throw exception here by ourselves but leave the decision to take by S3 server. LOG_WARNING(&Logger::get("WriteBufferFromS3"), "Maximum part number in S3 protocol has reached (too much parts). Server may not accept this whole upload."); } for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { session = makeHTTPSession(part_uri, timeouts); - request = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, part_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - request->setHost(part_uri.getHost()); // use original, not resolved host name in header + request_ptr = std::make_unique(Poco::Net::HTTPRequest::HTTP_PUT, part_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request_ptr->setHost(part_uri.getHost()); // use original, not resolved host name in header if (auth_request.hasCredentials()) { Poco::Net::HTTPBasicCredentials credentials(auth_request); - credentials.authenticate(*request); + credentials.authenticate(*request_ptr); } - request->setExpectContinue(true); + request_ptr->setExpectContinue(true); - request->setContentLength(data.size()); + request_ptr->setContentLength(data.size()); LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << part_uri.toString()); - std::ostream & ostr = session->sendRequest(*request); + std::ostream & ostr = session->sendRequest(*request_ptr); if (session->peekResponse(response)) { // Received 100-continue. @@ -206,7 +207,7 @@ void WriteBufferFromS3::writePart(const String & data) part_uri = location_iterator->second; } - assertResponseIsOk(*request, response, *istr); + assertResponseIsOk(*request_ptr, response, *istr); auto etag_iterator = response.find("ETag"); if (etag_iterator == response.end()) @@ -221,7 +222,7 @@ void WriteBufferFromS3::complete() { // See https://docs.aws.amazon.com/AmazonS3/latest/API/mpUploadComplete.html Poco::Net::HTTPResponse response; - std::unique_ptr request; + std::unique_ptr request_ptr; HTTPSessionPtr session; std::istream * istr = nullptr; /// owned by session Poco::URI complete_uri = uri; @@ -244,22 +245,22 @@ void WriteBufferFromS3::complete() for (int i = 0; i < DEFAULT_S3_MAX_FOLLOW_PUT_REDIRECT; ++i) { session = makeHTTPSession(complete_uri, timeouts); - request = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, complete_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); - request->setHost(complete_uri.getHost()); // use original, not resolved host name in header + request_ptr = std::make_unique(Poco::Net::HTTPRequest::HTTP_POST, complete_uri.getPathAndQuery(), Poco::Net::HTTPRequest::HTTP_1_1); + request_ptr->setHost(complete_uri.getHost()); // use original, not resolved host name in header if (auth_request.hasCredentials()) { Poco::Net::HTTPBasicCredentials credentials(auth_request); - credentials.authenticate(*request); + credentials.authenticate(*request_ptr); } - request->setExpectContinue(true); + request_ptr->setExpectContinue(true); - request->setContentLength(data.size()); + request_ptr->setContentLength(data.size()); LOG_TRACE((&Logger::get("WriteBufferFromS3")), "Sending request to " << complete_uri.toString()); - std::ostream & ostr = session->sendRequest(*request); + std::ostream & ostr = session->sendRequest(*request_ptr); if (session->peekResponse(response)) { // Received 100-continue. @@ -279,7 +280,7 @@ void WriteBufferFromS3::complete() complete_uri = location_iterator->second; } - assertResponseIsOk(*request, response, *istr); + assertResponseIsOk(*request_ptr, response, *istr); } } diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 2013daa6ae6..88be4640388 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -15,7 +15,7 @@ logging.getLogger().addHandler(logging.StreamHandler()) def get_communication_data(started_cluster): - conn = httplib.HTTPConnection(started_cluster.instances['dummy'].ip_address, started_cluster.communication_port) + conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port) conn.request("GET", "/") r = conn.getresponse() raw_data = r.read() @@ -24,7 +24,7 @@ def get_communication_data(started_cluster): def put_communication_data(started_cluster, body): - conn = httplib.HTTPConnection(started_cluster.instances['dummy'].ip_address, started_cluster.communication_port) + conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port) conn.request("PUT", "/", body) r = conn.getresponse() conn.close() @@ -34,29 +34,29 @@ def put_communication_data(started_cluster, body): def started_cluster(): try: cluster = ClickHouseCluster(__file__) - instance = cluster.add_instance('dummy', config_dir="configs", main_configs=['configs/min_chunk_size.xml']) + instance = cluster.add_instance("dummy", config_dir="configs", main_configs=["configs/min_chunk_size.xml"]) cluster.start() cluster.communication_port = 10000 - instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), 'test_server.py'), 'test_server.py') - cluster.bucket = 'abc' - instance.exec_in_container(['python', 'test_server.py', str(cluster.communication_port), cluster.bucket], detach=True) + instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), "test_server.py"), "test_server.py") + cluster.bucket = "abc" + instance.exec_in_container(["python", "test_server.py", str(cluster.communication_port), cluster.bucket], detach=True) cluster.mock_host = instance.ip_address for i in range(10): try: data = get_communication_data(cluster) - cluster.redirecting_to_http_port = data['redirecting_to_http_port'] - cluster.preserving_data_port = data['preserving_data_port'] - cluster.multipart_preserving_data_port = data['multipart_preserving_data_port'] - cluster.redirecting_preserving_data_port = data['redirecting_preserving_data_port'] + cluster.redirecting_to_http_port = data["redirecting_to_http_port"] + cluster.preserving_data_port = data["preserving_data_port"] + cluster.multipart_preserving_data_port = data["multipart_preserving_data_port"] + cluster.redirecting_preserving_data_port = data["redirecting_preserving_data_port"] except: logging.error(traceback.format_exc()) time.sleep(0.5) else: break else: - assert False, 'Could not initialize mock server' + assert False, "Could not initialize mock server" yield cluster @@ -65,92 +65,97 @@ def started_cluster(): def run_query(instance, query, stdin=None): - logging.info('Running query "{}"...'.format(query)) + logging.info("Running query '{}'...".format(query)) result = instance.query(query, stdin=stdin) - logging.info('Query finished') + logging.info("Query finished") return result -def test_get_with_redirect(started_cluster): - instance = started_cluster.instances['dummy'] - format = 'column1 UInt32, column2 UInt32, column3 UInt32' - put_communication_data(started_cluster, '=== Get with redirect test ===') +def test_get_with_redirect(started_cluster): + instance = started_cluster.instances["dummy"] + format = "column1 UInt32, column2 UInt32, column3 UInt32" + + put_communication_data(started_cluster, "=== Get with redirect test ===") query = "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.redirecting_to_http_port, format) stdout = run_query(instance, query) assert list(map(str.split, stdout.splitlines())) == [ - ['42', '87', '44', '160776'], - ['55', '33', '81', '147015'], - ['1', '0', '9', '0'], + ["42", "87", "44", "160776"], + ["55", "33", "81", "147015"], + ["1", "0", "9", "0"], ] -def test_put(started_cluster): - instance = started_cluster.instances['dummy'] - format = 'column1 UInt32, column2 UInt32, column3 UInt32' - logging.info('Phase 3') - put_communication_data(started_cluster, '=== Put test ===') - values = '(1, 2, 3), (3, 2, 1), (78, 43, 45)' +def test_put(started_cluster): + instance = started_cluster.instances["dummy"] + format = "column1 UInt32, column2 UInt32, column3 UInt32" + + logging.info("Phase 3") + put_communication_data(started_cluster, "=== Put test ===") + values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)" put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format, values) run_query(instance, put_query) data = get_communication_data(started_cluster) - received_data_completed = data['received_data_completed'] - received_data = data['received_data'] - finalize_data = data['finalize_data'] - finalize_data_query = data['finalize_data_query'] - assert received_data[-1].decode() == '1,2,3\n3,2,1\n78,43,45\n' + received_data_completed = data["received_data_completed"] + received_data = data["received_data"] + finalize_data = data["finalize_data"] + finalize_data_query = data["finalize_data_query"] + assert received_data[-1].decode() == "1,2,3\n3,2,1\n78,43,45\n" assert received_data_completed - assert finalize_data == '1hello-etag' - assert finalize_data_query == 'uploadId=TEST' + assert finalize_data == "1hello-etag" + assert finalize_data_query == "uploadId=TEST" + def test_put_csv(started_cluster): - instance = started_cluster.instances['dummy'] - format = 'column1 UInt32, column2 UInt32, column3 UInt32' + instance = started_cluster.instances["dummy"] + format = "column1 UInt32, column2 UInt32, column3 UInt32" - put_communication_data(started_cluster, '=== Put test CSV ===') + put_communication_data(started_cluster, "=== Put test CSV ===") put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) - csv_data = '8,9,16\n11,18,13\n22,14,2\n' + csv_data = "8,9,16\n11,18,13\n22,14,2\n" run_query(instance, put_query, stdin=csv_data) data = get_communication_data(started_cluster) - received_data_completed = data['received_data_completed'] - received_data = data['received_data'] - finalize_data = data['finalize_data'] - finalize_data_query = data['finalize_data_query'] + received_data_completed = data["received_data_completed"] + received_data = data["received_data"] + finalize_data = data["finalize_data"] + finalize_data_query = data["finalize_data_query"] assert received_data[-1].decode() == csv_data assert received_data_completed - assert finalize_data == '1hello-etag' - assert finalize_data_query == 'uploadId=TEST' + assert finalize_data == "1hello-etag" + assert finalize_data_query == "uploadId=TEST" + def test_put_with_redirect(started_cluster): - instance = started_cluster.instances['dummy'] - format = 'column1 UInt32, column2 UInt32, column3 UInt32' + instance = started_cluster.instances["dummy"] + format = "column1 UInt32, column2 UInt32, column3 UInt32" - put_communication_data(started_cluster, '=== Put with redirect test ===') - other_values = '(1, 1, 1), (1, 1, 1), (11, 11, 11)' + put_communication_data(started_cluster, "=== Put with redirect test ===") + other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.redirecting_preserving_data_port, started_cluster.bucket, format, other_values) run_query(instance, query) query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format) stdout = run_query(instance, query) assert list(map(str.split, stdout.splitlines())) == [ - ['1', '1', '1', '1'], - ['1', '1', '1', '1'], - ['11', '11', '11', '1331'], + ["1", "1", "1", "1"], + ["1", "1", "1", "1"], + ["11", "11", "11", "1331"], ] data = get_communication_data(started_cluster) - received_data = data['received_data'] - assert received_data[-1].decode() == '1,1,1\n1,1,1\n11,11,11\n' + received_data = data["received_data"] + assert received_data[-1].decode() == "1,1,1\n1,1,1\n11,11,11\n" + def test_multipart_put(started_cluster): - instance = started_cluster.instances['dummy'] - format = 'column1 UInt32, column2 UInt32, column3 UInt32' + instance = started_cluster.instances["dummy"] + format = "column1 UInt32, column2 UInt32, column3 UInt32" - put_communication_data(started_cluster, '=== Multipart test ===') + put_communication_data(started_cluster, "=== Multipart test ===") long_data = [[i, i+1, i+2] for i in range(100000)] - long_values = ''.join([ '{},{},{}\n'.format(x,y,z) for x, y, z in long_data ]) + long_values = "".join([ "{},{},{}\n".format(x,y,z) for x, y, z in long_data ]) put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.multipart_preserving_data_port, started_cluster.bucket, format) run_query(instance, put_query, stdin=long_values) data = get_communication_data(started_cluster) - assert 'multipart_received_data' in data - received_data = data['multipart_received_data'] - assert received_data[-1].decode() == ''.join([ '{},{},{}\n'.format(x, y, z) for x, y, z in long_data ]) - assert 1 < data['multipart_parts'] < 10000 + assert "multipart_received_data" in data + received_data = data["multipart_received_data"] + assert received_data[-1].decode() == "".join([ "{},{},{}\n".format(x, y, z) for x, y, z in long_data ]) + assert 1 < data["multipart_parts"] < 10000 diff --git a/dbms/tests/integration/test_storage_s3/test_server.py b/dbms/tests/integration/test_storage_s3/test_server.py index 3c10445566a..09dfa1ca958 100644 --- a/dbms/tests/integration/test_storage_s3/test_server.py +++ b/dbms/tests/integration/test_storage_s3/test_server.py @@ -25,8 +25,8 @@ import xml.etree.ElementTree logging.getLogger().setLevel(logging.INFO) -file_handler = logging.FileHandler('/var/log/clickhouse-server/test-server.log', 'a', encoding='utf-8') -file_handler.setFormatter(logging.Formatter('%(asctime)s %(message)s')) +file_handler = logging.FileHandler("/var/log/clickhouse-server/test-server.log", "a", encoding="utf-8") +file_handler.setFormatter(logging.Formatter("%(asctime)s %(message)s")) logging.getLogger().addHandler(file_handler) logging.getLogger().addHandler(logging.StreamHandler()) @@ -54,21 +54,21 @@ def GetFreeTCPPortsAndIP(n): ), localhost = GetFreeTCPPortsAndIP(5) data = { - 'redirecting_to_http_port': redirecting_to_http_port, - 'preserving_data_port': preserving_data_port, - 'multipart_preserving_data_port': multipart_preserving_data_port, - 'redirecting_preserving_data_port': redirecting_preserving_data_port, + "redirecting_to_http_port": redirecting_to_http_port, + "preserving_data_port": preserving_data_port, + "multipart_preserving_data_port": multipart_preserving_data_port, + "redirecting_preserving_data_port": redirecting_preserving_data_port, } class SimpleHTTPServerHandler(BaseHTTPRequestHandler): def do_GET(self): - logging.info('GET {}'.format(self.path)) - if self.path == '/milovidov/test.csv': + logging.info("GET {}".format(self.path)) + if self.path == "/milovidov/test.csv": self.send_response(200) - self.send_header('Content-type', 'text/plain') + self.send_header("Content-type", "text/plain") self.end_headers() - self.wfile.write('42,87,44\n55,33,81\n1,0,9\n') + self.wfile.write("42,87,44\n55,33,81\n1,0,9\n") else: self.send_response(404) self.end_headers() @@ -78,27 +78,27 @@ class SimpleHTTPServerHandler(BaseHTTPRequestHandler): class RedirectingToHTTPHandler(BaseHTTPRequestHandler): def do_GET(self): self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{}:{}/milovidov/test.csv'.format(localhost, simple_server_port)) + self.send_header("Content-type", "text/xml") + self.send_header("Location", "http://{}:{}/milovidov/test.csv".format(localhost, simple_server_port)) self.end_headers() - self.wfile.write(r''' + self.wfile.write(r""" TemporaryRedirect Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. storage.yandexcloud.net -'''.encode()) +
""".encode()) self.finish() class PreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' + protocol_version = "HTTP/1.1" def parse_request(self): result = BaseHTTPRequestHandler.parse_request(self) # Adaptation to Python 3. if sys.version_info.major == 2 and result == True: - expect = self.headers.get('Expect', "") + expect = self.headers.get("Expect", "") if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): if not self.handle_expect_100(): return False @@ -109,12 +109,12 @@ class PreservingDataHandler(BaseHTTPRequestHandler): if code in self.responses: message = self.responses[code][0] else: - message = '' - if self.request_version != 'HTTP/0.9': + message = "" + if self.request_version != "HTTP/0.9": self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) def handle_expect_100(self): - logging.info('Received Expect-100') + logging.info("Received Expect-100") self.send_response_only(100) self.end_headers() return True @@ -122,37 +122,37 @@ class PreservingDataHandler(BaseHTTPRequestHandler): def do_POST(self): self.send_response(200) query = urlparse.urlparse(self.path).query - logging.info('PreservingDataHandler POST ?' + query) - if query == 'uploads': - post_data = r''' -TEST'''.encode() - self.send_header('Content-length', str(len(post_data))) - self.send_header('Content-type', 'text/plain') + logging.info("PreservingDataHandler POST ?" + query) + if query == "uploads": + post_data = r""" +TEST""".encode() + self.send_header("Content-length", str(len(post_data))) + self.send_header("Content-type", "text/plain") self.end_headers() self.wfile.write(post_data) else: - post_data = self.rfile.read(int(self.headers.get('Content-Length'))) - self.send_header('Content-type', 'text/plain') + post_data = self.rfile.read(int(self.headers.get("Content-Length"))) + self.send_header("Content-type", "text/plain") self.end_headers() - data['received_data_completed'] = True - data['finalize_data'] = post_data - data['finalize_data_query'] = query + data["received_data_completed"] = True + data["finalize_data"] = post_data + data["finalize_data_query"] = query self.finish() def do_PUT(self): self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('ETag', 'hello-etag') + self.send_header("Content-type", "text/plain") + self.send_header("ETag", "hello-etag") self.end_headers() query = urlparse.urlparse(self.path).query path = urlparse.urlparse(self.path).path - logging.info('Content-Length = ' + self.headers.get('Content-Length')) - logging.info('PUT ' + query) - assert self.headers.get('Content-Length') - assert self.headers['Expect'] == '100-continue' + logging.info("Content-Length = " + self.headers.get("Content-Length")) + logging.info("PUT " + query) + assert self.headers.get("Content-Length") + assert self.headers["Expect"] == "100-continue" put_data = self.rfile.read() - data.setdefault('received_data', []).append(put_data) - logging.info('PUT to {}'.format(path)) + data.setdefault("received_data", []).append(put_data) + logging.info("PUT to {}".format(path)) self.server.storage[path] = put_data self.finish() @@ -160,8 +160,8 @@ class PreservingDataHandler(BaseHTTPRequestHandler): path = urlparse.urlparse(self.path).path if path in self.server.storage: self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('Content-length', str(len(self.server.storage[path]))) + self.send_header("Content-type", "text/plain") + self.send_header("Content-length", str(len(self.server.storage[path]))) self.end_headers() self.wfile.write(self.server.storage[path]) else: @@ -171,13 +171,13 @@ class PreservingDataHandler(BaseHTTPRequestHandler): class MultipartPreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' + protocol_version = "HTTP/1.1" def parse_request(self): result = BaseHTTPRequestHandler.parse_request(self) # Adaptation to Python 3. if sys.version_info.major == 2 and result == True: - expect = self.headers.get('Expect', "") + expect = self.headers.get("Expect", "") if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): if not self.handle_expect_100(): return False @@ -188,78 +188,78 @@ class MultipartPreservingDataHandler(BaseHTTPRequestHandler): if code in self.responses: message = self.responses[code][0] else: - message = '' - if self.request_version != 'HTTP/0.9': + message = "" + if self.request_version != "HTTP/0.9": self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) def handle_expect_100(self): - logging.info('Received Expect-100') + logging.info("Received Expect-100") self.send_response_only(100) self.end_headers() return True def do_POST(self): query = urlparse.urlparse(self.path).query - logging.info('MultipartPreservingDataHandler POST ?' + query) - if query == 'uploads': + logging.info("MultipartPreservingDataHandler POST ?" + query) + if query == "uploads": self.send_response(200) - post_data = r''' -TEST'''.encode() - self.send_header('Content-length', str(len(post_data))) - self.send_header('Content-type', 'text/plain') + post_data = r""" +TEST""".encode() + self.send_header("Content-length", str(len(post_data))) + self.send_header("Content-type", "text/plain") self.end_headers() self.wfile.write(post_data) else: try: - assert query == 'uploadId=TEST' - logging.info('Content-Length = ' + self.headers.get('Content-Length')) - post_data = self.rfile.read(int(self.headers.get('Content-Length'))) + assert query == "uploadId=TEST" + logging.info("Content-Length = " + self.headers.get("Content-Length")) + post_data = self.rfile.read(int(self.headers.get("Content-Length"))) root = xml.etree.ElementTree.fromstring(post_data) - assert root.tag == 'CompleteMultipartUpload' + assert root.tag == "CompleteMultipartUpload" assert len(root) > 1 - content = '' + content = "" for i, part in enumerate(root): - assert part.tag == 'Part' + assert part.tag == "Part" assert len(part) == 2 - assert part[0].tag == 'PartNumber' - assert part[1].tag == 'ETag' + assert part[0].tag == "PartNumber" + assert part[1].tag == "ETag" assert int(part[0].text) == i + 1 - content += self.server.storage['@'+part[1].text] - data.setdefault('multipart_received_data', []).append(content) - data['multipart_parts'] = len(root) + content += self.server.storage["@"+part[1].text] + data.setdefault("multipart_received_data", []).append(content) + data["multipart_parts"] = len(root) self.send_response(200) - self.send_header('Content-type', 'text/plain') + self.send_header("Content-type", "text/plain") self.end_headers() - logging.info('Sending 200') + logging.info("Sending 200") except: - logging.error('Sending 500') + logging.error("Sending 500") self.send_response(500) self.finish() def do_PUT(self): uid = uuid.uuid4() self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('ETag', str(uid)) + self.send_header("Content-type", "text/plain") + self.send_header("ETag", str(uid)) self.end_headers() query = urlparse.urlparse(self.path).query path = urlparse.urlparse(self.path).path - logging.info('Content-Length = ' + self.headers.get('Content-Length')) - logging.info('PUT ' + query) - assert self.headers.get('Content-Length') - assert self.headers['Expect'] == '100-continue' + logging.info("Content-Length = " + self.headers.get("Content-Length")) + logging.info("PUT " + query) + assert self.headers.get("Content-Length") + assert self.headers["Expect"] == "100-continue" put_data = self.rfile.read() - data.setdefault('received_data', []).append(put_data) - logging.info('PUT to {}'.format(path)) - self.server.storage['@'+str(uid)] = put_data + data.setdefault("received_data", []).append(put_data) + logging.info("PUT to {}".format(path)) + self.server.storage["@"+str(uid)] = put_data self.finish() def do_GET(self): path = urlparse.urlparse(self.path).path if path in self.server.storage: self.send_response(200) - self.send_header('Content-type', 'text/plain') - self.send_header('Content-length', str(len(self.server.storage[path]))) + self.send_header("Content-type", "text/plain") + self.send_header("Content-length", str(len(self.server.storage[path]))) self.end_headers() self.wfile.write(self.server.storage[path]) else: @@ -269,13 +269,13 @@ class MultipartPreservingDataHandler(BaseHTTPRequestHandler): class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): - protocol_version = 'HTTP/1.1' + protocol_version = "HTTP/1.1" def parse_request(self): result = BaseHTTPRequestHandler.parse_request(self) # Adaptation to Python 3. if sys.version_info.major == 2 and result == True: - expect = self.headers.get('Expect', "") + expect = self.headers.get("Expect", "") if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"): if not self.handle_expect_100(): return False @@ -286,46 +286,46 @@ class RedirectingPreservingDataHandler(BaseHTTPRequestHandler): if code in self.responses: message = self.responses[code][0] else: - message = '' - if self.request_version != 'HTTP/0.9': + message = "" + if self.request_version != "HTTP/0.9": self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message)) def handle_expect_100(self): - logging.info('Received Expect-100') + logging.info("Received Expect-100") return True def do_POST(self): query = urlparse.urlparse(self.path).query if query: - query = '?{}'.format(query) + query = "?{}".format(query) self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) + self.send_header("Content-type", "text/xml") + self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) self.end_headers() - self.wfile.write(r''' + self.wfile.write(r""" TemporaryRedirect Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. {host}:{port} -'''.format(host=localhost, port=preserving_data_port).encode()) +""".format(host=localhost, port=preserving_data_port).encode()) self.finish() def do_PUT(self): query = urlparse.urlparse(self.path).query if query: - query = '?{}'.format(query) + query = "?{}".format(query) self.send_response(307) - self.send_header('Content-type', 'text/xml') - self.send_header('Location', 'http://{host}:{port}/{bucket}/test.csv{query}'.format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) + self.send_header("Content-type", "text/xml") + self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query)) self.end_headers() - self.wfile.write(r''' + self.wfile.write(r""" TemporaryRedirect Please re-send this request to the specified temporary endpoint. Continue to use the original request endpoint for future requests. {host}:{port} -'''.format(host=localhost, port=preserving_data_port).encode()) +""".format(host=localhost, port=preserving_data_port).encode()) self.finish() @@ -357,8 +357,8 @@ jobs = [ threading.Thread(target=server.serve_forever) for server in servers ] time.sleep(60) # Timeout -logging.info('Shutting down') +logging.info("Shutting down") [ server.shutdown() for server in servers ] -logging.info('Joining threads') +logging.info("Joining threads") [ job.join() for job in jobs ] -logging.info('Done') +logging.info("Done") From 4608da13449dacbeabda77ec9d7d10bb8db1358b Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Sun, 22 Sep 2019 11:03:02 +0000 Subject: [PATCH 101/108] Minor tests improvement. --- dbms/tests/integration/test_storage_s3/test.py | 8 +++----- dbms/tests/integration/test_storage_s3/test_server.py | 3 ++- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/dbms/tests/integration/test_storage_s3/test.py b/dbms/tests/integration/test_storage_s3/test.py index 88be4640388..84f6bf72f60 100644 --- a/dbms/tests/integration/test_storage_s3/test.py +++ b/dbms/tests/integration/test_storage_s3/test.py @@ -78,11 +78,9 @@ def test_get_with_redirect(started_cluster): put_communication_data(started_cluster, "=== Get with redirect test ===") query = "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.redirecting_to_http_port, format) stdout = run_query(instance, query) - assert list(map(str.split, stdout.splitlines())) == [ - ["42", "87", "44", "160776"], - ["55", "33", "81", "147015"], - ["1", "0", "9", "0"], - ] + data = get_communication_data(started_cluster) + expected = [ [str(row[0]), str(row[1]), str(row[2]), str(row[0]*row[1]*row[2])] for row in data["redirect_csv_data"] ] + assert list(map(str.split, stdout.splitlines())) == expected def test_put(started_cluster): diff --git a/dbms/tests/integration/test_storage_s3/test_server.py b/dbms/tests/integration/test_storage_s3/test_server.py index 09dfa1ca958..8896af9c23e 100644 --- a/dbms/tests/integration/test_storage_s3/test_server.py +++ b/dbms/tests/integration/test_storage_s3/test_server.py @@ -68,7 +68,8 @@ class SimpleHTTPServerHandler(BaseHTTPRequestHandler): self.send_response(200) self.send_header("Content-type", "text/plain") self.end_headers() - self.wfile.write("42,87,44\n55,33,81\n1,0,9\n") + data["redirect_csv_data"] = [[42, 87, 44], [55, 33, 81], [1, 0, 9]] + self.wfile.write("".join([ "{},{},{}\n".format(*row) for row in data["redirect_csv_data"]])) else: self.send_response(404) self.end_headers() From 8fd66ac4fd6a9c992bec170bf79f3f16a382ea65 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 22 Sep 2019 15:52:33 +0300 Subject: [PATCH 102/108] Update compressed_buffer_fuzz.cpp --- .../tests/compressed_buffer_fuzz.cpp | 42 +++++++------------ 1 file changed, 16 insertions(+), 26 deletions(-) diff --git a/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp b/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp index ff4907dcb2d..8e0e529ff8e 100644 --- a/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp +++ b/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp @@ -1,32 +1,22 @@ -#include - #include -#include -#include -#include - -#include -#include -#include -#include -#include -#include +#include #include -#include -#include +#include -extern "C" int LLVMFuzzerTestOneInput(const uint8_t *data, size_t size) { - try { - std::string Str(reinterpret_cast(data), size); - - DB::ReadBufferFromString from(Str); - DB::CompressedReadBuffer in{from}; - } - catch (const DB::Exception & e) - { - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; - } +extern "C" int LLVMFuzzerTestOneInput(const uint8_t * data, size_t size) +try +{ + DB::ReadBufferFromMemory from(data, size); + DB::CompressedReadBuffer in{from}; + + while (!in.eof()) + in.next(); + return 0; } +catch (...) +{ + std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; + return 1; +} From 3270ad439a82392fcc3bb05c1173bce7f53d148f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 23 Sep 2019 00:18:14 +0300 Subject: [PATCH 103/108] Update compressed_buffer_fuzz.cpp --- dbms/src/Compression/tests/compressed_buffer_fuzz.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp b/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp index 8e0e529ff8e..a87046eff5c 100644 --- a/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp +++ b/dbms/src/Compression/tests/compressed_buffer_fuzz.cpp @@ -9,10 +9,10 @@ try { DB::ReadBufferFromMemory from(data, size); DB::CompressedReadBuffer in{from}; - + while (!in.eof()) in.next(); - + return 0; } catch (...) From c5f9b4ad0a6c210339cb966e782b863387eea5c8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 23 Sep 2019 00:21:41 +0300 Subject: [PATCH 104/108] Update sanitize.cmake --- cmake/sanitize.cmake | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 381c186212b..6bd97b925f2 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -44,8 +44,10 @@ if (SANITIZE) endif () elseif (SANITIZE STREQUAL "libfuzzer") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") + # NOTE: Eldar Zaitov decided to name it "libfuzzer" instead of "fuzzer" to keep in mind another possible fuzzer backends. + # NOTE: no-link means that all the targets are built with instrumentation for fuzzer, but only some of them (tests) have entry point for fuzzer and it's not checked. + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,undefined -fsanitize-address-use-after-scope") + set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,undefined -fsanitize-address-use-after-scope") if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") endif() From b0dd36db7c3241251dcf13dea59759cd80e432d8 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Mon, 23 Sep 2019 00:22:22 +0300 Subject: [PATCH 105/108] Update sanitize.cmake --- cmake/sanitize.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index 6bd97b925f2..9e8ef3e857a 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -49,12 +49,12 @@ if (SANITIZE) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,undefined -fsanitize-address-use-after-scope") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=fuzzer-no-link,address,undefined -fsanitize-address-use-after-scope") if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=fuzzer-no-link,address,signed-integer-overflow -fsanitize-address-use-after-scope") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=fuzzer-no-link,address,undefined -fsanitize-address-use-after-scope") endif() if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan -static-libubsan") endif () - set (LIBFUZZER_CMAKE_CXX_FLAGS "-fsanitize=fuzzer,address,signed-integer-overflow -fsanitize-address-use-after-scope") + set (LIBFUZZER_CMAKE_CXX_FLAGS "-fsanitize=fuzzer,address,undefined -fsanitize-address-use-after-scope") else () message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") endif () From 70bc89557fd5b7bda192045a5d63a6c1a933e1a2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 23 Sep 2019 01:57:06 +0300 Subject: [PATCH 106/108] Updates for #6969 --- dbms/programs/server/MetricsTransmitter.cpp | 6 +++--- dbms/programs/server/MetricsTransmitter.h | 8 ++++---- dbms/programs/server/config.xml | 4 ++-- docs/en/operations/server_settings/settings.md | 6 +++--- docs/ru/operations/server_settings/settings.md | 6 +++--- 5 files changed, 15 insertions(+), 15 deletions(-) diff --git a/dbms/programs/server/MetricsTransmitter.cpp b/dbms/programs/server/MetricsTransmitter.cpp index 73413cad1c0..17c759ce98c 100644 --- a/dbms/programs/server/MetricsTransmitter.cpp +++ b/dbms/programs/server/MetricsTransmitter.cpp @@ -21,7 +21,7 @@ MetricsTransmitter::MetricsTransmitter( { interval_seconds = config.getInt(config_name + ".interval", 60); send_events = config.getBool(config_name + ".events", true); - send_events_absolute = config.getBool(config_name + ".events_absolute", false); + send_events_cumulative = config.getBool(config_name + ".events_cumulative", false); send_metrics = config.getBool(config_name + ".metrics", true); send_asynchronous_metrics = config.getBool(config_name + ".asynchronous_metrics", true); } @@ -96,13 +96,13 @@ void MetricsTransmitter::transmit(std::vector & prev_count } } - if (send_events_absolute) + if (send_events_cumulative) { for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); std::string key{ProfileEvents::getName(static_cast(i))}; - key_vals.emplace_back(profile_events_absolute_path_prefix + key, counter); + key_vals.emplace_back(profile_events_cumulative_path_prefix + key, counter); } } diff --git a/dbms/programs/server/MetricsTransmitter.h b/dbms/programs/server/MetricsTransmitter.h index 1d5795c24db..79840616dd3 100644 --- a/dbms/programs/server/MetricsTransmitter.h +++ b/dbms/programs/server/MetricsTransmitter.h @@ -24,8 +24,8 @@ class AsynchronousMetrics; /** Automatically sends - * - values deltas of ProfileEvents; - * - absolute values of ProfileEvents; + * - delta values of ProfileEvents; + * - cumulative values of ProfileEvents; * - values of CurrentMetrics; * - values of AsynchronousMetrics; * to Graphite at beginning of every minute. @@ -45,7 +45,7 @@ private: std::string config_name; UInt32 interval_seconds; bool send_events; - bool send_events_absolute; + bool send_events_cumulative; bool send_metrics; bool send_asynchronous_metrics; @@ -55,7 +55,7 @@ private: ThreadFromGlobalPool thread{&MetricsTransmitter::run, this}; static inline constexpr auto profile_events_path_prefix = "ClickHouse.ProfileEvents."; - static inline constexpr auto profile_events_absolute_path_prefix = "ClickHouse.ProfileEventsAbsolute."; + static inline constexpr auto profile_events_cumulative_path_prefix = "ClickHouse.ProfileEventsCumulative."; static inline constexpr auto current_metrics_path_prefix = "ClickHouse.Metrics."; static inline constexpr auto asynchronous_metrics_path_prefix = "ClickHouse.AsynchronousMetrics."; }; diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index 7263992b3d3..c8d33922167 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -258,7 +258,7 @@ true true - false + false true @@ -270,7 +270,7 @@ true true - false + false false --> diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index baf9013fbbf..f884a7b2963 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -141,8 +141,8 @@ Settings: - timeout – The timeout for sending data, in seconds. - root_path – Prefix for keys. - metrics – Sending data from a :ref:`system_tables-system.metrics` table. -- events – Sending deltas data from a :ref:`system_tables-system.events` table -- events_absolute – Sending absolute data from a :ref:`system_tables-system.events` table +- events – Sending deltas data accumulated for the time period from a :ref:`system_tables-system.events` table +- events_cumulative – Sending cumulative data from a :ref:`system_tables-system.events` table - asynchronous_metrics – Sending data from a :ref:`system_tables-system.asynchronous_metrics` table. You can configure multiple `` clauses. For instance, you can use this for sending different data at different intervals. @@ -158,7 +158,7 @@ You can configure multiple `` clauses. For instance, you can use this one_min true true - false + false true ``` diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index 2f6362b7635..39523db7d36 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -140,8 +140,8 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat - timeout - Таймаут отправки данных в секундах. - root_path - Префикс для ключей. - metrics - Отправка данных из таблицы :ref:`system_tables-system.metrics`. -- events - Отправка дельты данных из таблицы :ref:`system_tables-system.events` -- events_absolute - Отправка абсолютных данных из таблицы :ref:`system_tables-system.events` +- events - Отправка дельты данных, накопленной за промежуток времени из таблицы :ref:`system_tables-system.events` +- events_cumulative - Отправка суммарных данных из таблицы :ref:`system_tables-system.events` - asynchronous_metrics - Отправка данных из таблицы :ref:`system_tables-system.asynchronous_metrics`. Можно определить несколько секций ``, например, для передачи различных данных с различной частотой. @@ -157,7 +157,7 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat one_min true true - false + false true ``` From 7a822ad815f725f275202f57f15b541d9f73e8ac Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 23 Sep 2019 14:08:56 +0300 Subject: [PATCH 107/108] fix Jinja2 (#7011) --- docs/tools/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/tools/requirements.txt b/docs/tools/requirements.txt index b216433f772..2c395da402c 100644 --- a/docs/tools/requirements.txt +++ b/docs/tools/requirements.txt @@ -12,7 +12,7 @@ futures==3.1.1 htmlmin==0.1.12 idna==2.6 imagesize==0.7.1 -Jinja2==2.10 +Jinja2==2.10.1 jsmin==2.2.2 livereload==2.5.1 Markdown==2.6.11 From 6b6714761b2cc85c8e28d0cd4df40c90376e06d3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 23 Sep 2019 15:20:08 +0300 Subject: [PATCH 108/108] Also pack config into shared archive --- docker/packager/binary/build.sh | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docker/packager/binary/build.sh b/docker/packager/binary/build.sh index ed30feb8cb7..b5c50763b17 100755 --- a/docker/packager/binary/build.sh +++ b/docker/packager/binary/build.sh @@ -18,6 +18,10 @@ find . -name '*.so.*' -print -exec mv '{}' /output \; count=`ls -1 /output/*.so 2>/dev/null | wc -l` if [ $count != 0 ] then + mkdir -p /output/config + cp ../dbms/programs/server/config.xml /output/config + cp ../dbms/programs/server/users.xml /output/config + cp -r ../dbms/programs/server/config.d /output/config tar -czvf shared_build.tgz /output rm -r /output/* mv shared_build.tgz /output