From 97788b9c21308fe912a356497be35ad8b439233a Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 29 Dec 2021 21:03:15 +0300 Subject: [PATCH 01/37] Allow to create new files on insert for File/S3/HDFS engines --- src/Common/ErrorCodes.cpp | 1 + src/Core/Settings.h | 5 + src/Formats/FormatFactory.cpp | 20 +++ src/Formats/FormatFactory.h | 13 ++ .../Formats/Impl/ArrowBlockOutputFormat.cpp | 2 + .../Impl/CustomSeparatedRowOutputFormat.cpp | 5 + .../Formats/Impl/JSONRowOutputFormat.cpp | 2 + .../Formats/Impl/ORCBlockOutputFormat.cpp | 1 + .../Formats/Impl/ParquetBlockOutputFormat.cpp | 1 + .../Impl/TemplateBlockOutputFormat.cpp | 14 ++ .../Formats/Impl/XMLRowOutputFormat.cpp | 1 + src/Storages/HDFS/StorageHDFS.cpp | 127 ++++++++++++--- src/Storages/HDFS/StorageHDFS.h | 16 +- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 13 +- src/Storages/StorageFile.cpp | 49 +++++- src/Storages/StorageFile.h | 2 + src/Storages/StorageS3.cpp | 150 +++++++++++++++--- src/Storages/StorageS3.h | 17 +- tests/integration/test_storage_hdfs/test.py | 37 +++++ tests/integration/test_storage_s3/test.py | 42 +++++ ..._inserts_for_formats_with_suffix.reference | 100 ++++++++++++ ...ltiple_inserts_for_formats_with_suffix.sql | 39 +++++ 22 files changed, 592 insertions(+), 65 deletions(-) create mode 100644 tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.reference create mode 100644 tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.sql diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp index 16f85fcae61..f568955e531 100644 --- a/src/Common/ErrorCodes.cpp +++ b/src/Common/ErrorCodes.cpp @@ -605,6 +605,7 @@ M(634, MONGODB_ERROR) \ M(635, CANNOT_POLL) \ M(636, CANNOT_EXTRACT_TABLE_STRUCTURE) \ + M(637, CANNOT_APPEND_TO_FILE) \ \ M(999, KEEPER_EXCEPTION) \ M(1000, POCO_EXCEPTION) \ diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6e53fa4342c..ad43c2a04d7 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -75,7 +75,11 @@ class IColumn; M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ + M(Bool, s3_truncate_on_insert, false, "", 0) \ + M(Bool, s3_create_new_file_on_insert, false, "", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ + M(Bool, hdfs_truncate_on_insert, false, "", 0) \ + M(Bool, hdfs_create_new_file_on_insert, false, "", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ @@ -490,6 +494,7 @@ class IColumn; \ M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ + M(Bool, engine_file_allow_create_multiple_files, false, ".", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index a0a5550627d..baf825c2bbc 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -383,6 +383,26 @@ void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name target = std::move(non_trivial_prefix_and_suffix_checker); } +void FormatFactory::registerSuffixChecker(const String & name, SuffixChecker suffix_checker) +{ + auto & target = dict[name].suffix_checker; + if (target) + throw Exception("FormatFactory: Suffix checker " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); + target = std::move(suffix_checker); +} + +void FormatFactory::markFormatWithSuffix(const String & name) +{ + registerSuffixChecker(name, [](const FormatSettings &){ return true; }); +} + +bool FormatFactory::checkIfFormatHasSuffix(const String & name, ContextPtr context, const std::optional & format_settings_) +{ + auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); + auto & suffix_checker = dict[name].suffix_checker; + return suffix_checker && suffix_checker(format_settings); +} + void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator) { auto & target = dict[name].output_creator; diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index a62b32da0cc..860b1e19ae6 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -92,6 +92,10 @@ private: /// The checker should return true if parallel parsing should be disabled. using NonTrivialPrefixAndSuffixChecker = std::function; + /// Some formats can have suffix after data depending on settings. + /// The checker should return true if format will write some suffix after data. + using SuffixChecker = std::function; + using SchemaReaderCreator = std::function; using ExternalSchemaReaderCreator = std::function; @@ -105,6 +109,7 @@ private: bool supports_parallel_formatting{false}; bool is_column_oriented{false}; NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker; + SuffixChecker suffix_checker; }; using FormatsDictionary = std::unordered_map; @@ -165,6 +170,14 @@ public: void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker); + void registerSuffixChecker(const String & name, SuffixChecker suffix_checker); + + /// If format always contains suffix, you an use this method instead of + /// registerSuffixChecker with suffix_checker that always returns true. + void markFormatWithSuffix(const String & name); + + bool checkIfFormatHasSuffix(const String & name, ContextPtr context, const std::optional & format_settings_ = std::nullopt); + /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); void registerOutputFormat(const String & name, OutputCreator output_creator); diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 692f17f843a..b60d38f317c 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -93,6 +93,7 @@ void registerOutputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, false, format_settings); }); + factory.markFormatWithSuffix("Arrow"); factory.registerOutputFormat( "ArrowStream", @@ -103,6 +104,7 @@ void registerOutputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, true, format_settings); }); + factory.markFormatWithSuffix("ArrowStream"); } } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index 21cb549d4cb..3fb7c0a36a3 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -91,6 +91,11 @@ void registerOutputFormatCustomSeparated(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting(format_name); + + factory.registerSuffixChecker(format_name, [](const FormatSettings & settings) + { + return !settings.custom.result_after_delimiter.empty(); + }); }; registerWithNamesAndTypes("CustomSeparated", register_func); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 8e2b2617c4c..577efdb1a21 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -284,6 +284,7 @@ void registerOutputFormatJSON(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSON"); + factory.markFormatWithSuffix("JSON"); factory.registerOutputFormat("JSONStrings", []( WriteBuffer & buf, @@ -295,6 +296,7 @@ void registerOutputFormatJSON(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONStrings"); + factory.markFormatWithSuffix("JSONStrings"); } } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 651b9545c81..33d4b3e568d 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -526,6 +526,7 @@ void registerOutputFormatORC(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings); }); + factory.markFormatWithSuffix("ORC"); } } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index a10858ee668..96c3a80b732 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -85,6 +85,7 @@ void registerOutputFormatParquet(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings); }); + factory.markFormatWithSuffix("Parquet"); } } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index d981b92e1dd..eded88298e8 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -235,5 +235,19 @@ void registerOutputFormatTemplate(FormatFactory & factory) return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); + + factory.registerSuffixChecker("Template", [](const FormatSettings & settings) + { + if (settings.template_settings.resultset_format.empty()) + return false; + auto resultset_format = ParsedTemplateFormatString( + FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false, + settings.schema.is_server, settings.schema.format_schema_path), + [&](const String & partName) + { + return static_cast(TemplateBlockOutputFormat::stringToResultsetPart(partName)); + }); + return !resultset_format.delimiters.empty() && !resultset_format.delimiters.back().empty(); + }); } } diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index d96981fc091..56bbbc0673c 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -256,6 +256,7 @@ void registerOutputFormatXML(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("XML"); + factory.markFormatWithSuffix("XML"); } } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index f22f6f66ced..36287b92855 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -14,9 +14,8 @@ #include #include -#include +#include -#include #include #include #include @@ -28,7 +27,6 @@ #include #include - #include #include #include @@ -52,6 +50,7 @@ namespace ErrorCodes { extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int ACCESS_DENIED; + extern const int DATABASE_ACCESS_DENIED; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } namespace @@ -130,20 +129,23 @@ StorageHDFS::StorageHDFS( ASTPtr partition_by_) : IStorage(table_id_) , WithContext(context_) - , uri(uri_) + , uris({uri_}) , format_name(format_name_) , compression_method(compression_method_) , distributed_processing(distributed_processing_) , partition_by(partition_by_) { - context_->getRemoteHostFilter().checkURL(Poco::URI(uri)); - checkHDFSURL(uri); + context_->getRemoteHostFilter().checkURL(Poco::URI(uri_)); + checkHDFSURL(uri_); + + String path = uri_.substr(uri_.find('/', uri_.find("//") + 2)); + is_path_with_globs = path.find_first_of("*?{") != std::string::npos; StorageInMemoryMetadata storage_metadata; if (columns_.empty()) { - auto columns = getTableStructureFromData(format_name, uri, compression_method, context_); + auto columns = getTableStructureFromData(format_name, uri_, compression_method, context_); storage_metadata.setColumns(columns); } else @@ -208,6 +210,25 @@ private: Strings::iterator uris_iter; }; +class HDFSSource::URISIterator::Impl +{ +public: + Impl(const std::vector & uris_) : uris(uris_), index(0) + { + } + + String next() + { + if (index == uris.size()) + return ""; + return uris[index++]; + } + +private: + const std::vector & uris; + size_t index; +}; + Block HDFSSource::getHeader(const StorageMetadataPtr & metadata_snapshot, bool need_path_column, bool need_file_column) { auto header = metadata_snapshot->getSampleBlock(); @@ -241,6 +262,15 @@ String HDFSSource::DisclosedGlobIterator::next() return pimpl->next(); } +HDFSSource::URISIterator::URISIterator(const std::vector & uris_) + : pimpl(std::make_shared(uris_)) +{ +} + +String HDFSSource::URISIterator::next() +{ + return pimpl->next(); +} HDFSSource::HDFSSource( StorageHDFSPtr storage_, @@ -275,9 +305,8 @@ bool HDFSSource::initialize() current_path = (*file_iterator)(); if (current_path.empty()) return false; - const size_t begin_of_path = current_path.find('/', current_path.find("//") + 2); - const String path_from_uri = current_path.substr(begin_of_path); - const String uri_without_path = current_path.substr(0, begin_of_path); + + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_path); auto compression = chooseCompressionMethod(path_from_uri, storage->compression_method); read_buf = wrapReadBufferWithCompressionMethod(std::make_unique(uri_without_path, path_from_uri, getContext()->getGlobalContext()->getConfigRef()), compression); @@ -460,15 +489,23 @@ Pipe StorageHDFS::read( return callback(); }); } - else + else if (is_path_with_globs) { /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(context_, uri); + auto glob_iterator = std::make_shared(context_, uris[0]); iterator_wrapper = std::make_shared([glob_iterator]() { return glob_iterator->next(); }); } + else + { + auto uris_iterator = std::make_shared(uris); + iterator_wrapper = std::make_shared([uris_iterator]() + { + return uris_iterator->next(); + }); + } Pipes pipes; auto this_ptr = std::static_pointer_cast(shared_from_this()); @@ -496,9 +533,44 @@ Pipe StorageHDFS::read( return Pipe::unitePipes(std::move(pipes)); } -SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/) +SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_) { - bool has_wildcards = uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; + if (is_path_with_globs) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); + + String current_uri = uris.back(); + + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_uri); + + HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + + bool truncate_on_insert = context_->getSettingsRef().hdfs_truncate_on_insert; + if (!truncate_on_insert && !hdfsExists(fs.get(), path_from_uri.c_str())) + { + if (context_->getSettingsRef().hdfs_create_new_file_on_insert) + { + auto pos = uris[0].find_first_of('.', uris[0].find_last_of('/')); + size_t index = uris.size(); + String new_uri; + do + { + new_uri = uris[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : uris[0].substr(pos)); + ++index; + } + while (!hdfsExists(fs.get(), new_uri.c_str())); + uris.push_back(new_uri); + current_uri = new_uri; + } + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "File with path {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert, " + "if you want to create new file on each insert, enable setting hdfs_create_new_file_on_insert", + path_from_uri); + } + + bool has_wildcards = current_uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; const auto * insert_query = dynamic_cast(query.get()); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; bool is_partitioned_implementation = partition_by_ast && has_wildcards; @@ -507,34 +579,37 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP { return std::make_shared( partition_by_ast, - uri, + current_uri, format_name, metadata_snapshot->getSampleBlock(), - getContext(), - chooseCompressionMethod(uri, compression_method)); + context_, + chooseCompressionMethod(current_uri, compression_method)); } else { - return std::make_shared(uri, + return std::make_shared(current_uri, format_name, metadata_snapshot->getSampleBlock(), - getContext(), - chooseCompressionMethod(uri, compression_method)); + context_, + chooseCompressionMethod(current_uri, compression_method)); } } void StorageHDFS::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, ContextPtr local_context, TableExclusiveLockHolder &) { - const size_t begin_of_path = uri.find('/', uri.find("//") + 2); - const String path = uri.substr(begin_of_path); - const String url = uri.substr(0, begin_of_path); + const size_t begin_of_path = uris[0].find('/', uris[0].find("//") + 2); + const String url = uris[0].substr(0, begin_of_path); HDFSBuilderWrapper builder = createHDFSBuilder(url + "/", local_context->getGlobalContext()->getConfigRef()); HDFSFSPtr fs = createHDFSFS(builder.get()); - int ret = hdfsDelete(fs.get(), path.data(), 0); - if (ret) - throw Exception(ErrorCodes::ACCESS_DENIED, "Unable to truncate hdfs table: {}", std::string(hdfsGetLastError())); + for (const auto & uri : uris) + { + const String path = uri.substr(begin_of_path); + int ret = hdfsDelete(fs.get(), path.data(), 0); + if (ret) + throw Exception(ErrorCodes::ACCESS_DENIED, "Unable to truncate hdfs table: {}", std::string(hdfsGetLastError())); + } } diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 9e845d8fd74..53be640248c 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -31,7 +31,7 @@ public: size_t max_block_size, unsigned num_streams) override; - SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override; + SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override; void truncate( const ASTPtr & query, @@ -70,11 +70,12 @@ protected: ASTPtr partition_by = nullptr); private: - const String uri; + std::vector uris; String format_name; String compression_method; const bool distributed_processing; ASTPtr partition_by; + bool is_path_with_globs; Poco::Logger * log = &Poco::Logger::get("StorageHDFS"); }; @@ -95,6 +96,17 @@ public: std::shared_ptr pimpl; }; + class URISIterator + { + public: + URISIterator(const std::vector & uris_); + String next(); + private: + class Impl; + /// shared_ptr to have copy constructor + std::shared_ptr pimpl; + }; + using IteratorWrapper = std::function; using StorageHDFSPtr = std::shared_ptr; diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 9f5e3c1f7d2..a5050953b7c 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -38,11 +38,14 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); const String path = hdfs_uri.substr(begin_of_path); - if (path.find_first_of("*?{") != std::string::npos) - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "URI '{}' contains globs, so the table is in readonly mode", hdfs_uri); - - if (!hdfsExists(fs.get(), path.c_str())) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} already exists", path); +// if (path.find_first_of("*?{") != std::string::npos) +// throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "URI '{}' contains globs, so the table is in readonly mode", hdfs_uri); +// +// if (!hdfsExists(fs.get(), path.c_str()) && !truncate_) +// throw Exception( +// ErrorCodes::BAD_ARGUMENTS, +// "File {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert", +// path); fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index a479f982c70..586b8d97875 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -65,6 +65,7 @@ namespace ErrorCodes extern const int INCOMPATIBLE_COLUMNS; extern const int CANNOT_STAT; extern const int LOGICAL_ERROR; + extern const int CANNOT_APPEND_TO_FILE; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } @@ -285,6 +286,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us { is_db_table = false; paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read); + is_path_with_globs = paths.size() > 1; path_for_partitioned_write = table_path_; setStorageMetadata(args); } @@ -666,10 +668,9 @@ public: } else { - if (paths.size() != 1) - throw Exception("Table '" + table_name_for_log + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); + assert(!paths.empty()); flags |= O_WRONLY | O_APPEND | O_CREAT; - naked_buffer = std::make_unique(paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags); + naked_buffer = std::make_unique(paths.back(), DBMS_DEFAULT_BUFFER_SIZE, flags); } /// In case of formats with prefixes if file is not empty we have already written prefix. @@ -827,6 +828,35 @@ SinkToStoragePtr StorageFile::write( { path = paths[0]; fs::create_directories(fs::path(path).parent_path()); + + if (is_path_with_globs) + throw Exception("Table '" + getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); + + if (!context->getSettingsRef().engine_file_truncate_on_insert && !is_path_with_globs + && FormatFactory::instance().checkIfFormatHasSuffix(format_name, context, format_settings) && fs::exists(paths.back()) + && fs::file_size(paths.back()) != 0) + { + if (context->getSettingsRef().engine_file_allow_create_multiple_files) + { + auto pos = paths[0].find_first_of('.', paths[0].find_last_of('/')); + size_t index = paths.size(); + String new_path; + do + { + new_path = paths[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : paths[0].substr(pos)); + ++index; + } + while (fs::exists(new_path)); + paths.push_back(new_path); + } + else + throw Exception( + ErrorCodes::CANNOT_APPEND_TO_FILE, + "Cannot append data in format {} to file, because this format contains suffix and " + "data can be written to a file only once. You can allow to create a new file " + "on each insert by enabling setting engine_file_allow_create_multiple_files", + format_name); + } } return std::make_shared( @@ -882,7 +912,7 @@ void StorageFile::truncate( ContextPtr /* context */, TableExclusiveLockHolder &) { - if (paths.size() != 1) + if (is_path_with_globs) throw Exception("Can't truncate table '" + getStorageID().getNameForLogs() + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); if (use_table_fd) @@ -892,11 +922,14 @@ void StorageFile::truncate( } else { - if (!fs::exists(paths[0])) - return; + for (const auto & path : paths) + { + if (!fs::exists(path)) + continue; - if (0 != ::truncate(paths[0].c_str(), 0)) - throwFromErrnoWithPath("Cannot truncate file " + paths[0], paths[0], ErrorCodes::CANNOT_TRUNCATE_FILE); + if (0 != ::truncate(path.c_str(), 0)) + throwFromErrnoWithPath("Cannot truncate file " + path, path, ErrorCodes::CANNOT_TRUNCATE_FILE); + } } } diff --git a/src/Storages/StorageFile.h b/src/Storages/StorageFile.h index 6b015976589..8564d93ccc2 100644 --- a/src/Storages/StorageFile.h +++ b/src/Storages/StorageFile.h @@ -120,6 +120,8 @@ private: size_t total_bytes_to_read = 0; String path_for_partitioned_write; + + bool is_path_with_globs = false; }; } diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 3d988472b54..ea563937ca6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -68,7 +68,7 @@ namespace ErrorCodes extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int S3_ERROR; extern const int UNEXPECTED_EXPRESSION; - extern const int CANNOT_OPEN_FILE; + extern const int DATABASE_ACCESS_DENIED; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; } @@ -176,6 +176,33 @@ String StorageS3Source::DisclosedGlobIterator::next() return pimpl->next(); } +class StorageS3Source::KeysIterator::Impl +{ +public: + explicit Impl(const std::vector & keys_) : keys(keys_), index(0) + { + } + + String next() + { + if (index == keys.size()) + return ""; + return keys[index++]; + } + +private: + const std::vector & keys; + size_t index; +}; + +StorageS3Source::KeysIterator::KeysIterator(const std::vector & keys_) : pimpl(std::make_shared(keys_)) +{ +} + +String StorageS3Source::KeysIterator::next() +{ + return pimpl->next(); +} Block StorageS3Source::getHeader(Block sample_block, bool with_path_column, bool with_file_column) { @@ -296,6 +323,39 @@ Chunk StorageS3Source::generate() return generate(); } +static bool checkIfObjectExists(const std::shared_ptr & client, const String & bucket, const String & key) +{ + bool is_finished = false; + Aws::S3::Model::ListObjectsV2Request request; + Aws::S3::Model::ListObjectsV2Outcome outcome; + + request.SetBucket(bucket); + request.SetPrefix(key); + while (!is_finished) + { + outcome = client->ListObjectsV2(request); + if (!outcome.IsSuccess()) + throw Exception( + ErrorCodes::S3_ERROR, + "Could not list objects in bucket {} with key {}, S3 exception: {}, message: {}", + quoteString(bucket), + quoteString(key), + backQuote(outcome.GetError().GetExceptionName()), + quoteString(outcome.GetError().GetMessage())); + + const auto & result_batch = outcome.GetResult().GetContents(); + for (const auto & obj : result_batch) + { + if (obj.GetKey() == key) + return true; + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + is_finished = !outcome.GetResult().GetIsTruncated(); + } + + return false; +} class StorageS3Sink : public SinkToStorage { @@ -315,9 +375,6 @@ public: , sample_block(sample_block_) , format_settings(format_settings_) { - if (key.find_first_of("*?{") != std::string::npos) - throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "S3 key '{}' contains globs, so the table is in readonly mode", key); - write_buf = wrapWriteBufferWithCompressionMethod( std::make_unique(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format, *write_buf, sample_block, context, {}, format_settings); @@ -419,7 +476,6 @@ private: std::optional format_settings; ExpressionActionsPtr partition_by_expr; - String partition_by_column_name; static void validateBucket(const String & str) { @@ -468,6 +524,7 @@ StorageS3::StorageS3( ASTPtr partition_by_) : IStorage(table_id_) , client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later + , keys({uri_.key}) , format_name(format_name_) , max_single_read_retries(max_single_read_retries_) , min_upload_part_size(min_upload_part_size_) @@ -477,6 +534,7 @@ StorageS3::StorageS3( , distributed_processing(distributed_processing_) , format_settings(format_settings_) , partition_by(partition_by_) + , is_key_with_globs(uri_.key.find_first_of("*?{") != std::string::npos) { context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri); StorageInMemoryMetadata storage_metadata; @@ -484,7 +542,7 @@ StorageS3::StorageS3( updateClientAndAuthSettings(context_, client_auth); if (columns_.empty()) { - auto columns = getTableStructureFromDataImpl(format_name, client_auth, max_single_read_retries_, compression_method, distributed_processing_, format_settings, context_); + auto columns = getTableStructureFromDataImpl(format_name, client_auth, max_single_read_retries_, compression_method, distributed_processing_, is_key_with_globs, format_settings, context_); storage_metadata.setColumns(columns); } else @@ -495,7 +553,7 @@ StorageS3::StorageS3( setInMemoryMetadata(storage_metadata); } -std::shared_ptr StorageS3::createFileIterator(const ClientAuthentication & client_auth, bool distributed_processing, ContextPtr local_context) +std::shared_ptr StorageS3::createFileIterator(const ClientAuthentication & client_auth, const std::vector & keys, bool is_key_with_globs, bool distributed_processing, ContextPtr local_context) { std::shared_ptr iterator_wrapper{nullptr}; if (distributed_processing) @@ -505,13 +563,23 @@ std::shared_ptr StorageS3::createFileIterator( return callback(); }); } - - /// Iterate through disclosed globs and make a source for each file - auto glob_iterator = std::make_shared(*client_auth.client, client_auth.uri); - return std::make_shared([glob_iterator]() + else if (is_key_with_globs) { - return glob_iterator->next(); - }); + /// Iterate through disclosed globs and make a source for each file + auto glob_iterator = std::make_shared(*client_auth.client, client_auth.uri); + iterator_wrapper = std::make_shared([glob_iterator]() + { + return glob_iterator->next(); + }); + } + else + { + auto keys_iterator = std::make_shared(keys); + iterator_wrapper = std::make_shared([keys_iterator]() + { + return keys_iterator->next(); + }); + } } Pipe StorageS3::read( @@ -536,7 +604,7 @@ Pipe StorageS3::read( need_file_column = true; } - std::shared_ptr iterator_wrapper = createFileIterator(client_auth, distributed_processing, local_context); + std::shared_ptr iterator_wrapper = createFileIterator(client_auth, keys, is_key_with_globs, distributed_processing, local_context); for (size_t i = 0; i < num_streams; ++i) { @@ -566,9 +634,38 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr { updateClientAndAuthSettings(local_context, client_auth); + if (is_key_with_globs) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "S3 key '{}' contains globs, so the table is in readonly mode", client_auth.uri.key); + + bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; + + if (!truncate_in_insert && checkIfObjectExists(client_auth.client, client_auth.uri.bucket, keys.back())) + { + if (local_context->getSettingsRef().s3_create_new_file_on_insert) + { + size_t index = keys.size(); + auto pos = keys[0].find_first_of('.'); + String new_key; + do + { + new_key = keys[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : keys[0].substr(pos)); + ++index; + } + while (checkIfObjectExists(client_auth.client, client_auth.uri.bucket, new_key)); + keys.push_back(new_key); + } + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Object in bucket {} with key {} already exists. If you want to overwrite it, enable setting s3_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", + client_auth.uri.bucket, + keys.back()); + } + auto sample_block = metadata_snapshot->getSampleBlock(); - auto chosen_compression_method = chooseCompressionMethod(client_auth.uri.key, compression_method); - bool has_wildcards = client_auth.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || client_auth.uri.key.find(PARTITION_ID_WILDCARD) != String::npos; + auto chosen_compression_method = chooseCompressionMethod(keys.back(), compression_method); + bool has_wildcards = client_auth.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; auto insert_query = std::dynamic_pointer_cast(query); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; @@ -585,7 +682,7 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr chosen_compression_method, client_auth.client, client_auth.uri.bucket, - client_auth.uri.key, + keys.back(), min_upload_part_size, max_single_part_upload_size); } @@ -599,7 +696,7 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr chosen_compression_method, client_auth.client, client_auth.uri.bucket, - client_auth.uri.key, + keys.back(), min_upload_part_size, max_single_part_upload_size); } @@ -610,11 +707,17 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &, { updateClientAndAuthSettings(local_context, client_auth); - Aws::S3::Model::ObjectIdentifier obj; - obj.SetKey(client_auth.uri.key); + if (is_key_with_globs) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "S3 key '{}' contains globs, so the table is in readonly mode", client_auth.uri.key); Aws::S3::Model::Delete delkeys; - delkeys.AddObjects(std::move(obj)); + + for (const auto & key : keys) + { + Aws::S3::Model::ObjectIdentifier obj; + obj.SetKey(key); + delkeys.AddObjects(std::move(obj)); + } Aws::S3::Model::DeleteObjectsRequest request; request.SetBucket(client_auth.uri.bucket); @@ -731,7 +834,7 @@ ColumnsDescription StorageS3::getTableStructureFromData( { ClientAuthentication client_auth{uri, access_key_id, secret_access_key, max_connections, {}, {}}; updateClientAndAuthSettings(ctx, client_auth); - return getTableStructureFromDataImpl(format, client_auth, max_single_read_retries, compression_method, distributed_processing, format_settings, ctx); + return getTableStructureFromDataImpl(format, client_auth, max_single_read_retries, compression_method, distributed_processing, uri.key.find_first_of("*?{") != std::string::npos, format_settings, ctx); } ColumnsDescription StorageS3::getTableStructureFromDataImpl( @@ -740,12 +843,13 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( UInt64 max_single_read_retries, const String & compression_method, bool distributed_processing, + bool is_key_with_globs, const std::optional & format_settings, ContextPtr ctx) { auto read_buffer_creator = [&]() { - auto file_iterator = createFileIterator(client_auth, distributed_processing, ctx); + auto file_iterator = createFileIterator(client_auth, {client_auth.uri.key}, is_key_with_globs, distributed_processing, ctx); String current_key = (*file_iterator)(); if (current_key.empty()) throw Exception( diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index 0690040915d..1b766676427 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -44,6 +44,18 @@ public: std::shared_ptr pimpl; }; + class KeysIterator + { + public: + explicit KeysIterator(const std::vector & keys_); + String next(); + + private: + class Impl; + /// shared_ptr to have copy constructor + std::shared_ptr pimpl; + }; + using IteratorWrapper = std::function; static Block getHeader(Block sample_block, bool with_path_column, bool with_file_column); @@ -174,6 +186,7 @@ private: }; ClientAuthentication client_auth; + std::vector keys; String format_name; UInt64 max_single_read_retries; @@ -184,10 +197,11 @@ private: const bool distributed_processing; std::optional format_settings; ASTPtr partition_by; + bool is_key_with_globs = false; static void updateClientAndAuthSettings(ContextPtr, ClientAuthentication &); - static std::shared_ptr createFileIterator(const ClientAuthentication & client_auth, bool distributed_processing, ContextPtr local_context); + static std::shared_ptr createFileIterator(const ClientAuthentication & client_auth, const std::vector & keys, bool is_key_with_globs, bool distributed_processing, ContextPtr local_context); static ColumnsDescription getTableStructureFromDataImpl( const String & format, @@ -195,6 +209,7 @@ private: UInt64 max_single_read_retries, const String & compression_method, bool distributed_processing, + bool is_key_with_globs, const std::optional & format_settings, ContextPtr ctx); }; diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index f317fb5429a..d0305d6c932 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -362,6 +362,43 @@ def test_hdfsCluster(started_cluster): fs.delete(dir, recursive=True) +def test_overwrite(started_cluster): + hdfs_api = started_cluster.hdfs_api + + table_function = f"hdfs('hdfs://hdfs1:9000/data', 'Parquet', 'a Int32, b String')" + node1.query(f"create table test as {table_function}") + node1.query(f"insert into test select number, randomString(100) from numbers(5)") + node1.query_and_get_error(f"insert into test select number, randomString(100) FROM numbers(10)") + node1.query(f"insert into test select number, randomString(100) from numbers(10) settings hdfs_truncate_on_insert=1") + + result = node1.query(f"select count() from test") + assert(int(result) == 10) + + +def test_multiple_inserts(started_cluster): + hdfs_api = started_cluster.hdfs_api + + table_function = f"hdfs('hdfs://hdfs1:9000/data_multiple_inserts', 'Parquet', 'a Int32, b String')" + node1.query(f"create table test_multiple_inserts as {table_function}") + node1.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10)") + node1.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings hdfs_create_new_file_on_insert=1") + node1.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings hdfs_create_new_file_on_insert=1") + + result = node1.query(f"select count() from test_multiple_inserts") + assert(int(result) == 60) + + result = node1.query(f"drop table test_multiple_inserts") + + table_function = f"hdfs('hdfs://hdfs1:9000/data_multiple_inserts.gz', 'Parquet', 'a Int32, b String')" + node1.query(f"create table test_multiple_inserts as {table_function}") + node1.query(f"insert into test_multiple_inserts select number, randomString(100) FROM numbers(10)") + node1.query(f"insert into test_multiple_inserts select number, randomString(100) FROM numbers(20) settings hdfs_create_new_file_on_insert=1") + node1.query(f"insert into test_multiple_inserts select number, randomString(100) FROM numbers(30) settings hdfs_create_new_file_on_insert=1") + + result = node1.query(f"select count() from test_multiple_inserts") + assert(int(result) == 60) + + if __name__ == '__main__': cluster.start() input("Cluster created, press any key to destroy...") diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 885a37f875c..50a9bd3b49d 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -915,3 +915,45 @@ def test_empty_file(started_cluster): result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 0) + +def test_overwrite(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" + instance.query(f"create table test_overwrite as {table_function}") + instance.query(f"truncate table test_overwrite") + instance.query(f"insert into test_overwrite select number, randomString(100) from numbers(50) settings s3_truncate_on_insert=1") + instance.query_and_get_error(f"insert into test_overwrite select number, randomString(100) from numbers(100)") + instance.query(f"insert into test_overwrite select number, randomString(100) from numbers(200) settings s3_truncate_on_insert=1") + + result = instance.query(f"select count() from test_overwrite") + assert(int(result) == 200) + + +def test_create_new_files_on_insert(started_cluster): + bucket = started_cluster.minio_bucket + instance = started_cluster.instances["dummy"] + + table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" + instance.query(f"create table test_multiple_inserts as {table_function}") + instance.query(f"truncate table test_multiple_inserts") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10)") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings s3_create_new_file_on_insert=1") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings s3_create_new_file_on_insert=1") + + result = instance.query(f"select count() from test_multiple_inserts") + assert(int(result) == 60) + + instance.query(f"drop table test_multiple_inserts") + + table_function = f"s3(s3_parquet_gz, structure='a Int32, b String', format='Parquet')" + instance.query(f"create table test_multiple_inserts as {table_function}") + instance.query(f"truncate table test_multiple_inserts") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10)") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings s3_create_new_file_on_insert=1") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings s3_create_new_file_on_insert=1") + + result = instance.query(f"select count() from test_multiple_inserts") + assert(int(result) == 60) + diff --git a/tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.reference b/tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.reference new file mode 100644 index 00000000000..beeb89a5947 --- /dev/null +++ b/tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.reference @@ -0,0 +1,100 @@ +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 +10 +11 +12 +13 +14 +15 +16 +17 +18 +19 diff --git a/tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.sql b/tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.sql new file mode 100644 index 00000000000..7947536bc79 --- /dev/null +++ b/tests/queries/0_stateless/02155_multiple_inserts_for_formats_with_suffix.sql @@ -0,0 +1,39 @@ +-- Tags: no-fasttest, no-parallel + +drop table if exists test; +create table test (number UInt64) engine=File('Parquet'); +insert into test select * from numbers(10); +insert into test select * from numbers(10, 10); -- { serverError CANNOT_APPEND_TO_FILE } +insert into test select * from numbers(10, 10) settings engine_file_allow_create_multiple_files=1; +select * from test order by number; +truncate table test; +drop table test; + +create table test (number UInt64) engine=File('Parquet', 'test_02155/test1/data.Parquet'); +insert into test select * from numbers(10) settings engine_file_truncate_on_insert=1; +insert into test select * from numbers(10, 10); -- { serverError CANNOT_APPEND_TO_FILE } +insert into test select * from numbers(10, 10) settings engine_file_allow_create_multiple_files=1; +select * from test order by number; +drop table test; + + +insert into table function file(concat(currentDatabase(), '/test2/data.Parquet'), 'Parquet', 'number UInt64') select * from numbers(10) settings engine_file_truncate_on_insert=1; +insert into table function file(concat(currentDatabase(), '/test2/data.Parquet'), 'Parquet', 'number UInt64') select * from numbers(10, 10); -- { serverError CANNOT_APPEND_TO_FILE } +insert into table function file(concat(currentDatabase(), '/test2/data.Parquet'), 'Parquet', 'number UInt64') select * from numbers(10, 10) settings engine_file_allow_create_multiple_files=1; +select * from file(concat(currentDatabase(), '/test2/data.Parquet'), 'Parquet', 'number UInt64'); +select * from file(concat(currentDatabase(), '/test2/data.1.Parquet'), 'Parquet', 'number UInt64'); + +create table test (number UInt64) engine=File('Parquet', 'test_02155/test3/data.Parquet.gz'); +insert into test select * from numbers(10) settings engine_file_truncate_on_insert=1; +; +insert into test select * from numbers(10, 10); -- { serverError CANNOT_APPEND_TO_FILE } +insert into test select * from numbers(10, 10) settings engine_file_allow_create_multiple_files=1; +select * from test order by number; +drop table test; + +insert into table function file(concat(currentDatabase(), '/test4/data.Parquet.gz'), 'Parquet', 'number UInt64') select * from numbers(10) settings engine_file_truncate_on_insert=1; +insert into table function file(concat(currentDatabase(), '/test4/data.Parquet.gz'), 'Parquet', 'number UInt64') select * from numbers(10, 10); -- { serverError CANNOT_APPEND_TO_FILE } +insert into table function file(concat(currentDatabase(), '/test4/data.Parquet.gz'), 'Parquet', 'number UInt64') select * from numbers(10, 10) settings engine_file_allow_create_multiple_files=1; +select * from file(concat(currentDatabase(), '/test4/data.Parquet.gz'), 'Parquet', 'number UInt64'); +select * from file(concat(currentDatabase(), '/test4/data.1.Parquet.gz'), 'Parquet', 'number UInt64'); + From da7209df294679e21c2c38215dc7057233555c48 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 29 Dec 2021 21:25:06 +0300 Subject: [PATCH 02/37] Add description to settings --- src/Core/Settings.h | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index ad43c2a04d7..18746638621 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -75,11 +75,11 @@ class IColumn; M(UInt64, s3_max_single_read_retries, 4, "The maximum number of retries during single S3 read.", 0) \ M(UInt64, s3_max_redirects, 10, "Max number of S3 redirects hops allowed.", 0) \ M(UInt64, s3_max_connections, 1024, "The maximum number of connections per server.", 0) \ - M(Bool, s3_truncate_on_insert, false, "", 0) \ - M(Bool, s3_create_new_file_on_insert, false, "", 0) \ + M(Bool, s3_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables.", 0) \ + M(Bool, s3_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in s3 engine tables", 0) \ M(UInt64, hdfs_replication, 0, "The actual number of replications can be specified when the hdfs file is created.", 0) \ - M(Bool, hdfs_truncate_on_insert, false, "", 0) \ - M(Bool, hdfs_create_new_file_on_insert, false, "", 0) \ + M(Bool, hdfs_truncate_on_insert, false, "Enables or disables truncate before insert in s3 engine tables", 0) \ + M(Bool, hdfs_create_new_file_on_insert, false, "Enables or disables creating a new file on each insert in hdfs engine tables", 0) \ M(UInt64, hsts_max_age, 0, "Expired time for hsts. 0 means disable HSTS.", 0) \ M(Bool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \ M(Bool, use_uncompressed_cache, false, "Whether to use the cache of uncompressed blocks.", 0) \ @@ -494,7 +494,7 @@ class IColumn; \ M(Bool, engine_file_empty_if_not_exists, false, "Allows to select data from a file engine table without file", 0) \ M(Bool, engine_file_truncate_on_insert, false, "Enables or disables truncate before insert in file engine tables", 0) \ - M(Bool, engine_file_allow_create_multiple_files, false, ".", 0) \ + M(Bool, engine_file_allow_create_multiple_files, false, "Enables or disables creating a new file on each insert in file engine tables if format has suffix.", 0) \ M(Bool, allow_experimental_database_replicated, false, "Allow to create databases with Replicated engine", 0) \ M(UInt64, database_replicated_initial_query_timeout_sec, 300, "How long initial DDL query should wait for Replicated database to precess previous DDL queue entries", 0) \ M(UInt64, max_distributed_depth, 5, "Maximum distributed query depth", 0) \ From 131d49e6be334050745020675f57ec74798937ca Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 29 Dec 2021 21:26:23 +0300 Subject: [PATCH 03/37] Remove comments --- src/Storages/HDFS/WriteBufferFromHDFS.cpp | 9 --------- 1 file changed, 9 deletions(-) diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index a5050953b7c..2950046b633 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -38,15 +38,6 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); const String path = hdfs_uri.substr(begin_of_path); -// if (path.find_first_of("*?{") != std::string::npos) -// throw Exception(ErrorCodes::CANNOT_OPEN_FILE, "URI '{}' contains globs, so the table is in readonly mode", hdfs_uri); -// -// if (!hdfsExists(fs.get(), path.c_str()) && !truncate_) -// throw Exception( -// ErrorCodes::BAD_ARGUMENTS, -// "File {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert", -// path); - fout = hdfsOpenFile(fs.get(), path.c_str(), flags, 0, replication_, 0); /// O_WRONLY meaning create or overwrite i.e., implies O_TRUNCAT here if (fout == nullptr) From ac1665cdcb67afc01b45906f4396bd1f10337e3e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Fri, 14 Jan 2022 15:46:03 +0100 Subject: [PATCH 04/37] AsynchronousMetrics: Ignore inaccessible sensors --- src/Interpreters/AsynchronousMetrics.cpp | 34 ++++++++++++++++++++++++ 1 file changed, 34 insertions(+) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index d1c5fbebbc7..72e49505b54 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -109,6 +109,23 @@ void AsynchronousMetrics::openSensors() else break; } + + file->rewind(); + Int64 temperature = 0; + try + { + readText(temperature, *file); + } + catch (const ErrnoException & e) + { + LOG_WARNING( + &Poco::Logger::get("AsynchronousMetrics"), + "Thermal monitor '{}' exists but could not be read, error {}.", + thermal_device_index, + e.getErrno()); + continue; + } + thermal.emplace_back(std::move(file)); } } @@ -222,6 +239,23 @@ void AsynchronousMetrics::openSensorsChips() std::replace(sensor_name.begin(), sensor_name.end(), ' ', '_'); } + file->rewind(); + Int64 temperature = 0; + try + { + readText(temperature, *file); + } + catch (const ErrnoException & e) + { + LOG_WARNING( + &Poco::Logger::get("AsynchronousMetrics"), + "Hardware monitor '{}', sensor '{}' exists but could not be read, error {}.", + hwmon_name, + sensor_name, + e.getErrno()); + continue; + } + hwmon_devices[hwmon_name][sensor_name] = std::move(file); } } From 89a181bd195ac9a761bd79375c75f4efee6e442c Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Jan 2022 18:16:18 +0300 Subject: [PATCH 05/37] Make better --- src/Formats/FormatFactory.cpp | 17 +++++++++-------- src/Formats/FormatFactory.h | 18 +++++++++--------- .../Formats/Impl/ArrowBlockOutputFormat.cpp | 4 ++-- .../Impl/CustomSeparatedRowOutputFormat.cpp | 2 +- .../Formats/Impl/JSONRowOutputFormat.cpp | 4 ++-- .../Formats/Impl/ORCBlockOutputFormat.cpp | 2 +- .../Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- .../Formats/Impl/TemplateBlockOutputFormat.cpp | 2 +- .../Formats/Impl/XMLRowOutputFormat.cpp | 2 +- src/Storages/HDFS/StorageHDFS.cpp | 1 + src/Storages/HDFS/WriteBufferFromHDFS.cpp | 1 - src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageS3.cpp | 7 +++---- 13 files changed, 32 insertions(+), 32 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index ad1db85386c..9548bb754fa 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -392,24 +392,25 @@ void FormatFactory::registerNonTrivialPrefixAndSuffixChecker(const String & name target = std::move(non_trivial_prefix_and_suffix_checker); } -void FormatFactory::registerSuffixChecker(const String & name, SuffixChecker suffix_checker) +void FormatFactory::registerAppendSupportChecker(const String & name, AppendSupportChecker append_support_checker) { - auto & target = dict[name].suffix_checker; + auto & target = dict[name].append_support_checker; if (target) throw Exception("FormatFactory: Suffix checker " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); - target = std::move(suffix_checker); + target = std::move(append_support_checker); } -void FormatFactory::markFormatWithSuffix(const String & name) +void FormatFactory::markFormatDoesntSupportAppend(const String & name) { - registerSuffixChecker(name, [](const FormatSettings &){ return true; }); + registerAppendSupportChecker(name, [](const FormatSettings &){ return true; }); } -bool FormatFactory::checkIfFormatHasSuffix(const String & name, ContextPtr context, const std::optional & format_settings_) +bool FormatFactory::checkIfFormatSupportAppend(const String & name, ContextPtr context, const std::optional & format_settings_) { auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context); - auto & suffix_checker = dict[name].suffix_checker; - return suffix_checker && suffix_checker(format_settings); + auto & append_support_checker = dict[name].append_support_checker; + /// By default we consider that format supports append + return !append_support_checker || append_support_checker(format_settings); } void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator) diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 5acf79f2047..8a671d5a365 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -93,9 +93,9 @@ private: /// The checker should return true if parallel parsing should be disabled. using NonTrivialPrefixAndSuffixChecker = std::function; - /// Some formats can have suffix after data depending on settings. - /// The checker should return true if format will write some suffix after data. - using SuffixChecker = std::function; + /// Some formats can support append depending on settings. + /// The checker should return true if format support append. + using AppendSupportChecker = std::function; using SchemaReaderCreator = std::function; using ExternalSchemaReaderCreator = std::function; @@ -110,7 +110,7 @@ private: bool supports_parallel_formatting{false}; bool is_column_oriented{false}; NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker; - SuffixChecker suffix_checker; + AppendSupportChecker append_support_checker; }; using FormatsDictionary = std::unordered_map; @@ -172,13 +172,13 @@ public: void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker); - void registerSuffixChecker(const String & name, SuffixChecker suffix_checker); + void registerAppendSupportChecker(const String & name, AppendSupportChecker append_support_checker); - /// If format always contains suffix, you an use this method instead of - /// registerSuffixChecker with suffix_checker that always returns true. - void markFormatWithSuffix(const String & name); + /// If format always doesn't support append, you can use this method instead of + /// registerAppendSupportChecker with append_support_checker that always returns true. + void markFormatDoesntSupportAppend(const String & name); - bool checkIfFormatHasSuffix(const String & name, ContextPtr context, const std::optional & format_settings_ = std::nullopt); + bool checkIfFormatSupportAppend(const String & name, ContextPtr context, const std::optional & format_settings_ = std::nullopt); /// Register format by its name. void registerInputFormat(const String & name, InputCreator input_creator); diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index b60d38f317c..c87191d0a12 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -93,7 +93,7 @@ void registerOutputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, false, format_settings); }); - factory.markFormatWithSuffix("Arrow"); + factory.markFormatDoesntSupportAppend("Arrow"); factory.registerOutputFormat( "ArrowStream", @@ -104,7 +104,7 @@ void registerOutputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, true, format_settings); }); - factory.markFormatWithSuffix("ArrowStream"); + factory.markFormatDoesntSupportAppend("ArrowStream"); } } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index 3fb7c0a36a3..88b9226fd8a 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -92,7 +92,7 @@ void registerOutputFormatCustomSeparated(FormatFactory & factory) factory.markOutputFormatSupportsParallelFormatting(format_name); - factory.registerSuffixChecker(format_name, [](const FormatSettings & settings) + factory.registerAppendSupportChecker(format_name, [](const FormatSettings & settings) { return !settings.custom.result_after_delimiter.empty(); }); diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 577efdb1a21..7ac7d45d26d 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -284,7 +284,7 @@ void registerOutputFormatJSON(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSON"); - factory.markFormatWithSuffix("JSON"); + factory.markFormatDoesntSupportAppend("JSON"); factory.registerOutputFormat("JSONStrings", []( WriteBuffer & buf, @@ -296,7 +296,7 @@ void registerOutputFormatJSON(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONStrings"); - factory.markFormatWithSuffix("JSONStrings"); + factory.markFormatDoesntSupportAppend("JSONStrings"); } } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 33d4b3e568d..67287c8a661 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -526,7 +526,7 @@ void registerOutputFormatORC(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings); }); - factory.markFormatWithSuffix("ORC"); + factory.markFormatDoesntSupportAppend("ORC"); } } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 96c3a80b732..c3ddc632641 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -85,7 +85,7 @@ void registerOutputFormatParquet(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings); }); - factory.markFormatWithSuffix("Parquet"); + factory.markFormatDoesntSupportAppend("Parquet"); } } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index eded88298e8..6babac98102 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -236,7 +236,7 @@ void registerOutputFormatTemplate(FormatFactory & factory) return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); - factory.registerSuffixChecker("Template", [](const FormatSettings & settings) + factory.registerAppendSupportChecker("Template", [](const FormatSettings & settings) { if (settings.template_settings.resultset_format.empty()) return false; diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 56bbbc0673c..6a80131a65e 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -256,7 +256,7 @@ void registerOutputFormatXML(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("XML"); - factory.markFormatWithSuffix("XML"); + factory.markFormatDoesntSupportAppend("XML"); } } diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 36287b92855..c088ad07a34 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -52,6 +52,7 @@ namespace ErrorCodes extern const int ACCESS_DENIED; extern const int DATABASE_ACCESS_DENIED; extern const int CANNOT_EXTRACT_TABLE_STRUCTURE; + extern const int BAD_ARGUMENTS; } namespace { diff --git a/src/Storages/HDFS/WriteBufferFromHDFS.cpp b/src/Storages/HDFS/WriteBufferFromHDFS.cpp index 2950046b633..2addfc0069f 100644 --- a/src/Storages/HDFS/WriteBufferFromHDFS.cpp +++ b/src/Storages/HDFS/WriteBufferFromHDFS.cpp @@ -15,7 +15,6 @@ namespace ErrorCodes extern const int NETWORK_ERROR; extern const int CANNOT_OPEN_FILE; extern const int CANNOT_FSYNC; -extern const int BAD_ARGUMENTS; } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 586b8d97875..48e0e82821a 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -833,7 +833,7 @@ SinkToStoragePtr StorageFile::write( throw Exception("Table '" + getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); if (!context->getSettingsRef().engine_file_truncate_on_insert && !is_path_with_globs - && FormatFactory::instance().checkIfFormatHasSuffix(format_name, context, format_settings) && fs::exists(paths.back()) + && !FormatFactory::instance().checkIfFormatSupportAppend(format_name, context, format_settings) && fs::exists(paths.back()) && fs::file_size(paths.back()) != 0) { if (context->getSettingsRef().engine_file_allow_create_multiple_files) diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index ea563937ca6..e15878cf893 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -555,7 +555,6 @@ StorageS3::StorageS3( std::shared_ptr StorageS3::createFileIterator(const ClientAuthentication & client_auth, const std::vector & keys, bool is_key_with_globs, bool distributed_processing, ContextPtr local_context) { - std::shared_ptr iterator_wrapper{nullptr}; if (distributed_processing) { return std::make_shared( @@ -567,7 +566,7 @@ std::shared_ptr StorageS3::createFileIterator( { /// Iterate through disclosed globs and make a source for each file auto glob_iterator = std::make_shared(*client_auth.client, client_auth.uri); - iterator_wrapper = std::make_shared([glob_iterator]() + return std::make_shared([glob_iterator]() { return glob_iterator->next(); }); @@ -575,7 +574,7 @@ std::shared_ptr StorageS3::createFileIterator( else { auto keys_iterator = std::make_shared(keys); - iterator_wrapper = std::make_shared([keys_iterator]() + return std::make_shared([keys_iterator]() { return keys_iterator->next(); }); @@ -662,7 +661,7 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr client_auth.uri.bucket, keys.back()); } - + auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(keys.back(), compression_method); bool has_wildcards = client_auth.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; From 253035a5df2b900f77dfbf0a1cf774123bf36efb Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Jan 2022 19:17:06 +0300 Subject: [PATCH 06/37] Fix --- src/Formats/FormatFactory.cpp | 2 +- src/Processors/Formats/Impl/AvroRowOutputFormat.cpp | 1 + .../Formats/Impl/CustomSeparatedRowOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp | 4 ++-- src/Storages/StorageFile.cpp | 4 ++-- tests/queries/0_stateless/02168_avro_bug.sql | 6 +++--- 6 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index 9548bb754fa..53f13bf97a7 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -402,7 +402,7 @@ void FormatFactory::registerAppendSupportChecker(const String & name, AppendSupp void FormatFactory::markFormatDoesntSupportAppend(const String & name) { - registerAppendSupportChecker(name, [](const FormatSettings &){ return true; }); + registerAppendSupportChecker(name, [](const FormatSettings &){ return false; }); } bool FormatFactory::checkIfFormatSupportAppend(const String & name, ContextPtr context, const std::optional & format_settings_) diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index fb3389475ac..ae5ce6099a9 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -479,6 +479,7 @@ void registerOutputFormatAvro(FormatFactory & factory) { return std::make_shared(buf, sample, params, settings); }); + factory.markFormatDoesntSupportAppend("Avro"); } } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index 88b9226fd8a..4c8cf19b923 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -94,7 +94,7 @@ void registerOutputFormatCustomSeparated(FormatFactory & factory) factory.registerAppendSupportChecker(format_name, [](const FormatSettings & settings) { - return !settings.custom.result_after_delimiter.empty(); + return settings.custom.result_after_delimiter.empty(); }); }; diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index 6babac98102..5c5b99f61da 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -239,7 +239,7 @@ void registerOutputFormatTemplate(FormatFactory & factory) factory.registerAppendSupportChecker("Template", [](const FormatSettings & settings) { if (settings.template_settings.resultset_format.empty()) - return false; + return true; auto resultset_format = ParsedTemplateFormatString( FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false, settings.schema.is_server, settings.schema.format_schema_path), @@ -247,7 +247,7 @@ void registerOutputFormatTemplate(FormatFactory & factory) { return static_cast(TemplateBlockOutputFormat::stringToResultsetPart(partName)); }); - return !resultset_format.delimiters.empty() && !resultset_format.delimiters.back().empty(); + return resultset_format.delimiters.empty() || resultset_format.delimiters.back().empty(); }); } } diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 48e0e82821a..dae162c9e1f 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -852,8 +852,8 @@ SinkToStoragePtr StorageFile::write( else throw Exception( ErrorCodes::CANNOT_APPEND_TO_FILE, - "Cannot append data in format {} to file, because this format contains suffix and " - "data can be written to a file only once. You can allow to create a new file " + "Cannot append data in format {} to file, because this format doesn't support appends." + " You can allow to create a new file " "on each insert by enabling setting engine_file_allow_create_multiple_files", format_name); } diff --git a/tests/queries/0_stateless/02168_avro_bug.sql b/tests/queries/0_stateless/02168_avro_bug.sql index 78eedf3258e..e50c78fb0b8 100644 --- a/tests/queries/0_stateless/02168_avro_bug.sql +++ b/tests/queries/0_stateless/02168_avro_bug.sql @@ -1,5 +1,5 @@ -- Tags: no-fasttest -insert into table function file('data.avro', 'Avro', 'x UInt64') select * from numbers(10); -insert into table function file('data.avro', 'Avro', 'x UInt64') select * from numbers(10); -insert into table function file('data.avro', 'Avro', 'x UInt64') select * from numbers(10); +insert into table function file('data.avro', 'Parquet', 'x UInt64') select * from numbers(10); +insert into table function file('data.avro', 'Parquet', 'x UInt64') select * from numbers(10); -- { serverError CANNOT_APPEND_TO_FILE } +insert into table function file('data.avro', 'Parquet', 'x UInt64') select * from numbers(10); -- { serverError CANNOT_APPEND_TO_FILE } select 'OK'; From af54ca4866c642fb63deb73fb89c62fd283aa52a Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 14 Jan 2022 21:18:16 +0300 Subject: [PATCH 07/37] Fix partition write --- src/Storages/HDFS/StorageHDFS.cpp | 66 +++++++++++++++---------------- src/Storages/StorageFile.cpp | 29 +++++++------- src/Storages/StorageS3.cpp | 58 +++++++++++++-------------- 3 files changed, 76 insertions(+), 77 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index 6e975190ad1..a5d01bb1428 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -536,41 +536,8 @@ Pipe StorageHDFS::read( SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context_) { - if (is_path_with_globs) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); - String current_uri = uris.back(); - const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_uri); - - HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); - HDFSFSPtr fs = createHDFSFS(builder.get()); - - bool truncate_on_insert = context_->getSettingsRef().hdfs_truncate_on_insert; - if (!truncate_on_insert && !hdfsExists(fs.get(), path_from_uri.c_str())) - { - if (context_->getSettingsRef().hdfs_create_new_file_on_insert) - { - auto pos = uris[0].find_first_of('.', uris[0].find_last_of('/')); - size_t index = uris.size(); - String new_uri; - do - { - new_uri = uris[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : uris[0].substr(pos)); - ++index; - } - while (!hdfsExists(fs.get(), new_uri.c_str())); - uris.push_back(new_uri); - current_uri = new_uri; - } - else - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "File with path {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert, " - "if you want to create new file on each insert, enable setting hdfs_create_new_file_on_insert", - path_from_uri); - } - bool has_wildcards = current_uri.find(PartitionedSink::PARTITION_ID_WILDCARD) != String::npos; const auto * insert_query = dynamic_cast(query.get()); auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; @@ -588,6 +555,39 @@ SinkToStoragePtr StorageHDFS::write(const ASTPtr & query, const StorageMetadataP } else { + if (is_path_with_globs) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "URI '{}' contains globs, so the table is in readonly mode", uris.back()); + + const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(current_uri); + + HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context_->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + + bool truncate_on_insert = context_->getSettingsRef().hdfs_truncate_on_insert; + if (!truncate_on_insert && !hdfsExists(fs.get(), path_from_uri.c_str())) + { + if (context_->getSettingsRef().hdfs_create_new_file_on_insert) + { + auto pos = uris[0].find_first_of('.', uris[0].find_last_of('/')); + size_t index = uris.size(); + String new_uri; + do + { + new_uri = uris[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : uris[0].substr(pos)); + ++index; + } + while (!hdfsExists(fs.get(), new_uri.c_str())); + uris.push_back(new_uri); + current_uri = new_uri; + } + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "File with path {} already exists. If you want to overwrite it, enable setting hdfs_truncate_on_insert, " + "if you want to create new file on each insert, enable setting hdfs_create_new_file_on_insert", + path_from_uri); + } + return std::make_shared(current_uri, format_name, metadata_snapshot->getSampleBlock(), diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index dae162c9e1f..e6c5f25dd57 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -605,7 +605,7 @@ public: int table_fd_, bool use_table_fd_, std::string base_path_, - std::vector paths_, + std::string path_, const CompressionMethod compression_method_, const std::optional & format_settings_, const String format_name_, @@ -617,7 +617,7 @@ public: , table_fd(table_fd_) , use_table_fd(use_table_fd_) , base_path(base_path_) - , paths(paths_) + , path(path_) , compression_method(compression_method_) , format_name(format_name_) , format_settings(format_settings_) @@ -634,7 +634,7 @@ public: int table_fd_, bool use_table_fd_, std::string base_path_, - std::vector paths_, + const std::string & path_, const CompressionMethod compression_method_, const std::optional & format_settings_, const String format_name_, @@ -646,7 +646,7 @@ public: , table_fd(table_fd_) , use_table_fd(use_table_fd_) , base_path(base_path_) - , paths(paths_) + , path(path_) , compression_method(compression_method_) , format_name(format_name_) , format_settings(format_settings_) @@ -668,9 +668,8 @@ public: } else { - assert(!paths.empty()); flags |= O_WRONLY | O_APPEND | O_CREAT; - naked_buffer = std::make_unique(paths.back(), DBMS_DEFAULT_BUFFER_SIZE, flags); + naked_buffer = std::make_unique(path, DBMS_DEFAULT_BUFFER_SIZE, flags); } /// In case of formats with prefixes if file is not empty we have already written prefix. @@ -710,7 +709,7 @@ private: int table_fd; bool use_table_fd; std::string base_path; - std::vector paths; + std::string path; CompressionMethod compression_method; std::string format_name; std::optional format_settings; @@ -753,7 +752,6 @@ public: { auto partition_path = PartitionedSink::replaceWildcards(path, partition_id); PartitionedSink::validatePartitionKey(partition_path, true); - Strings result_paths = {partition_path}; checkCreationIsAllowed(context, context->getUserFilesPath(), partition_path); return std::make_shared( metadata_snapshot, @@ -761,7 +759,7 @@ public: -1, /* use_table_fd */false, base_path, - result_paths, + partition_path, compression_method, format_settings, format_name, @@ -795,7 +793,6 @@ SinkToStoragePtr StorageFile::write( int flags = 0; - std::string path; if (context->getSettingsRef().engine_file_truncate_on_insert) flags |= O_TRUNC; @@ -816,7 +813,7 @@ SinkToStoragePtr StorageFile::write( std::unique_lock{rwlock, getLockTimeout(context)}, base_path, path_for_partitioned_write, - chooseCompressionMethod(path, compression_method), + chooseCompressionMethod(path_for_partitioned_write, compression_method), format_settings, format_name, context, @@ -824,14 +821,15 @@ SinkToStoragePtr StorageFile::write( } else { + String path; if (!paths.empty()) { - path = paths[0]; - fs::create_directories(fs::path(path).parent_path()); - if (is_path_with_globs) throw Exception("Table '" + getStorageID().getNameForLogs() + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); + path = paths.back(); + fs::create_directories(fs::path(path).parent_path()); + if (!context->getSettingsRef().engine_file_truncate_on_insert && !is_path_with_globs && !FormatFactory::instance().checkIfFormatSupportAppend(format_name, context, format_settings) && fs::exists(paths.back()) && fs::file_size(paths.back()) != 0) @@ -848,6 +846,7 @@ SinkToStoragePtr StorageFile::write( } while (fs::exists(new_path)); paths.push_back(new_path); + path = new_path; } else throw Exception( @@ -866,7 +865,7 @@ SinkToStoragePtr StorageFile::write( table_fd, use_table_fd, base_path, - paths, + path, chooseCompressionMethod(path, compression_method), format_settings, format_name, diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 44fe7c3d4bf..2fa5ee0fa64 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -633,35 +633,6 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr { updateClientAndAuthSettings(local_context, client_auth); - if (is_key_with_globs) - throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "S3 key '{}' contains globs, so the table is in readonly mode", client_auth.uri.key); - - bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; - - if (!truncate_in_insert && checkIfObjectExists(client_auth.client, client_auth.uri.bucket, keys.back())) - { - if (local_context->getSettingsRef().s3_create_new_file_on_insert) - { - size_t index = keys.size(); - auto pos = keys[0].find_first_of('.'); - String new_key; - do - { - new_key = keys[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : keys[0].substr(pos)); - ++index; - } - while (checkIfObjectExists(client_auth.client, client_auth.uri.bucket, new_key)); - keys.push_back(new_key); - } - else - throw Exception( - ErrorCodes::BAD_ARGUMENTS, - "Object in bucket {} with key {} already exists. If you want to overwrite it, enable setting s3_truncate_on_insert, if you " - "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", - client_auth.uri.bucket, - keys.back()); - } - auto sample_block = metadata_snapshot->getSampleBlock(); auto chosen_compression_method = chooseCompressionMethod(keys.back(), compression_method); bool has_wildcards = client_auth.uri.bucket.find(PARTITION_ID_WILDCARD) != String::npos || keys.back().find(PARTITION_ID_WILDCARD) != String::npos; @@ -687,6 +658,35 @@ SinkToStoragePtr StorageS3::write(const ASTPtr & query, const StorageMetadataPtr } else { + if (is_key_with_globs) + throw Exception(ErrorCodes::DATABASE_ACCESS_DENIED, "S3 key '{}' contains globs, so the table is in readonly mode", client_auth.uri.key); + + bool truncate_in_insert = local_context->getSettingsRef().s3_truncate_on_insert; + + if (!truncate_in_insert && checkIfObjectExists(client_auth.client, client_auth.uri.bucket, keys.back())) + { + if (local_context->getSettingsRef().s3_create_new_file_on_insert) + { + size_t index = keys.size(); + auto pos = keys[0].find_first_of('.'); + String new_key; + do + { + new_key = keys[0].substr(0, pos) + "." + std::to_string(index) + (pos == std::string::npos ? "" : keys[0].substr(pos)); + ++index; + } + while (checkIfObjectExists(client_auth.client, client_auth.uri.bucket, new_key)); + keys.push_back(new_key); + } + else + throw Exception( + ErrorCodes::BAD_ARGUMENTS, + "Object in bucket {} with key {} already exists. If you want to overwrite it, enable setting s3_truncate_on_insert, if you " + "want to create a new file on each insert, enable setting s3_create_new_file_on_insert", + client_auth.uri.bucket, + keys.back()); + } + return std::make_shared( format_name, sample_block, From 4efadfad3cf0aa7d0263108fe6ff153af874bb05 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Jan 2022 22:26:13 +0300 Subject: [PATCH 08/37] Fix tests --- src/Parsers/ParserInsertQuery.cpp | 4 +++ src/Storages/HDFS/StorageHDFS.cpp | 30 ++++++++++++++----- src/Storages/HDFS/StorageHDFS.h | 2 +- src/Storages/StorageS3.cpp | 19 +++++++----- tests/integration/test_storage_hdfs/test.py | 10 +++---- .../configs/named_collections.xml | 5 ++++ tests/integration/test_storage_s3/test.py | 22 +++++++------- 7 files changed, 59 insertions(+), 33 deletions(-) diff --git a/src/Parsers/ParserInsertQuery.cpp b/src/Parsers/ParserInsertQuery.cpp index 568f486a5cf..631b4f7ed53 100644 --- a/src/Parsers/ParserInsertQuery.cpp +++ b/src/Parsers/ParserInsertQuery.cpp @@ -132,6 +132,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) Pos before_values = pos; String format_str; + bool values = false; /// VALUES or FORMAT or SELECT or WITH or WATCH. /// After FROM INFILE we expect FORMAT, SELECT, WITH or nothing. @@ -140,6 +141,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// If VALUES is defined in query, everything except setting will be parsed as data data = pos->begin; format_str = "Values"; + values = true; } else if (s_format.ignore(pos, expected)) { @@ -184,6 +186,8 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserSetQuery parser_settings(true); if (!parser_settings.parse(pos, settings_ast, expected)) return false; + if (values) + data = pos->begin; } if (select) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index a5d01bb1428..72cb4941b8c 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -214,20 +214,34 @@ private: class HDFSSource::URISIterator::Impl { public: - Impl(const std::vector & uris_) : uris(uris_), index(0) + explicit Impl(const std::vector & uris_, ContextPtr context) { + auto path_and_uri = getPathFromUriAndUriWithoutPath(uris_[0]); + HDFSBuilderWrapper builder = createHDFSBuilder(path_and_uri.second + "/", context->getGlobalContext()->getConfigRef()); + HDFSFSPtr fs = createHDFSFS(builder.get()); + for (const auto & uri : uris_) + { + path_and_uri = getPathFromUriAndUriWithoutPath(uri); + if (!hdfsExists(fs.get(), path_and_uri.first.c_str())) + uris.push_back(uri); + } + uris_iter = uris.begin(); } String next() { - if (index == uris.size()) + std::lock_guard lock(mutex); + if (uris_iter == uris.end()) return ""; - return uris[index++]; + auto key = *uris_iter; + ++uris_iter; + return key; } private: - const std::vector & uris; - size_t index; + std::mutex mutex; + Strings uris; + Strings::iterator uris_iter; }; Block HDFSSource::getHeader(const StorageMetadataPtr & metadata_snapshot, bool need_path_column, bool need_file_column) @@ -263,8 +277,8 @@ String HDFSSource::DisclosedGlobIterator::next() return pimpl->next(); } -HDFSSource::URISIterator::URISIterator(const std::vector & uris_) - : pimpl(std::make_shared(uris_)) +HDFSSource::URISIterator::URISIterator(const std::vector & uris_, ContextPtr context) + : pimpl(std::make_shared(uris_, context)) { } @@ -501,7 +515,7 @@ Pipe StorageHDFS::read( } else { - auto uris_iterator = std::make_shared(uris); + auto uris_iterator = std::make_shared(uris, context_); iterator_wrapper = std::make_shared([uris_iterator]() { return uris_iterator->next(); diff --git a/src/Storages/HDFS/StorageHDFS.h b/src/Storages/HDFS/StorageHDFS.h index 53be640248c..8e455189bc6 100644 --- a/src/Storages/HDFS/StorageHDFS.h +++ b/src/Storages/HDFS/StorageHDFS.h @@ -99,7 +99,7 @@ public: class URISIterator { public: - URISIterator(const std::vector & uris_); + URISIterator(const std::vector & uris_, ContextPtr context); String next(); private: class Impl; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 2fa5ee0fa64..e2294aaabc5 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -82,8 +82,6 @@ public: Impl(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_) : client(client_), globbed_uri(globbed_uri_) { - std::lock_guard lock(mutex); - if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos) throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION); @@ -179,20 +177,24 @@ String StorageS3Source::DisclosedGlobIterator::next() class StorageS3Source::KeysIterator::Impl { public: - explicit Impl(const std::vector & keys_) : keys(keys_), index(0) + explicit Impl(const std::vector & keys_) : keys(keys_), keys_iter(keys.begin()) { } String next() { - if (index == keys.size()) + std::lock_guard lock(mutex); + if (keys_iter == keys.end()) return ""; - return keys[index++]; + auto key = *keys_iter; + ++keys_iter; + return key; } private: - const std::vector & keys; - size_t index; + std::mutex mutex; + Strings keys; + Strings::iterator keys_iter; }; StorageS3Source::KeysIterator::KeysIterator(const std::vector & keys_) : pimpl(std::make_shared(keys_)) @@ -849,9 +851,10 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( const std::optional & format_settings, ContextPtr ctx) { + std::vector keys = {client_auth.uri.key}; auto read_buffer_creator = [&]() { - auto file_iterator = createFileIterator(client_auth, {client_auth.uri.key}, is_key_with_globs, distributed_processing, ctx); + auto file_iterator = createFileIterator(client_auth, keys, is_key_with_globs, distributed_processing, ctx); String current_key = (*file_iterator)(); if (current_key.empty()) throw Exception( diff --git a/tests/integration/test_storage_hdfs/test.py b/tests/integration/test_storage_hdfs/test.py index 1b569daef1b..7f4eccad823 100644 --- a/tests/integration/test_storage_hdfs/test.py +++ b/tests/integration/test_storage_hdfs/test.py @@ -366,12 +366,12 @@ def test_overwrite(started_cluster): hdfs_api = started_cluster.hdfs_api table_function = f"hdfs('hdfs://hdfs1:9000/data', 'Parquet', 'a Int32, b String')" - node1.query(f"create table test as {table_function}") - node1.query(f"insert into test select number, randomString(100) from numbers(5)") - node1.query_and_get_error(f"insert into test select number, randomString(100) FROM numbers(10)") - node1.query(f"insert into test select number, randomString(100) from numbers(10) settings hdfs_truncate_on_insert=1") + node1.query(f"create table test_overwrite as {table_function}") + node1.query(f"insert into test_overwrite select number, randomString(100) from numbers(5)") + node1.query_and_get_error(f"insert into test_overwrite select number, randomString(100) FROM numbers(10)") + node1.query(f"insert into test_overwrite select number, randomString(100) from numbers(10) settings hdfs_truncate_on_insert=1") - result = node1.query(f"select count() from test") + result = node1.query(f"select count() from test_overwrite") assert(int(result) == 10) diff --git a/tests/integration/test_storage_s3/configs/named_collections.xml b/tests/integration/test_storage_s3/configs/named_collections.xml index ef21ced4d0c..f22440d17c9 100644 --- a/tests/integration/test_storage_s3/configs/named_collections.xml +++ b/tests/integration/test_storage_s3/configs/named_collections.xml @@ -10,6 +10,11 @@ minio minio123 + + http://minio1:9001/root/test_parquet_gz + minio + minio123 + http://minio1:9001/root/test_orc minio diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 703329ce9cb..a804053d4fd 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -136,7 +136,7 @@ def test_put(started_cluster, maybe_auth, positive, compression): values_csv = "1,2,3\n3,2,1\n78,43,45\n" filename = "test.csv" put_query = f"""insert into table function s3('http://{started_cluster.minio_ip}:{started_cluster.minio_port}/{bucket}/{filename}', - {maybe_auth}'CSV', '{table_format}', {compression}) values {values}""" + {maybe_auth}'CSV', '{table_format}', {compression}) values settings s3_truncate_on_insert=1 {values}""" try: run_query(instance, put_query) @@ -298,7 +298,7 @@ def test_put_csv(started_cluster, maybe_auth, positive): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" - put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV".format( + put_query = "insert into table function s3('http://{}:{}/{}/{}', {}'CSV', '{}') format CSV settings s3_truncate_on_insert=1".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename, maybe_auth, table_format) csv_data = "8,9,16\n11,18,13\n22,14,2\n" @@ -322,7 +322,7 @@ def test_put_get_with_redirect(started_cluster): values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)" values_csv = "1,1,1\n1,1,1\n11,11,11\n" filename = "test.csv" - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format( started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) run_query(instance, query) @@ -350,12 +350,12 @@ def test_put_with_zero_redirect(started_cluster): filename = "test.csv" # Should work without redirect - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format( started_cluster.minio_ip, MINIO_INTERNAL_PORT, bucket, filename, table_format, values) run_query(instance, query) # Should not work with redirect - query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format( + query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values settings s3_truncate_on_insert=1 {}".format( started_cluster.minio_redirect_host, started_cluster.minio_redirect_port, bucket, filename, table_format, values) exception_raised = False try: @@ -805,13 +805,13 @@ def test_seekable_formats(started_cluster): instance = started_cluster.instances["dummy"] # type: ClickHouseInstance table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" - instance.query(f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)") + instance.query(f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) settings s3_truncate_on_insert=1") result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" - exec_query_with_retry(instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000)") + exec_query_with_retry(instance, f"insert into table function {table_function} SELECT number, randomString(100) FROM numbers(5000000) settings s3_truncate_on_insert=1") result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) @@ -827,14 +827,14 @@ def test_seekable_formats_url(started_cluster): instance = started_cluster.instances["dummy"] table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" - instance.query(f"insert into table function {table_function} select number, randomString(100) from numbers(5000000)") + instance.query(f"insert into table function {table_function} select number, randomString(100) from numbers(5000000) settings s3_truncate_on_insert=1") table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_parquet', 'Parquet', 'a Int32, b String')" result = instance.query(f"SELECT count() FROM {table_function}") assert(int(result) == 5000000) table_function = f"s3(s3_orc, structure='a Int32, b String', format='ORC')" - exec_query_with_retry(instance, f"insert into table function {table_function} select number, randomString(100) from numbers(5000000)") + exec_query_with_retry(instance, f"insert into table function {table_function} select number, randomString(100) from numbers(5000000) settings s3_truncate_on_insert=1") table_function = f"url('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_orc', 'ORC', 'a Int32, b String')" result = instance.query(f"SELECT count() FROM {table_function}") @@ -939,7 +939,7 @@ def test_create_new_files_on_insert(started_cluster): table_function = f"s3(s3_parquet, structure='a Int32, b String', format='Parquet')" instance.query(f"create table test_multiple_inserts as {table_function}") instance.query(f"truncate table test_multiple_inserts") - instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10)") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10) settings s3_truncate_on_insert=1") instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings s3_create_new_file_on_insert=1") instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings s3_create_new_file_on_insert=1") @@ -951,7 +951,7 @@ def test_create_new_files_on_insert(started_cluster): table_function = f"s3(s3_parquet_gz, structure='a Int32, b String', format='Parquet')" instance.query(f"create table test_multiple_inserts as {table_function}") instance.query(f"truncate table test_multiple_inserts") - instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10)") + instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(10) settings s3_truncate_on_insert=1") instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(20) settings s3_create_new_file_on_insert=1") instance.query(f"insert into test_multiple_inserts select number, randomString(100) from numbers(30) settings s3_create_new_file_on_insert=1") From a166963d919b83ea1a0a04b1b0c7e239a6b69271 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Jan 2022 13:14:03 +0300 Subject: [PATCH 09/37] Update 02168_avro_bug.sql --- tests/queries/0_stateless/02168_avro_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02168_avro_bug.sql b/tests/queries/0_stateless/02168_avro_bug.sql index e50c78fb0b8..ac98119845f 100644 --- a/tests/queries/0_stateless/02168_avro_bug.sql +++ b/tests/queries/0_stateless/02168_avro_bug.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-parallel insert into table function file('data.avro', 'Parquet', 'x UInt64') select * from numbers(10); insert into table function file('data.avro', 'Parquet', 'x UInt64') select * from numbers(10); -- { serverError CANNOT_APPEND_TO_FILE } insert into table function file('data.avro', 'Parquet', 'x UInt64') select * from numbers(10); -- { serverError CANNOT_APPEND_TO_FILE } From 8921725d716e05b13dc82f2db1c3d5bfc3ad6983 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 20 Jan 2022 21:55:59 +0300 Subject: [PATCH 10/37] fix logical error on RESTORE REPLICA --- src/Interpreters/InterpreterSystemQuery.cpp | 26 +-------------------- src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++++++++---- src/Storages/StorageReplicatedMergeTree.h | 2 +- 3 files changed, 16 insertions(+), 31 deletions(-) diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 123ff6ba2ca..df0c5635af9 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -56,7 +56,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int TIMEOUT_EXCEEDED; extern const int TABLE_WAS_NOT_DROPPED; - extern const int NO_ZOOKEEPER; } @@ -472,12 +471,6 @@ void InterpreterSystemQuery::restoreReplica() { getContext()->checkAccess(AccessType::SYSTEM_RESTORE_REPLICA, table_id); - const zkutil::ZooKeeperPtr & zookeeper = getContext()->getZooKeeper(); - - if (zookeeper->expired()) - throw Exception(ErrorCodes::NO_ZOOKEEPER, - "Cannot restore table metadata because ZooKeeper session has expired"); - const StoragePtr table_ptr = DatabaseCatalog::instance().getTable(table_id, getContext()); auto * const table_replicated_ptr = dynamic_cast(table_ptr.get()); @@ -485,24 +478,7 @@ void InterpreterSystemQuery::restoreReplica() if (table_replicated_ptr == nullptr) throw Exception(ErrorCodes::BAD_ARGUMENTS, table_is_not_replicated.data(), table_id.getNameForLogs()); - auto & table_replicated = *table_replicated_ptr; - - StorageReplicatedMergeTree::Status status; - table_replicated.getStatus(status); - - if (!status.is_readonly) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica must be readonly"); - - const String replica_name = table_replicated.getReplicaName(); - const String& zk_root_path = status.zookeeper_path; - - if (String replica_path = zk_root_path + "replicas/" + replica_name; zookeeper->exists(replica_path)) - throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Replica path is present at {} -- nothing to restore. " - "If you are sure that metadata it lost and replica path contain some garbage, " - "then use SYSTEM DROP REPLICA query first.", replica_path); - - table_replicated.restoreMetadataInZooKeeper(); + table_replicated_ptr->restoreMetadataInZooKeeper(); } StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, ContextMutablePtr system_context, bool need_ddl_guard) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index a23859e7b5e..5379c3f89c1 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -774,7 +774,8 @@ void StorageReplicatedMergeTree::drop() /// or metadata of staled replica were removed manually, /// in this case, has_metadata_in_zookeeper = false, and we also permit to drop the table. - if (has_metadata_in_zookeeper) + bool maybe_has_metadata_in_zookeeper = !has_metadata_in_zookeeper.has_value() || *has_metadata_in_zookeeper; + if (maybe_has_metadata_in_zookeeper) { /// Table can be shut down, restarting thread is not active /// and calling StorageReplicatedMergeTree::getZooKeeper()/getAuxiliaryZooKeeper() won't suffice. @@ -4801,12 +4802,22 @@ bool StorageReplicatedMergeTree::getFakePartCoveringAllPartsInPartition(const St void StorageReplicatedMergeTree::restoreMetadataInZooKeeper() { LOG_INFO(log, "Restoring replica metadata"); + if (!is_readonly) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica must be readonly"); - if (!is_readonly || has_metadata_in_zookeeper) - throw Exception(ErrorCodes::LOGICAL_ERROR, "It's a bug: replica is not readonly"); + if (getZooKeeper()->exists(replica_path)) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Replica path is present at {} - nothing to restore. " + "If you are sure that metadata is lost and replica path contain some garbage, " + "then use SYSTEM DROP REPLICA query first.", replica_path); + + if (has_metadata_in_zookeeper.has_value() && *has_metadata_in_zookeeper) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica has metadata in ZooKeeper: " + "it's a bug or a result of manual intervention into ZooKeeper"); if (are_restoring_replica.exchange(true)) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Replica restoration in progress"); + SCOPE_EXIT({ are_restoring_replica.store(false); }); auto metadata_snapshot = getInMemoryMetadataPtr(); @@ -4847,8 +4858,6 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper() LOG_INFO(log, "Attached all partitions, starting table"); startup(); - - are_restoring_replica.store(false); } void StorageReplicatedMergeTree::dropPartNoWaitNoThrow(const String & part_name) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c91152ca0f3..54a1d897bd7 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -322,7 +322,7 @@ private: /// If true, the table is offline and can not be written to it. std::atomic_bool is_readonly {false}; /// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case. - bool has_metadata_in_zookeeper = true; + std::optional has_metadata_in_zookeeper; static constexpr auto default_zookeeper_name = "default"; String zookeeper_name; From 4d30e6eab6056c7aef863e5ae681d89e8eb4b5b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 23 Jan 2022 08:27:29 +0300 Subject: [PATCH 11/37] Fix aggregate function combinator -If with Nullable filter argument --- .../AggregateFunctionFactory.cpp | 15 ++- .../AggregateFunctionIf.cpp | 102 +++++++++--------- .../0_stateless/02183_combinator_if.reference | 40 +++++++ .../0_stateless/02183_combinator_if.sql | 40 +++++++ 4 files changed, 138 insertions(+), 59 deletions(-) create mode 100644 tests/queries/0_stateless/02183_combinator_if.reference create mode 100644 tests/queries/0_stateless/02183_combinator_if.sql diff --git a/src/AggregateFunctions/AggregateFunctionFactory.cpp b/src/AggregateFunctions/AggregateFunctionFactory.cpp index eac761c1a82..347f4607dbf 100644 --- a/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -70,11 +70,11 @@ static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types) AggregateFunctionPtr AggregateFunctionFactory::get( const String & name, const DataTypes & argument_types, const Array & parameters, AggregateFunctionProperties & out_properties) const { - auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types); + auto types_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types); /// If one of the types is Nullable, we apply aggregate function combinator "Null". - if (std::any_of(type_without_low_cardinality.begin(), type_without_low_cardinality.end(), + if (std::any_of(types_without_low_cardinality.begin(), types_without_low_cardinality.end(), [](const auto & type) { return type->isNullable(); })) { AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null"); @@ -82,10 +82,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get( throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR); - DataTypes nested_types = combinator->transformArguments(type_without_low_cardinality); + DataTypes nested_types = combinator->transformArguments(types_without_low_cardinality); Array nested_parameters = combinator->transformParameters(parameters); - bool has_null_arguments = std::any_of(type_without_low_cardinality.begin(), type_without_low_cardinality.end(), + bool has_null_arguments = std::any_of(types_without_low_cardinality.begin(), types_without_low_cardinality.end(), [](const auto & type) { return type->onlyNull(); }); AggregateFunctionPtr nested_function = getImpl( @@ -97,13 +97,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get( // that are rewritten to AggregateFunctionNothing, in this case // nested_function is nullptr. if (!nested_function || !nested_function->isOnlyWindowFunction()) - { - return combinator->transformAggregateFunction(nested_function, - out_properties, type_without_low_cardinality, parameters); - } + return combinator->transformAggregateFunction(nested_function, out_properties, types_without_low_cardinality, parameters); } - auto with_original_arguments = getImpl(name, type_without_low_cardinality, parameters, out_properties, false); + auto with_original_arguments = getImpl(name, types_without_low_cardinality, parameters, out_properties, false); if (!with_original_arguments) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index d752900c018..c102f20d6ca 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -40,28 +40,6 @@ public: } }; -/** Given an array of flags, checks if it's all zeros - * When the buffer is all zeros, this is slightly faster than doing a memcmp since doesn't require allocating memory - * When the buffer has values, this is much faster since it avoids visiting all memory (and the allocation and function calls) - */ -static bool ALWAYS_INLINE inline is_all_zeros(const UInt8 * flags, size_t size) -{ - size_t unroll_size = size - size % 8; - size_t i = 0; - while (i < unroll_size) - { - UInt64 v = *reinterpret_cast(&flags[i]); - if (v) - return false; - i += 8; - } - - for (; i < size; ++i) - if (flags[i]) - return false; - - return true; -} /** There are two cases: for single argument and variadic. * Code for single argument is much more efficient. @@ -73,6 +51,7 @@ class AggregateFunctionIfNullUnary final { private: size_t num_arguments; + bool filter_is_nullable = false; /// The name of the nested function, including combinators (i.e. *If) /// @@ -92,8 +71,26 @@ private: using Base = AggregateFunctionNullBase>; -public: + inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) const + { + const IColumn * filter_column = columns[num_arguments - 1]; + + if (filter_is_nullable) + { + const ColumnNullable * nullable_column = assert_cast(filter_column); + filter_column = nullable_column->getNestedColumnPtr().get(); + const UInt8 * filter_null_map = nullable_column->getNullMapData().data(); + + return assert_cast(*filter_column).getData()[row_num] && !filter_null_map[row_num]; + } + else + { + return assert_cast(*filter_column).getData()[row_num]; + } + } + +public: String getName() const override { return name; @@ -105,17 +102,10 @@ public: , name(name_) { if (num_arguments == 0) - throw Exception("Aggregate function " + getName() + " require at least one argument", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - } + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Aggregate function {} require at least one argument", getName()); - static inline bool singleFilter(const IColumn ** columns, size_t row_num, size_t num_arguments) - { - const IColumn * filter_column = columns[num_arguments - 1]; - if (const ColumnNullable * nullable_column = typeid_cast(filter_column)) - filter_column = nullable_column->getNestedColumnPtr().get(); - - return assert_cast(*filter_column).getData()[row_num]; + filter_is_nullable = arguments[num_arguments - 1]->isNullable(); } void add(AggregateDataPtr __restrict place, const IColumn ** columns, size_t row_num, Arena * arena) const override @@ -136,28 +126,36 @@ public: const IColumn * columns_param[] = {&column->getNestedColumn()}; const IColumn * filter_column = columns[num_arguments - 1]; - if (const ColumnNullable * nullable_column = typeid_cast(filter_column)) - filter_column = nullable_column->getNestedColumnPtr().get(); - if constexpr (result_is_nullable) + + const UInt8 * filter_values = nullptr; + const UInt8 * filter_null_map = nullptr; + + if (filter_is_nullable) { - /// We need to check if there is work to do as otherwise setting the flag would be a mistake, - /// it would mean that the return value would be the default value of the nested type instead of NULL - if (is_all_zeros(assert_cast(filter_column)->getData().data(), batch_size)) - return; + const ColumnNullable * nullable_column = assert_cast(filter_column); + filter_column = nullable_column->getNestedColumnPtr().get(); + filter_null_map = nullable_column->getNullMapData().data(); } + filter_values = assert_cast(filter_column)->getData().data(); + /// Combine the 2 flag arrays so we can call a simplified version (one check vs 2) /// Note that now the null map will contain 0 if not null and not filtered, or 1 for null or filtered (or both) - const auto * filter_flags = assert_cast(filter_column)->getData().data(); + auto final_nulls = std::make_unique(batch_size); - for (size_t i = 0; i < batch_size; ++i) - final_nulls[i] = (!!null_map[i]) | (!filter_flags[i]); + + if (filter_null_map) + for (size_t i = 0; i < batch_size; ++i) + final_nulls[i] = (!!null_map[i]) | (!filter_values[i]) | (!!filter_null_map[i]); + else + for (size_t i = 0; i < batch_size; ++i) + final_nulls[i] = (!!null_map[i]) | (!filter_values[i]); this->nested_function->addBatchSinglePlaceNotNull( batch_size, this->nestedPlace(place), columns_param, final_nulls.get(), arena, -1); if constexpr (result_is_nullable) - if (!memoryIsByte(null_map, batch_size, 1)) + if (!memoryIsByte(final_nulls.get(), batch_size, 1)) this->setFlag(place); } @@ -367,10 +365,14 @@ AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( const AggregateFunctionPtr & nested_function, const DataTypes & arguments, const Array & params, const AggregateFunctionProperties & properties) const { - bool return_type_is_nullable = !properties.returns_default_when_only_null && getReturnType()->canBeInsideNullable(); - size_t nullable_size = std::count_if(arguments.begin(), arguments.end(), [](const auto & element) { return element->isNullable(); }); - return_type_is_nullable &= nullable_size != 1 || !arguments.back()->isNullable(); /// If only condition is nullable. we should non-nullable type. - bool serialize_flag = return_type_is_nullable || properties.returns_default_when_only_null; + assert(!arguments.empty()); + + /// Nullability of the last argument (condition) does not affect the nullability of the result (NULL is processed as false). + /// For other arguments it is as usual (at least one is NULL then the result is NULL if possible). + bool return_type_is_nullable = !properties.returns_default_when_only_null && getReturnType()->canBeInsideNullable() + && std::any_of(arguments.begin(), arguments.end() - 1, [](const auto & element) { return element->isNullable(); }); + + bool need_to_serialize_flag = return_type_is_nullable || properties.returns_default_when_only_null; if (arguments.size() <= 2 && arguments.front()->isNullable()) { @@ -380,7 +382,7 @@ AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( } else { - if (serialize_flag) + if (need_to_serialize_flag) return std::make_shared>(nested_function->getName(), nested_func, arguments, params); else return std::make_shared>(nested_function->getName(), nested_func, arguments, params); @@ -394,7 +396,7 @@ AggregateFunctionPtr AggregateFunctionIf::getOwnNullAdapter( } else { - if (serialize_flag) + if (need_to_serialize_flag) return std::make_shared>(nested_function, arguments, params); else return std::make_shared>(nested_function, arguments, params); diff --git a/tests/queries/0_stateless/02183_combinator_if.reference b/tests/queries/0_stateless/02183_combinator_if.reference new file mode 100644 index 00000000000..05bda3751b5 --- /dev/null +++ b/tests/queries/0_stateless/02183_combinator_if.reference @@ -0,0 +1,40 @@ +\N +\N +Hello +Hello +Hello + +Hello +Hello +0 \N +1 \N +0 Hello +1 Hello +0 +1 +0 Hello +1 Hello +0 \N +1 \N +0 Hello +1 Hello +\N +\N +Hello +Hello +Hello + +Hello +Hello +0 \N +1 \N +0 Hello +1 Hello +0 +1 +0 Hello +1 Hello +0 \N +1 \N +0 Hello +1 Hello diff --git a/tests/queries/0_stateless/02183_combinator_if.sql b/tests/queries/0_stateless/02183_combinator_if.sql new file mode 100644 index 00000000000..ec716407939 --- /dev/null +++ b/tests/queries/0_stateless/02183_combinator_if.sql @@ -0,0 +1,40 @@ +SELECT anyIf(toNullable('Hello'), arrayJoin([1, NULL]) = 0); + +SELECT anyIf(toNullable('Hello'), arrayJoin([1, 1]) = 0); +SELECT anyIf(toNullable('Hello'), arrayJoin([1, 0]) = 0); +SELECT anyIf(toNullable('Hello'), arrayJoin([0, 1]) = 0); +SELECT anyIf(toNullable('Hello'), arrayJoin([0, 0]) = 0); + +SELECT anyIf('Hello', arrayJoin([1, NULL]) = 0); +SELECT anyIf('Hello', arrayJoin([1, NULL]) = 1); +SELECT anyIf('Hello', arrayJoin([1, NULL]) IS NULL); + +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, NULL]) = 0) FROM numbers(2) GROUP BY number ORDER BY number; +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, NULL, 0]) = 0) FROM numbers(2) GROUP BY number ORDER BY number; + +SELECT number, anyIf('Hello', arrayJoin([1, NULL]) = 0) FROM numbers(2) GROUP BY number ORDER BY number; +SELECT number, anyIf('Hello', arrayJoin([1, NULL, 0]) = 0) FROM numbers(2) GROUP BY number ORDER BY number; + +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, 1]) = 0) FROM numbers(2) GROUP BY number ORDER BY number; +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, 0]) = 0) FROM numbers(2) GROUP BY number ORDER BY number; + + +SELECT anyIf(toNullable('Hello'), arrayJoin([1, NULL]) = 0) FROM remote('127.0.0.{1,2}', system.one); + +SELECT anyIf(toNullable('Hello'), arrayJoin([1, 1]) = 0) FROM remote('127.0.0.{1,2}', system.one); +SELECT anyIf(toNullable('Hello'), arrayJoin([1, 0]) = 0) FROM remote('127.0.0.{1,2}', system.one); +SELECT anyIf(toNullable('Hello'), arrayJoin([0, 1]) = 0) FROM remote('127.0.0.{1,2}', system.one); +SELECT anyIf(toNullable('Hello'), arrayJoin([0, 0]) = 0) FROM remote('127.0.0.{1,2}', system.one); + +SELECT anyIf('Hello', arrayJoin([1, NULL]) = 0) FROM remote('127.0.0.{1,2}', system.one); +SELECT anyIf('Hello', arrayJoin([1, NULL]) = 1) FROM remote('127.0.0.{1,2}', system.one); +SELECT anyIf('Hello', arrayJoin([1, NULL]) IS NULL) FROM remote('127.0.0.{1,2}', system.one); + +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, NULL]) = 0) FROM remote('127.0.0.{1,2}', numbers(2)) GROUP BY number ORDER BY number; +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, NULL, 0]) = 0) FROM remote('127.0.0.{1,2}', numbers(2)) GROUP BY number ORDER BY number; + +SELECT number, anyIf('Hello', arrayJoin([1, NULL]) = 0) FROM remote('127.0.0.{1,2}', numbers(2)) GROUP BY number ORDER BY number; +SELECT number, anyIf('Hello', arrayJoin([1, NULL, 0]) = 0) FROM remote('127.0.0.{1,2}', numbers(2)) GROUP BY number ORDER BY number; + +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, 1]) = 0) FROM remote('127.0.0.{1,2}', numbers(2)) GROUP BY number ORDER BY number; +SELECT number, anyIf(toNullable('Hello'), arrayJoin([1, 0]) = 0) FROM remote('127.0.0.{1,2}', numbers(2)) GROUP BY number ORDER BY number; From 51a6d08bff28b74c73ee50b4f38241e40185a4fb Mon Sep 17 00:00:00 2001 From: alesapin Date: Sun, 23 Jan 2022 19:31:09 +0300 Subject: [PATCH 12/37] Fix test_replica_is_active flaky test --- tests/integration/test_replica_is_active/test.py | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tests/integration/test_replica_is_active/test.py b/tests/integration/test_replica_is_active/test.py index 14046ea7f7d..f786ff71958 100644 --- a/tests/integration/test_replica_is_active/test.py +++ b/tests/integration/test_replica_is_active/test.py @@ -1,6 +1,7 @@ import pytest from helpers.client import QueryRuntimeException from helpers.cluster import ClickHouseCluster +from ast import literal_eval cluster = ClickHouseCluster(__file__) node1 = cluster.add_instance('node1', with_zookeeper=True) @@ -30,12 +31,12 @@ def start_cluster(): def test_replica_is_active(start_cluster): query_result = node1.query("select replica_is_active from system.replicas where table = 'test_table'") - assert query_result == '{\'node1\':1,\'node2\':1,\'node3\':1}\n' + assert literal_eval(query_result) == {'node1': 1, 'node2': 1, 'node3': 1} node3.stop() query_result = node1.query("select replica_is_active from system.replicas where table = 'test_table'") - assert query_result == '{\'node1\':1,\'node2\':1,\'node3\':0}\n' + assert literal_eval(query_result) == {'node1': 1, 'node2': 1, 'node3': 0} node2.stop() query_result = node1.query("select replica_is_active from system.replicas where table = 'test_table'") - assert query_result == '{\'node1\':1,\'node2\':0,\'node3\':0}\n' + assert literal_eval(query_result) == {'node1': 1, 'node2': 0, 'node3': 0} From 6630e64f5199fb581dd6fa6afd6cc1aaeaa62e6c Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Sun, 23 Jan 2022 20:42:08 -0400 Subject: [PATCH 13/37] Update tips.md --- docs/en/operations/tips.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md index 477d3b52965..64e65575f3f 100644 --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -129,6 +129,10 @@ If you want to divide an existing ZooKeeper cluster into two, the correct way is Do not run ZooKeeper on the same servers as ClickHouse. Because ZooKeeper is very sensitive for latency and ClickHouse may utilize all available system resources. +You can have ZooKeeper observers in an ensemble but ClickHouse servers should not interact with observers. + +Do not change `minSessionTimeout` setting, large values may affect ClickHouse restart stability. + With the default settings, ZooKeeper is a time bomb: > The ZooKeeper server won’t delete files from old snapshots and logs when using the default configuration (see autopurge), and this is the responsibility of the operator. From 60bfae1bdd43d4a0075cd585bcd1029e60ae8131 Mon Sep 17 00:00:00 2001 From: cnmade Date: Mon, 24 Jan 2022 09:30:08 +0800 Subject: [PATCH 14/37] Translate zh/faq/general/columnar-database: rename old file --- .../general/{columnar-database.md => columnar-database.md.bak} | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename docs/zh/faq/general/{columnar-database.md => columnar-database.md.bak} (100%) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md.bak similarity index 100% rename from docs/zh/faq/general/columnar-database.md rename to docs/zh/faq/general/columnar-database.md.bak From ce38c38c98e8b33c89f4ff5d754f6dfaee6937fd Mon Sep 17 00:00:00 2001 From: cnmade Date: Mon, 24 Jan 2022 09:30:56 +0800 Subject: [PATCH 15/37] Translate zh/faq/general/columnar-database: create new file --- docs/zh/faq/general/columnar-database.md | 25 ++++++++++++++++++++++++ 1 file changed, 25 insertions(+) create mode 100644 docs/zh/faq/general/columnar-database.md diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md new file mode 100644 index 00000000000..cbc5f77d0ba --- /dev/null +++ b/docs/zh/faq/general/columnar-database.md @@ -0,0 +1,25 @@ +--- +title: What is a columnar database? +toc_hidden: true +toc_priority: 101 +--- + +# What Is a Columnar Database? {#what-is-a-columnar-database} + +A columnar database stores data of each column independently. This allows to read data from disks only for those columns that are used in any given query. The cost is that operations that affect whole rows become proportionally more expensive. The synonym for a columnar database is a column-oriented database management system. ClickHouse is a typical example of such a system. + +Key columnar database advantages are: + +- Queries that use only a few columns out of many. +- Aggregating queries against large volumes of data. +- Column-wise data compression. + +Here is the illustration of the difference between traditional row-oriented systems and columnar databases when building reports: + +**Traditional row-oriented** +![Traditional row-oriented](https://clickhouse.com/docs/en/images/row-oriented.gif#) + +**Columnar** +![Columnar](https://clickhouse.com/docs/en/images/column-oriented.gif#) + +A columnar database is a preferred choice for analytical applications because it allows to have many columns in a table just in case, but do not pay the cost for unused columns on read query execution time. Column-oriented databases are designed for big data processing because and data warehousing, they often natively scale using distributed clusters of low-cost hardware to increase throughput. ClickHouse does it with combination of [distributed](../../engines/table-engines/special/distributed.md) and [replicated](../../engines/table-engines/mergetree-family/replication.md) tables. From b62c4c47de121b16e35f7ecfb815be1c33823281 Mon Sep 17 00:00:00 2001 From: cnmade Date: Mon, 24 Jan 2022 09:38:00 +0800 Subject: [PATCH 16/37] Translate zh/faq/general/columnar-database: translate to zh --- docs/zh/faq/general/columnar-database.md | 26 ++++++++++++------------ 1 file changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md index cbc5f77d0ba..4c826f99ae7 100644 --- a/docs/zh/faq/general/columnar-database.md +++ b/docs/zh/faq/general/columnar-database.md @@ -1,25 +1,25 @@ --- -title: What is a columnar database? +title: 什么是列存储数据库? toc_hidden: true toc_priority: 101 --- -# What Is a Columnar Database? {#what-is-a-columnar-database} +# 什么是列存储数据库? {#what-is-a-columnar-database} -A columnar database stores data of each column independently. This allows to read data from disks only for those columns that are used in any given query. The cost is that operations that affect whole rows become proportionally more expensive. The synonym for a columnar database is a column-oriented database management system. ClickHouse is a typical example of such a system. +列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是,影响整行的操作会按比例变得更昂贵。柱状数据库的同义词是面向列的数据库管理系统。ClickHouse就是这样一个典型的例子。 -Key columnar database advantages are: +柱状数据库的主要优点是: -- Queries that use only a few columns out of many. -- Aggregating queries against large volumes of data. -- Column-wise data compression. +- 查询只使用许多列其中的少数列。 +— 聚合对大量数据的查询。 +— 按列压缩。 -Here is the illustration of the difference between traditional row-oriented systems and columnar databases when building reports: +下面是构建报表时传统的面向行系统和柱状数据库之间的区别: -**Traditional row-oriented** -![Traditional row-oriented](https://clickhouse.com/docs/en/images/row-oriented.gif#) +**传统行存储** +!(传统行存储)(https://clickhouse.com/docs/en/images/row-oriented.gif) -**Columnar** -![Columnar](https://clickhouse.com/docs/en/images/column-oriented.gif#) +**列存储** +!(列存储)(https://clickhouse.com/docs/en/images/column-oriented.gif) -A columnar database is a preferred choice for analytical applications because it allows to have many columns in a table just in case, but do not pay the cost for unused columns on read query execution time. Column-oriented databases are designed for big data processing because and data warehousing, they often natively scale using distributed clusters of low-cost hardware to increase throughput. ClickHouse does it with combination of [distributed](../../engines/table-engines/special/distributed.md) and [replicated](../../engines/table-engines/mergetree-family/replication.md) tables. +列存储数据库是分析应用程序的首选,因为它允许在一个表中有许多列以防万一,但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的,因为和数据仓库一样,它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两个表。 \ No newline at end of file From bc30dd38926e3d4ba156e789c997f84394bd0165 Mon Sep 17 00:00:00 2001 From: cnmade Date: Mon, 24 Jan 2022 09:38:25 +0800 Subject: [PATCH 17/37] Translate zh/faq/general/columnar-database: Remove backup file --- docs/zh/faq/general/columnar-database.md.bak | 1 - 1 file changed, 1 deletion(-) delete mode 120000 docs/zh/faq/general/columnar-database.md.bak diff --git a/docs/zh/faq/general/columnar-database.md.bak b/docs/zh/faq/general/columnar-database.md.bak deleted file mode 120000 index b7557b62010..00000000000 --- a/docs/zh/faq/general/columnar-database.md.bak +++ /dev/null @@ -1 +0,0 @@ -../../../en/faq/general/columnar-database.md \ No newline at end of file From 37f520505ebbf7034c5ca87a5158389a19f998d3 Mon Sep 17 00:00:00 2001 From: cnmade Date: Mon, 24 Jan 2022 09:41:03 +0800 Subject: [PATCH 18/37] Translate zh/faq/general/columnar-database: translate adjust --- docs/zh/faq/general/columnar-database.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md index 4c826f99ae7..7aa50f5d9ec 100644 --- a/docs/zh/faq/general/columnar-database.md +++ b/docs/zh/faq/general/columnar-database.md @@ -6,9 +6,9 @@ toc_priority: 101 # 什么是列存储数据库? {#what-is-a-columnar-database} -列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是,影响整行的操作会按比例变得更昂贵。柱状数据库的同义词是面向列的数据库管理系统。ClickHouse就是这样一个典型的例子。 +列存储数据库独立存储每个列的数据。这只允许从磁盘读取任何给定查询中使用的列的数据。其代价是,影响整行的操作会按比例变得更昂贵。列存储数据库的同义词是面向列的数据库管理系统。ClickHouse就是这样一个典型的例子。 -柱状数据库的主要优点是: +列存储数据库的主要优点是: - 查询只使用许多列其中的少数列。 — 聚合对大量数据的查询。 From 83c1343455a52d31190db0b00f9a94c76e1c2014 Mon Sep 17 00:00:00 2001 From: cnmade Date: Mon, 24 Jan 2022 09:42:47 +0800 Subject: [PATCH 19/37] Translate zh/faq/general/columnar-database: translate words adjust --- docs/zh/faq/general/columnar-database.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/faq/general/columnar-database.md b/docs/zh/faq/general/columnar-database.md index 7aa50f5d9ec..185deaa7406 100644 --- a/docs/zh/faq/general/columnar-database.md +++ b/docs/zh/faq/general/columnar-database.md @@ -22,4 +22,4 @@ toc_priority: 101 **列存储** !(列存储)(https://clickhouse.com/docs/en/images/column-oriented.gif) -列存储数据库是分析应用程序的首选,因为它允许在一个表中有许多列以防万一,但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的,因为和数据仓库一样,它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两个表。 \ No newline at end of file +列存储数据库是分析应用程序的首选,因为它允许在一个表中有许多列以防万一,但不会在读取查询执行时为未使用的列付出代价。面向列的数据库是为大数据处理而设计的,因为和数据仓库一样,它们通常使用分布式的低成本硬件集群来提高吞吐量。ClickHouse结合了[分布式](../../engines/table-engines/special/distributed.md)和[复制式](../../engines/table-engines/mergetree-family/replication.md)两类表。 \ No newline at end of file From 58bb1ffecfca490e047bc8f521cda0dd41754bcf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 24 Jan 2022 08:29:17 +0300 Subject: [PATCH 20/37] Shortcut --- src/AggregateFunctions/AggregateFunctionIf.cpp | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/AggregateFunctions/AggregateFunctionIf.cpp b/src/AggregateFunctions/AggregateFunctionIf.cpp index c102f20d6ca..5ba54ff8505 100644 --- a/src/AggregateFunctions/AggregateFunctionIf.cpp +++ b/src/AggregateFunctions/AggregateFunctionIf.cpp @@ -151,12 +151,16 @@ public: for (size_t i = 0; i < batch_size; ++i) final_nulls[i] = (!!null_map[i]) | (!filter_values[i]); - this->nested_function->addBatchSinglePlaceNotNull( - batch_size, this->nestedPlace(place), columns_param, final_nulls.get(), arena, -1); - if constexpr (result_is_nullable) + { if (!memoryIsByte(final_nulls.get(), batch_size, 1)) this->setFlag(place); + else + return; /// No work to do. + } + + this->nested_function->addBatchSinglePlaceNotNull( + batch_size, this->nestedPlace(place), columns_param, final_nulls.get(), arena, -1); } #if USE_EMBEDDED_COMPILER From 2f566d2af9627a799a83df37bdccb79a48404441 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 24 Jan 2022 12:12:03 +0300 Subject: [PATCH 21/37] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Bharat Nallan --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 5379c3f89c1..e6c92b460d3 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4808,7 +4808,7 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper() if (getZooKeeper()->exists(replica_path)) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Replica path is present at {} - nothing to restore. " - "If you are sure that metadata is lost and replica path contain some garbage, " + "If you are sure that metadata is lost and that replica path contains some garbage, " "then use SYSTEM DROP REPLICA query first.", replica_path); if (has_metadata_in_zookeeper.has_value() && *has_metadata_in_zookeeper) From 34baca12985032ba97de14c338144f44d762fb21 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 24 Jan 2022 12:12:08 +0300 Subject: [PATCH 22/37] Update src/Storages/StorageReplicatedMergeTree.cpp Co-authored-by: Bharat Nallan --- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index e6c92b460d3..d25413f56eb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -4813,7 +4813,7 @@ void StorageReplicatedMergeTree::restoreMetadataInZooKeeper() if (has_metadata_in_zookeeper.has_value() && *has_metadata_in_zookeeper) throw Exception(ErrorCodes::LOGICAL_ERROR, "Replica has metadata in ZooKeeper: " - "it's a bug or a result of manual intervention into ZooKeeper"); + "it's either a bug or it's a result of manual intervention to ZooKeeper"); if (are_restoring_replica.exchange(true)) throw Exception(ErrorCodes::CONCURRENT_ACCESS_NOT_SUPPORTED, "Replica restoration in progress"); From 766786b570032c9366799a56d4fa180cf0821ce2 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Mon, 24 Jan 2022 12:16:13 +0300 Subject: [PATCH 23/37] Update StorageReplicatedMergeTree.h --- src/Storages/StorageReplicatedMergeTree.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 54a1d897bd7..116f3f11972 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -321,6 +321,7 @@ private: /// If true, the table is offline and can not be written to it. std::atomic_bool is_readonly {false}; + /// If nullopt - ZooKeeper is not available, so we don't know if there is table metadata. /// If false - ZooKeeper is available, but there is no table metadata. It's safe to drop table in this case. std::optional has_metadata_in_zookeeper; From ab2aca26999cc36f5af4eb40a8f595badaed2a32 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Jan 2022 13:17:40 +0300 Subject: [PATCH 24/37] Fix bug in keeper which can lead to inconsistent snapshots --- src/Coordination/KeeperStateMachine.cpp | 2 +- src/Coordination/KeeperStorage.h | 4 ++-- src/Coordination/SnapshotableHashTable.h | 13 ++++++++----- src/Coordination/tests/gtest_coordination.cpp | 10 +++++----- 4 files changed, 16 insertions(+), 13 deletions(-) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index aa6bbe42608..20d3bcbfd30 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -216,7 +216,7 @@ void KeeperStateMachine::create_snapshot( std::lock_guard lock(storage_and_responses_lock); LOG_TRACE(log, "Clearing garbage after snapshot"); /// Turn off "snapshot mode" and clear outdate part of storage state - storage->clearGarbageAfterSnapshot(snapshot->snapshot_container_size); + storage->clearGarbageAfterSnapshot(); LOG_TRACE(log, "Cleared garbage after snapshot"); snapshot.reset(); } diff --git a/src/Coordination/KeeperStorage.h b/src/Coordination/KeeperStorage.h index 35dbd22ffc2..cbf33be61a0 100644 --- a/src/Coordination/KeeperStorage.h +++ b/src/Coordination/KeeperStorage.h @@ -177,9 +177,9 @@ public: } /// Clear outdated data from internal container. - void clearGarbageAfterSnapshot(size_t up_to_size) + void clearGarbageAfterSnapshot() { - container.clearOutdatedNodes(up_to_size); + container.clearOutdatedNodes(); } /// Get all active sessions diff --git a/src/Coordination/SnapshotableHashTable.h b/src/Coordination/SnapshotableHashTable.h index bf74fe7484c..b1d72578530 100644 --- a/src/Coordination/SnapshotableHashTable.h +++ b/src/Coordination/SnapshotableHashTable.h @@ -33,6 +33,7 @@ private: List list; IndexMap map; bool snapshot_mode{false}; + /// Allows to avoid additional copies in updateValue function size_t snapshot_up_to_size = 0; ArenaWithFreeLists arena; @@ -226,8 +227,10 @@ public: if (snapshot_mode) { + /// We in snapshot mode but updating some node which is already more + /// fresh than snapshot distance. So it will not participate in + /// snapshot and we don't need to copy it. size_t distance = std::distance(list.begin(), list_itr); - if (distance < snapshot_up_to_size) { auto elem_copy = *(list_itr); @@ -269,11 +272,11 @@ public: return it->getMapped()->value; } - void clearOutdatedNodes(size_t up_to_size) + void clearOutdatedNodes() { auto start = list.begin(); - size_t counter = 0; - for (auto itr = start; counter < up_to_size; ++counter) + auto end = list.end(); + for (auto itr = start; itr != end;) { if (!itr->active_in_map) { @@ -288,7 +291,6 @@ public: itr++; } } - } void clear() @@ -308,6 +310,7 @@ public: void disableSnapshotMode() { + snapshot_mode = false; snapshot_up_to_size = 0; } diff --git a/src/Coordination/tests/gtest_coordination.cpp b/src/Coordination/tests/gtest_coordination.cpp index bc575388f46..9c434ebb653 100644 --- a/src/Coordination/tests/gtest_coordination.cpp +++ b/src/Coordination/tests/gtest_coordination.cpp @@ -908,7 +908,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapTrySnapshot) EXPECT_EQ(itr->active_in_map, i != 3 && i != 2); itr = std::next(itr); } - map_snp.clearOutdatedNodes(map_snp.snapshotSize()); + map_snp.clearOutdatedNodes(); EXPECT_EQ(map_snp.snapshotSize(), 4); EXPECT_EQ(map_snp.size(), 4); @@ -957,13 +957,13 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) hello.updateValue("hello", [](IntNode & value) { value = 2; }); EXPECT_EQ(hello.getApproximateDataSize(), 18); - hello.clearOutdatedNodes(hello.snapshotSize()); + hello.clearOutdatedNodes(); EXPECT_EQ(hello.getApproximateDataSize(), 9); hello.erase("hello"); EXPECT_EQ(hello.getApproximateDataSize(), 9); - hello.clearOutdatedNodes(hello.snapshotSize()); + hello.clearOutdatedNodes(); EXPECT_EQ(hello.getApproximateDataSize(), 0); /// Node @@ -990,7 +990,7 @@ TEST_P(CoordinationTest, SnapshotableHashMapDataSize) world.updateValue("world", [&](Node & value) { value = n2; }); EXPECT_EQ(world.getApproximateDataSize(), 196); - world.clearOutdatedNodes(world.snapshotSize()); + world.clearOutdatedNodes(); EXPECT_EQ(world.getApproximateDataSize(), 98); world.erase("world"); @@ -1170,7 +1170,7 @@ TEST_P(CoordinationTest, TestStorageSnapshotMode) } EXPECT_TRUE(fs::exists("./snapshots/snapshot_50.bin" + params.extension)); EXPECT_EQ(storage.container.size(), 26); - storage.clearGarbageAfterSnapshot(storage.container.snapshotSize()); + storage.clearGarbageAfterSnapshot(); EXPECT_EQ(storage.container.snapshotSize(), 26); for (size_t i = 0; i < 50; ++i) { From 8d2750a92a2d4bfe314612747794b63fd97f7452 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 24 Jan 2022 13:23:58 +0300 Subject: [PATCH 25/37] Fix keeper data dumper build --- utils/keeper-data-dumper/main.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 485f8c09faf..0f86d34d334 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -37,9 +37,9 @@ void dumpMachine(std::shared_ptr machine) for (const auto & child : value.children) { if (key == "/") - keys.push(key + child); + keys.push(key + child.toString()); else - keys.push(key + "/" + child); + keys.push(key + "/" + child.toString()); } } std::cout << std::flush; From 37dbff7587da2bd19fb382362b8f61e8a599a727 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Jan 2022 16:00:52 +0300 Subject: [PATCH 26/37] Revert glibc compatibility (via .symver) in favor of hermetic build (bundled libc) This patch reverts glibc compatibility (via .symver) #29594, in favor of hermetic build (that provides bundled libc version) #30011 Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 8 ---- base/glibc-compatibility/glibc-compat-2.32.h | 50 -------------------- 2 files changed, 58 deletions(-) delete mode 100644 base/glibc-compatibility/glibc-compat-2.32.h diff --git a/CMakeLists.txt b/CMakeLists.txt index d33bdd8b33c..d50b815a800 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -208,14 +208,6 @@ elseif(GLIBC_COMPATIBILITY) message (${RECONFIGURE_MESSAGE_LEVEL} "Glibc compatibility cannot be enabled in current configuration") endif () -if (GLIBC_COMPATIBILITY) - # NOTE: we may also want to check glibc version and add -include only for 2.32+ - # however this is extra complexity, especially for cross compiling. - # And anyway it should not break anything for <2.32. - set(CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -include ${CMAKE_CURRENT_SOURCE_DIR}/base/glibc-compatibility/glibc-compat-2.32.h") - set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -include ${CMAKE_CURRENT_SOURCE_DIR}/base/glibc-compatibility/glibc-compat-2.32.h") -endif() - # Make sure the final executable has symbols exported set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -rdynamic") diff --git a/base/glibc-compatibility/glibc-compat-2.32.h b/base/glibc-compatibility/glibc-compat-2.32.h deleted file mode 100644 index 53ed34d60fa..00000000000 --- a/base/glibc-compatibility/glibc-compat-2.32.h +++ /dev/null @@ -1,50 +0,0 @@ -/// In glibc 2.32 new version of some symbols had been added [1]: -/// -/// $ nm -D clickhouse | fgrep -e @GLIBC_2.32 -/// U pthread_getattr_np@GLIBC_2.32 -/// U pthread_sigmask@GLIBC_2.32 -/// -/// [1]: https://www.spinics.net/lists/fedora-devel/msg273044.html -/// -/// Right now ubuntu 20.04 is used as official image for building -/// ClickHouse, however once it will be switched someone may not be happy -/// with that fact that he/she cannot use official binaries anymore because -/// they have glibc < 2.32. -/// -/// To avoid this dependency, let's force previous version of those -/// symbols from glibc. -/// -/// Also note, that the following approach had been tested: -/// a) -Wl,--wrap -- but it goes into endless recursion whey you try to do -/// something like this: -/// -/// int __pthread_getattr_np_compact(pthread_t thread, pthread_attr_t *attr); -/// GLIBC_COMPAT_SYMBOL(__pthread_getattr_np_compact, pthread_getattr_np) -/// int __pthread_getattr_np_compact(pthread_t thread, pthread_attr_t *attr); -/// int __wrap_pthread_getattr_np(pthread_t thread, pthread_attr_t *attr) -/// { -/// return __pthread_getattr_np_compact(thread, attr); -/// } -/// -/// int __pthread_sigmask_compact(int how, const sigset_t *set, sigset_t *oldset); -/// GLIBC_COMPAT_SYMBOL(__pthread_sigmask_compact, pthread_sigmask) -/// int __pthread_sigmask_compact(int how, const sigset_t *set, sigset_t *oldset); -/// int __wrap_pthread_sigmask(int how, const sigset_t *set, sigset_t *oldset) -/// { -/// return __pthread_sigmask_compact(how, set, oldset); -/// } -/// -/// b) -Wl,--defsym -- same problems (and you cannot use version of symbol with -/// version in the expression) -/// c) this approach -- simply add this file with -include directive. - -#if defined(__amd64__) -#define GLIBC_COMPAT_SYMBOL(func) __asm__(".symver " #func "," #func "@GLIBC_2.2.5"); -#elif defined(__aarch64__) -#define GLIBC_COMPAT_SYMBOL(func) __asm__(".symver " #func "," #func "@GLIBC_2.17"); -#else -#error Your platform is not supported. -#endif - -GLIBC_COMPAT_SYMBOL(pthread_sigmask) -GLIBC_COMPAT_SYMBOL(pthread_getattr_np) From 1f49acc164d7e3fa195e5b593e6aea4ba728a568 Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 24 Jan 2022 16:27:04 +0300 Subject: [PATCH 27/37] Better naming --- src/Formats/FormatFactory.cpp | 2 +- src/Formats/FormatFactory.h | 2 +- src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/AvroRowOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/JSONRowOutputFormat.cpp | 4 ++-- src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/XMLRowOutputFormat.cpp | 2 +- 8 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/Formats/FormatFactory.cpp b/src/Formats/FormatFactory.cpp index d0425bd79f8..3f220cbb20a 100644 --- a/src/Formats/FormatFactory.cpp +++ b/src/Formats/FormatFactory.cpp @@ -402,7 +402,7 @@ void FormatFactory::registerAppendSupportChecker(const String & name, AppendSupp target = std::move(append_support_checker); } -void FormatFactory::markFormatDoesntSupportAppend(const String & name) +void FormatFactory::markFormatHasNoAppendSupport(const String & name) { registerAppendSupportChecker(name, [](const FormatSettings &){ return false; }); } diff --git a/src/Formats/FormatFactory.h b/src/Formats/FormatFactory.h index 3eb99349ce8..228d5234959 100644 --- a/src/Formats/FormatFactory.h +++ b/src/Formats/FormatFactory.h @@ -176,7 +176,7 @@ public: /// If format always doesn't support append, you can use this method instead of /// registerAppendSupportChecker with append_support_checker that always returns true. - void markFormatDoesntSupportAppend(const String & name); + void markFormatHasNoAppendSupport(const String & name); bool checkIfFormatSupportAppend(const String & name, ContextPtr context, const std::optional & format_settings_ = std::nullopt); diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index c87191d0a12..60408f13ff0 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -93,7 +93,7 @@ void registerOutputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, false, format_settings); }); - factory.markFormatDoesntSupportAppend("Arrow"); + factory.markFormatHasNoAppendSupport("Arrow"); factory.registerOutputFormat( "ArrowStream", @@ -104,7 +104,7 @@ void registerOutputFormatArrow(FormatFactory & factory) { return std::make_shared(buf, sample, true, format_settings); }); - factory.markFormatDoesntSupportAppend("ArrowStream"); + factory.markFormatHasNoAppendSupport("ArrowStream"); } } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index ae5ce6099a9..70373480920 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -479,7 +479,7 @@ void registerOutputFormatAvro(FormatFactory & factory) { return std::make_shared(buf, sample, params, settings); }); - factory.markFormatDoesntSupportAppend("Avro"); + factory.markFormatHasNoAppendSupport("Avro"); } } diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 7ac7d45d26d..8130b2b4cb1 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -284,7 +284,7 @@ void registerOutputFormatJSON(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSON"); - factory.markFormatDoesntSupportAppend("JSON"); + factory.markFormatHasNoAppendSupport("JSON"); factory.registerOutputFormat("JSONStrings", []( WriteBuffer & buf, @@ -296,7 +296,7 @@ void registerOutputFormatJSON(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("JSONStrings"); - factory.markFormatDoesntSupportAppend("JSONStrings"); + factory.markFormatHasNoAppendSupport("JSONStrings"); } } diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 67287c8a661..106b71a9df5 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -526,7 +526,7 @@ void registerOutputFormatORC(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings); }); - factory.markFormatDoesntSupportAppend("ORC"); + factory.markFormatHasNoAppendSupport("ORC"); } } diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index c3ddc632641..68e2ae1c6eb 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -85,7 +85,7 @@ void registerOutputFormatParquet(FormatFactory & factory) { return std::make_shared(buf, sample, format_settings); }); - factory.markFormatDoesntSupportAppend("Parquet"); + factory.markFormatHasNoAppendSupport("Parquet"); } } diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp index 6a80131a65e..cc2b37189f9 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.cpp @@ -256,7 +256,7 @@ void registerOutputFormatXML(FormatFactory & factory) }); factory.markOutputFormatSupportsParallelFormatting("XML"); - factory.markFormatDoesntSupportAppend("XML"); + factory.markFormatHasNoAppendSupport("XML"); } } From aa296c36bb68e1b1b98dfd1b5acd66c42e73c7b2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jan 2022 15:43:35 +0100 Subject: [PATCH 28/37] Fix dummy PRInfo --- tests/ci/pr_info.py | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index a155786d815..2fe7254f610 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -11,6 +11,8 @@ DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xm ".jpg", ".py", ".sh", ".json"] def get_pr_for_commit(sha, ref): + if not ref: + return try_get_pr_url = f"https://api.github.com/repos/{GITHUB_REPOSITORY}/commits/{sha}/pulls" try: response = requests.get(try_get_pr_url) @@ -32,23 +34,24 @@ def get_pr_for_commit(sha, ref): class PRInfo: + default_event = { + 'commits': 1, + 'before': 'HEAD~', + 'after': 'HEAD', + 'ref': None, + } def __init__(self, github_event=None, need_orgs=False, need_changed_files=False, labels_from_api=False): if not github_event: if GITHUB_EVENT_PATH: with open(GITHUB_EVENT_PATH, 'r', encoding='utf-8') as event_file: github_event = json.load(event_file) else: - github_event = { - 'commits': 1, - 'before': 'HEAD~', - 'after': 'HEAD', - 'ref': None, - } + github_event = PRInfo.default_event.copy() self.event = github_event self.changed_files = set([]) self.body = "" ref = github_event.get("ref", "refs/head/master") - if ref.startswith('refs/heads/'): + if ref and ref.startswith('refs/heads/'): ref = ref[11:] # workflow completed event, used for PRs only From 3531318388eac9ec677226e5051338ac4480f2aa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jan 2022 15:54:14 +0100 Subject: [PATCH 29/37] Add tests for get_changed_docker_images --- tests/ci/docker_test.py | 40 +++++++ tests/ci/tests/docker_images.json | 166 ++++++++++++++++++++++++++++++ 2 files changed, 206 insertions(+) create mode 100644 tests/ci/docker_test.py create mode 100644 tests/ci/tests/docker_images.json diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py new file mode 100644 index 00000000000..da635631c54 --- /dev/null +++ b/tests/ci/docker_test.py @@ -0,0 +1,40 @@ +#!/usr/bin/env python + +import os +import unittest + +from pr_info import PRInfo +import docker_images_check as di + + +class TestDockerImageCheck(unittest.TestCase): + docker_images_path = os.path.join( + os.path.dirname(__file__), "tests/docker_images.json" + ) + + def test_get_changed_docker_images(self): + pr_info = PRInfo() + pr_info.changed_files = { + "docker/test/stateless", + "docker/test/base", + "docker/docs/builder", + } + images = di.get_changed_docker_images(pr_info, "/", self.docker_images_path) + expected = [ + ("docker/test/base", "clickhouse/test-base"), + ("docker/docs/builder", "clickhouse/docs-builder"), + ("docker/test/stateless", "clickhouse/stateless-test"), + ("docker/test/integration/base", "clickhouse/integration-test"), + ("docker/test/fuzzer", "clickhouse/fuzzer"), + ("docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test"), + ("docker/docs/check", "clickhouse/docs-check"), + ("docker/docs/release", "clickhouse/docs-release"), + ("docker/test/stateful", "clickhouse/stateful-test"), + ("docker/test/unit", "clickhouse/unit-test"), + ("docker/test/stress", "clickhouse/stress-test"), + ] + self.assertEqual(images, expected) + + +if __name__ == "__main__": + unittest.main() diff --git a/tests/ci/tests/docker_images.json b/tests/ci/tests/docker_images.json new file mode 100644 index 00000000000..354bdaa8728 --- /dev/null +++ b/tests/ci/tests/docker_images.json @@ -0,0 +1,166 @@ +{ + "docker/packager/deb": { + "name": "clickhouse/deb-builder", + "dependent": [] + }, + "docker/packager/binary": { + "name": "clickhouse/binary-builder", + "dependent": [ + "docker/test/split_build_smoke_test", + "docker/test/pvs", + "docker/test/codebrowser" + ] + }, + "docker/test/compatibility/centos": { + "name": "clickhouse/test-old-centos", + "dependent": [] + }, + "docker/test/compatibility/ubuntu": { + "name": "clickhouse/test-old-ubuntu", + "dependent": [] + }, + "docker/test/integration/base": { + "name": "clickhouse/integration-test", + "dependent": [] + }, + "docker/test/fuzzer": { + "name": "clickhouse/fuzzer", + "dependent": [] + }, + "docker/test/performance-comparison": { + "name": "clickhouse/performance-comparison", + "dependent": [] + }, + "docker/test/pvs": { + "name": "clickhouse/pvs-test", + "dependent": [] + }, + "docker/test/util": { + "name": "clickhouse/test-util", + "dependent": [ + "docker/test/base", + "docker/test/fasttest" + ] + }, + "docker/test/stateless": { + "name": "clickhouse/stateless-test", + "dependent": [ + "docker/test/stateful", + "docker/test/unit" + ] + }, + "docker/test/stateful": { + "name": "clickhouse/stateful-test", + "dependent": [ + "docker/test/stress" + ] + }, + "docker/test/unit": { + "name": "clickhouse/unit-test", + "dependent": [] + }, + "docker/test/stress": { + "name": "clickhouse/stress-test", + "dependent": [] + }, + "docker/test/split_build_smoke_test": { + "name": "clickhouse/split-build-smoke-test", + "dependent": [] + }, + "docker/test/codebrowser": { + "name": "clickhouse/codebrowser", + "dependent": [] + }, + "docker/test/integration/runner": { + "name": "clickhouse/integration-tests-runner", + "dependent": [] + }, + "docker/test/testflows/runner": { + "name": "clickhouse/testflows-runner", + "dependent": [] + }, + "docker/test/fasttest": { + "name": "clickhouse/fasttest", + "dependent": [] + }, + "docker/test/style": { + "name": "clickhouse/style-test", + "dependent": [] + }, + "docker/test/integration/s3_proxy": { + "name": "clickhouse/s3-proxy", + "dependent": [] + }, + "docker/test/integration/resolver": { + "name": "clickhouse/python-bottle", + "dependent": [] + }, + "docker/test/integration/helper_container": { + "name": "clickhouse/integration-helper", + "dependent": [] + }, + "docker/test/integration/mysql_golang_client": { + "name": "clickhouse/mysql-golang-client", + "dependent": [] + }, + "docker/test/integration/dotnet_client": { + "name": "clickhouse/dotnet-client", + "dependent": [] + }, + "docker/test/integration/mysql_java_client": { + "name": "clickhouse/mysql-java-client", + "dependent": [] + }, + "docker/test/integration/mysql_js_client": { + "name": "clickhouse/mysql-js-client", + "dependent": [] + }, + "docker/test/integration/mysql_php_client": { + "name": "clickhouse/mysql-php-client", + "dependent": [] + }, + "docker/test/integration/postgresql_java_client": { + "name": "clickhouse/postgresql-java-client", + "dependent": [] + }, + "docker/test/integration/kerberos_kdc": { + "name": "clickhouse/kerberos-kdc", + "dependent": [] + }, + "docker/test/base": { + "name": "clickhouse/test-base", + "dependent": [ + "docker/test/stateless", + "docker/test/integration/base", + "docker/test/fuzzer", + "docker/test/keeper-jepsen" + ] + }, + "docker/test/integration/kerberized_hadoop": { + "name": "clickhouse/kerberized-hadoop", + "dependent": [] + }, + "docker/test/sqlancer": { + "name": "clickhouse/sqlancer-test", + "dependent": [] + }, + "docker/test/keeper-jepsen": { + "name": "clickhouse/keeper-jepsen-test", + "dependent": [] + }, + "docker/docs/builder": { + "name": "clickhouse/docs-builder", + "dependent": [ + "docker/docs/check", + "docker/docs/release" + ] + }, + "docker/docs/check": { + "name": "clickhouse/docs-check", + "dependent": [] + }, + "docker/docs/release": { + "name": "clickhouse/docs-release", + "dependent": [] + } +} From 18ef029ed333c84596d0f7b43dc5eed580694210 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jan 2022 21:09:39 +0100 Subject: [PATCH 30/37] Fix issue with building from non-existing tags --- tests/ci/docker_images_check.py | 172 +++++++++++++++++++++----------- tests/ci/docker_test.py | 87 +++++++++++++--- 2 files changed, 184 insertions(+), 75 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index d698d18a58b..2fbca7ab548 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -6,7 +6,7 @@ import os import shutil import subprocess import time -from typing import List, Tuple +from typing import List, Optional, Set, Tuple, Union from github import Github @@ -24,9 +24,37 @@ NAME = "Push to Dockerhub (actions)" TEMP_PATH = os.path.join(RUNNER_TEMP, "docker_images_check") +class DockerImage: + def __init__( + self, + path: str, + repo: str, + parent: Optional["DockerImage"] = None, + gh_repo_path: str = GITHUB_WORKSPACE, + ): + self.path = path + self.full_path = os.path.join(gh_repo_path, path) + self.repo = repo + self.parent = parent + self.built = False + + def __eq__(self, other): + """Is used to check if DockerImage is in a set or not""" + return self.path == other.path + + def __hash__(self): + return hash(self.path) + + def __str__(self): + return self.repo + + def __repr__(self): + return f"DockerImage(path={self.path},path={self.path},parent={self.parent})" + + def get_changed_docker_images( pr_info: PRInfo, repo_path: str, image_file_path: str -) -> List[Tuple[str, str]]: +) -> Set[DockerImage]: images_dict = {} path_to_images_file = os.path.join(repo_path, image_file_path) if os.path.exists(path_to_images_file): @@ -38,7 +66,7 @@ def get_changed_docker_images( ) if not images_dict: - return [] + return set() files_changed = pr_info.changed_files @@ -54,14 +82,15 @@ def get_changed_docker_images( for dockerfile_dir, image_description in images_dict.items(): for f in files_changed: if f.startswith(dockerfile_dir): + name = image_description["name"] logging.info( "Found changed file '%s' which affects " "docker image '%s' with path '%s'", f, - image_description["name"], + name, dockerfile_dir, ) - changed_images.append(dockerfile_dir) + changed_images.append(DockerImage(dockerfile_dir, name)) break # The order is important: dependents should go later than bases, so that @@ -69,14 +98,14 @@ def get_changed_docker_images( index = 0 while index < len(changed_images): image = changed_images[index] - for dependent in images_dict[image]["dependent"]: + for dependent in images_dict[image.path]["dependent"]: logging.info( "Marking docker image '%s' as changed because it " "depends on changed docker image '%s'", dependent, image, ) - changed_images.append(dependent) + changed_images.append(DockerImage(dependent, image.repo, image)) index += 1 if index > 5 * len(images_dict): # Sanity check to prevent infinite loop. @@ -84,19 +113,9 @@ def get_changed_docker_images( f"Too many changed docker images, this is a bug. {changed_images}" ) - # If a dependent image was already in the list because its own files - # changed, but then it was added as a dependent of a changed base, we - # must remove the earlier entry so that it doesn't go earlier than its - # base. This way, the dependent will be rebuilt later than the base, and - # will correctly use the updated version of the base. - seen = set() - no_dups_reversed = [] - for x in reversed(changed_images): - if x not in seen: - seen.add(x) - no_dups_reversed.append(x) - - result = [(x, images_dict[x]["name"]) for x in reversed(no_dups_reversed)] + # With reversed changed_images set will use images with parents first, and + # images without parents then + result = set(reversed(changed_images)) logging.info( "Changed docker images for PR %s @ %s: '%s'", pr_info.number, @@ -106,66 +125,112 @@ def get_changed_docker_images( return result +def gen_versions( + pr_info: PRInfo, suffix: Optional[str] +) -> Tuple[List[str], Union[str, List[str]]]: + pr_commit_version = str(pr_info.number) + "-" + pr_info.sha + # The order is important, PR number is used as cache during the build + versions = [str(pr_info.number), pr_commit_version] + result_version = pr_commit_version + if pr_info.number == 0: + # First get the latest for cache + versions.insert(0, "latest") + + if suffix: + # We should build architecture specific images separately and merge a + # manifest lately in a different script + versions = [f"{v}-{suffix}" for v in versions] + # changed_images_{suffix}.json should contain all changed images + result_version = versions + + return versions, result_version + + def build_and_push_one_image( - path_to_dockerfile_folder: str, image_name: str, version_string: str, push: bool + image: DockerImage, + version_string: str, + push: bool, + child: bool, ) -> Tuple[bool, str]: - path = path_to_dockerfile_folder logging.info( "Building docker image %s with version %s from path %s", - image_name, + image.repo, version_string, - path, + image.full_path, ) build_log = os.path.join( TEMP_PATH, "build_and_push_log_{}_{}".format( - str(image_name).replace("/", "_"), version_string + str(image.repo).replace("/", "_"), version_string ), ) push_arg = "" if push: push_arg = "--push " + from_tag_arg = "" + if child: + from_tag_arg = f"--build-arg FROM_TAG={version_string} " + with open(build_log, "w") as bl: cmd = ( "docker buildx build --builder default " - f"--build-arg FROM_TAG={version_string} " + f"{from_tag_arg}" f"--build-arg BUILDKIT_INLINE_CACHE=1 " - f"--tag {image_name}:{version_string} " - f"--cache-from type=registry,ref={image_name}:{version_string} " + f"--tag {image.repo}:{version_string} " + f"--cache-from type=registry,ref={image.repo}:{version_string} " f"{push_arg}" - f"--progress plain {path}" + f"--progress plain {image.full_path}" ) logging.info("Docker command to run: %s", cmd) - retcode = subprocess.Popen(cmd, shell=True, stderr=bl, stdout=bl).wait() + with subprocess.Popen(cmd, shell=True, stderr=bl, stdout=bl) as proc: + retcode = proc.wait() + if retcode != 0: return False, build_log - logging.info("Processing of %s successfully finished", image_name) + logging.info("Processing of %s successfully finished", image.repo) return True, build_log def process_single_image( - versions: List[str], path_to_dockerfile_folder: str, image_name: str, push: bool + image: DockerImage, + versions: List[str], + push: bool, + child: bool, ) -> List[Tuple[str, str, str]]: logging.info("Image will be pushed with versions %s", ", ".join(versions)) result = [] for ver in versions: for i in range(5): - success, build_log = build_and_push_one_image( - path_to_dockerfile_folder, image_name, ver, push - ) + success, build_log = build_and_push_one_image(image, ver, push, child) if success: - result.append((image_name + ":" + ver, build_log, "OK")) + result.append((image.repo + ":" + ver, build_log, "OK")) break logging.info( "Got error will retry %s time and sleep for %s seconds", i, i * 5 ) time.sleep(i * 5) else: - result.append((image_name + ":" + ver, build_log, "FAIL")) + result.append((image.repo + ":" + ver, build_log, "FAIL")) logging.info("Processing finished") + image.built = True + return result + + +def process_image_with_parents( + image: DockerImage, versions: List[str], push: bool, child: bool = False +) -> List[Tuple[str, str, str]]: + result = [] # type: List[Tuple[str,str,str]] + if image.built: + return result + + if image.parent is not None: + result += process_image_with_parents(image.parent, versions, push, False) + child = True + + result += process_single_image(image, versions, push, child) return result @@ -255,8 +320,6 @@ def main(): shell=True, ) - repo_path = GITHUB_WORKSPACE - if os.path.exists(TEMP_PATH): shutil.rmtree(TEMP_PATH) os.makedirs(TEMP_PATH) @@ -267,36 +330,23 @@ def main(): else: pr_info = PRInfo(need_changed_files=True) - changed_images = get_changed_docker_images(pr_info, repo_path, "docker/images.json") - logging.info( - "Has changed images %s", ", ".join([str(image[0]) for image in changed_images]) + changed_images = get_changed_docker_images( + pr_info, GITHUB_WORKSPACE, "docker/images.json" ) - pr_commit_version = str(pr_info.number) + "-" + pr_info.sha - # The order is important, PR number is used as cache during the build - versions = [str(pr_info.number), pr_commit_version] - result_version = pr_commit_version - if pr_info.number == 0: - # First get the latest for cache - versions.insert(0, "latest") + logging.info("Has changed images %s", ", ".join([im.path for im in changed_images])) - if args.suffix: - # We should build architecture specific images separately and merge a - # manifest lately in a different script - versions = [f"{v}-{args.suffix}" for v in versions] - # changed_images_{suffix}.json should contain all changed images - result_version = versions + image_versions, result_version = gen_versions(pr_info, args.suffix) result_images = {} images_processing_result = [] - for rel_path, image_name in changed_images: - full_path = os.path.join(repo_path, rel_path) - images_processing_result += process_single_image( - versions, full_path, image_name, push + for image in changed_images: + images_processing_result += process_image_with_parents( + image, image_versions, push ) - result_images[image_name] = result_version + result_images[image.repo] = result_version if changed_images: - description = "Updated " + ",".join([im[1] for im in changed_images]) + description = "Updated " + ",".join([im.repo for im in changed_images]) else: description = "Nothing to update" diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index da635631c54..3fed5112005 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -2,10 +2,13 @@ import os import unittest +from unittest.mock import patch from pr_info import PRInfo import docker_images_check as di +# di.logging.basicConfig(level=di.logging.INFO) + class TestDockerImageCheck(unittest.TestCase): docker_images_path = os.path.join( @@ -13,28 +16,84 @@ class TestDockerImageCheck(unittest.TestCase): ) def test_get_changed_docker_images(self): - pr_info = PRInfo() + pr_info = PRInfo(PRInfo.default_event.copy()) pr_info.changed_files = { "docker/test/stateless", "docker/test/base", "docker/docs/builder", } images = di.get_changed_docker_images(pr_info, "/", self.docker_images_path) - expected = [ - ("docker/test/base", "clickhouse/test-base"), - ("docker/docs/builder", "clickhouse/docs-builder"), - ("docker/test/stateless", "clickhouse/stateless-test"), - ("docker/test/integration/base", "clickhouse/integration-test"), - ("docker/test/fuzzer", "clickhouse/fuzzer"), - ("docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test"), - ("docker/docs/check", "clickhouse/docs-check"), - ("docker/docs/release", "clickhouse/docs-release"), - ("docker/test/stateful", "clickhouse/stateful-test"), - ("docker/test/unit", "clickhouse/unit-test"), - ("docker/test/stress", "clickhouse/stress-test"), - ] + expected = { + di.DockerImage("docker/test/base", "clickhouse/test-base"), + di.DockerImage("docker/docs/builder", "clickhouse/docs-builder"), + di.DockerImage("docker/test/stateless", "clickhouse/stateless-test"), + di.DockerImage( + "docker/test/integration/base", "clickhouse/integration-test" + ), + di.DockerImage("docker/test/fuzzer", "clickhouse/fuzzer"), + di.DockerImage( + "docker/test/keeper-jepsen", "clickhouse/keeper-jepsen-test" + ), + di.DockerImage("docker/docs/check", "clickhouse/docs-check"), + di.DockerImage("docker/docs/release", "clickhouse/docs-release"), + di.DockerImage("docker/test/stateful", "clickhouse/stateful-test"), + di.DockerImage("docker/test/unit", "clickhouse/unit-test"), + di.DockerImage("docker/test/stress", "clickhouse/stress-test"), + } self.assertEqual(images, expected) + def test_gen_version(self): + pr_info = PRInfo(PRInfo.default_event.copy()) + versions, result_version = di.gen_versions(pr_info, None) + self.assertEqual(versions, ["latest", "0", "0-HEAD"]) + self.assertEqual(result_version, "0-HEAD") + versions, result_version = di.gen_versions(pr_info, "suffix") + self.assertEqual(versions, ["latest-suffix", "0-suffix", "0-HEAD-suffix"]) + self.assertEqual(result_version, versions) + pr_info.number = 1 + versions, result_version = di.gen_versions(pr_info, None) + self.assertEqual(versions, ["1", "1-HEAD"]) + self.assertEqual(result_version, "1-HEAD") + + @patch("builtins.open") + @patch("subprocess.Popen") + def test_build_and_push_one_image(self, mock_popen, mock_open): + mock_popen.return_value.__enter__.return_value.wait.return_value = 0 + image = di.DockerImage("path", "name", gh_repo_path="") + + result, _ = di.build_and_push_one_image(image, "version", True, True) + mock_open.assert_called_once() + mock_popen.assert_called_once() + self.assertIn( + "docker buildx build --builder default --build-arg FROM_TAG=version " + "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version --cache-from " + "type=registry,ref=name:version --push --progress plain path", + mock_popen.call_args.args, + ) + self.assertTrue(result) + + mock_open.reset() + mock_popen.reset() + mock_popen.return_value.__enter__.return_value.wait.return_value = 0 + result, _ = di.build_and_push_one_image(image, "version2", False, True) + self.assertIn( + "docker buildx build --builder default --build-arg FROM_TAG=version2 " + "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " + "type=registry,ref=name:version2 --progress plain path", + mock_popen.call_args.args, + ) + self.assertTrue(result) + + mock_popen.return_value.__enter__.return_value.wait.return_value = 1 + result, _ = di.build_and_push_one_image(image, "version2", False, False) + self.assertIn( + "docker buildx build --builder default " + "--build-arg BUILDKIT_INLINE_CACHE=1 --tag name:version2 --cache-from " + "type=registry,ref=name:version2 --progress plain path", + mock_popen.call_args.args, + ) + self.assertFalse(result) + if __name__ == "__main__": unittest.main() From 70d997cbdab60d48e55a70049bea5f6554b57c91 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jan 2022 22:28:26 +0100 Subject: [PATCH 31/37] Fix pylint issues --- tests/ci/docker_images_check.py | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 2fbca7ab548..24481878da7 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -58,7 +58,7 @@ def get_changed_docker_images( images_dict = {} path_to_images_file = os.path.join(repo_path, image_file_path) if os.path.exists(path_to_images_file): - with open(path_to_images_file, "r") as dict_file: + with open(path_to_images_file, "rb") as dict_file: images_dict = json.load(dict_file) else: logging.info( @@ -159,10 +159,7 @@ def build_and_push_one_image( image.full_path, ) build_log = os.path.join( - TEMP_PATH, - "build_and_push_log_{}_{}".format( - str(image.repo).replace("/", "_"), version_string - ), + TEMP_PATH, f"build_and_push_log_{image.repo.replace('/', '_')}_{version_string}" ) push_arg = "" if push: @@ -172,7 +169,7 @@ def build_and_push_one_image( if child: from_tag_arg = f"--build-arg FROM_TAG={version_string} " - with open(build_log, "w") as bl: + with open(build_log, "wb") as bl: cmd = ( "docker buildx build --builder default " f"{from_tag_arg}" @@ -247,7 +244,7 @@ def process_test_results( build_url = s3_client.upload_test_report_to_s3( build_log, s3_path_prefix + "/" + os.path.basename(build_log) ) - url_part += 'build_log'.format(build_url) + url_part += f'build_log' if url_part: test_name = image + " (" + url_part + ")" else: @@ -353,7 +350,7 @@ def main(): if len(description) >= 140: description = description[:136] + "..." - with open(changed_json, "w") as images_file: + with open(changed_json, "w", encoding="utf-8") as images_file: json.dump(result_images, images_file) s3_helper = S3Helper("https://s3.amazonaws.com") @@ -367,8 +364,8 @@ def main(): url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME) - print("::notice ::Report url: {}".format(url)) - print('::set-output name=url_output::"{}"'.format(url)) + print(f"::notice ::Report url: {url}") + print(f'::set-output name=url_output::"{url}"') if args.no_reports: return From d38b4f544032c41ccb2425b1aaae5546926f3473 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 18 Jan 2022 22:33:53 +0100 Subject: [PATCH 32/37] Add unit tests for python to CI --- .github/workflows/master.yml | 14 +++++++++++++- .github/workflows/pull_request.yml | 15 ++++++++++++++- 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 3970e64f959..f1e1fd51d21 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -9,6 +9,18 @@ on: # yamllint disable-line rule:truthy branches: - 'master' jobs: + PythonUnitTests: + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Python unit tests + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 -m unittest discover -s . -p '*_test.py' DockerHubPushAarch64: runs-on: [self-hosted, func-tester-aarch64] steps: @@ -44,7 +56,7 @@ jobs: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json DockerHubPush: - needs: [DockerHubPushAmd64, DockerHubPushAarch64] + needs: [DockerHubPushAmd64, DockerHubPushAarch64, PythonUnitTests] runs-on: [self-hosted, style-checker] steps: - name: Clear repository diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 0bd02de48d0..b131e3e8a12 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -31,6 +31,19 @@ jobs: run: | cd "$GITHUB_WORKSPACE/tests/ci" python3 run_check.py + PythonUnitTests: + needs: CheckLabels + runs-on: [self-hosted, style-checker] + steps: + - name: Clear repository + run: | + sudo rm -fr "$GITHUB_WORKSPACE" && mkdir "$GITHUB_WORKSPACE" + - name: Check out repository code + uses: actions/checkout@v2 + - name: Python unit tests + run: | + cd "$GITHUB_WORKSPACE/tests/ci" + python3 -m unittest discover -s . -p '*_test.py' DockerHubPushAarch64: needs: CheckLabels runs-on: [self-hosted, func-tester-aarch64] @@ -68,7 +81,7 @@ jobs: name: changed_images_amd64 path: ${{ runner.temp }}/docker_images_check/changed_images_amd64.json DockerHubPush: - needs: [DockerHubPushAmd64, DockerHubPushAarch64] + needs: [DockerHubPushAmd64, DockerHubPushAarch64, PythonUnitTests] runs-on: [self-hosted, style-checker] steps: - name: Clear repository From 7a1f471d6f184ceb78577f115d0850aa8a5ab4e3 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 19 Jan 2022 11:26:16 +0100 Subject: [PATCH 33/37] Add tests for process_image_with_parrents --- tests/ci/docker_test.py | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/tests/ci/docker_test.py b/tests/ci/docker_test.py index 3fed5112005..e3dcda6afe5 100644 --- a/tests/ci/docker_test.py +++ b/tests/ci/docker_test.py @@ -94,6 +94,43 @@ class TestDockerImageCheck(unittest.TestCase): ) self.assertFalse(result) + @patch("docker_images_check.build_and_push_one_image") + def test_process_image_with_parents(self, mock_build): + mock_build.side_effect = lambda w, x, y, z: (True, f"{w.repo}_{x}.log") + im1 = di.DockerImage("path1", "repo1") + im2 = di.DockerImage("path2", "repo2", im1) + im3 = di.DockerImage("path3", "repo3", im2) + im4 = di.DockerImage("path4", "repo4", im1) + # We use list to have determined order of image builgings + images = [im4, im1, im3, im2, im1] + results = [ + di.process_image_with_parents(im, ["v1", "v2", "latest"], True) + for im in images + ] + + expected = [ + [ # repo4 -> repo1 + ("repo1:v1", "repo1_v1.log", "OK"), + ("repo1:v2", "repo1_v2.log", "OK"), + ("repo1:latest", "repo1_latest.log", "OK"), + ("repo4:v1", "repo4_v1.log", "OK"), + ("repo4:v2", "repo4_v2.log", "OK"), + ("repo4:latest", "repo4_latest.log", "OK"), + ], + [], # repo1 is built + [ # repo3 -> repo2 -> repo1 + ("repo2:v1", "repo2_v1.log", "OK"), + ("repo2:v2", "repo2_v2.log", "OK"), + ("repo2:latest", "repo2_latest.log", "OK"), + ("repo3:v1", "repo3_v1.log", "OK"), + ("repo3:v2", "repo3_v2.log", "OK"), + ("repo3:latest", "repo3_latest.log", "OK"), + ], + [], # repo2 -> repo1 are built + [], # repo1 is built + ] + self.assertEqual(results, expected) + if __name__ == "__main__": unittest.main() From 4a0facd341a7e79d80abbd30951862a5fdb9eea4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Jan 2022 14:32:17 +0300 Subject: [PATCH 34/37] Remove MAKE_STATIC_LIBRARIES (in favor of USE_STATIC_LIBRARIES) There is no more MAKE_*, so remove this alias. Signed-off-by: Azat Khuzhin --- CMakeLists.txt | 16 +++++++--------- base/base/CMakeLists.txt | 2 +- base/daemon/CMakeLists.txt | 2 +- base/glibc-compatibility/CMakeLists.txt | 2 +- cmake/sanitize.cmake | 8 ++++---- contrib/arrow-cmake/CMakeLists.txt | 4 ++-- contrib/grpc-cmake/CMakeLists.txt | 2 +- contrib/libuv-cmake/CMakeLists.txt | 2 +- programs/CMakeLists.txt | 2 +- src/CMakeLists.txt | 6 +++--- tests/integration/CMakeLists.txt | 2 +- utils/CMakeLists.txt | 2 +- 12 files changed, 24 insertions(+), 26 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index d33bdd8b33c..498fef3fe58 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -104,9 +104,8 @@ message (STATUS "CMAKE_BUILD_TYPE: ${CMAKE_BUILD_TYPE}") string (TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) option(USE_STATIC_LIBRARIES "Disable to use shared libraries" ON) -option(MAKE_STATIC_LIBRARIES "Disable to make shared libraries" ${USE_STATIC_LIBRARIES}) -if (NOT MAKE_STATIC_LIBRARIES) +if (NOT USE_STATIC_LIBRARIES) # DEVELOPER ONLY. # Faster linking if turned on. option(SPLIT_SHARED_LIBRARIES "Keep all internal libraries as separate .so files") @@ -115,11 +114,11 @@ if (NOT MAKE_STATIC_LIBRARIES) "Make several binaries (clickhouse-server, clickhouse-client etc.) instead of one bundled") endif () -if (MAKE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) - message(FATAL_ERROR "Defining SPLIT_SHARED_LIBRARIES=1 without MAKE_STATIC_LIBRARIES=0 has no effect.") +if (USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) + message(FATAL_ERROR "Defining SPLIT_SHARED_LIBRARIES=1 without USE_STATIC_LIBRARIES=0 has no effect.") endif() -if (NOT MAKE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) +if (NOT USE_STATIC_LIBRARIES AND SPLIT_SHARED_LIBRARIES) set(BUILD_SHARED_LIBS 1 CACHE INTERNAL "") endif () @@ -201,7 +200,7 @@ endif () option(ENABLE_TESTS "Provide unit_test_dbms target with Google.Test unit tests" ON) option(ENABLE_EXAMPLES "Build all example programs in 'examples' subdirectories" OFF) -if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND MAKE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND NOT USE_MUSL) +if (OS_LINUX AND (ARCH_AMD64 OR ARCH_AARCH64) AND USE_STATIC_LIBRARIES AND NOT SPLIT_SHARED_LIBRARIES AND NOT USE_MUSL) # Only for Linux, x86_64 or aarch64. option(GLIBC_COMPATIBILITY "Enable compatibility with older glibc libraries." ON) elseif(GLIBC_COMPATIBILITY) @@ -444,7 +443,7 @@ endif () set (CMAKE_POSTFIX_VARIABLE "CMAKE_${CMAKE_BUILD_TYPE_UC}_POSTFIX") -if (MAKE_STATIC_LIBRARIES) +if (USE_STATIC_LIBRARIES) set (CMAKE_POSITION_INDEPENDENT_CODE OFF) if (OS_LINUX AND NOT ARCH_ARM) # Slightly more efficient code can be generated @@ -480,7 +479,6 @@ endif () message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE_LIBRARY_ARCHITECTURE} ; USE_STATIC_LIBRARIES=${USE_STATIC_LIBRARIES} - MAKE_STATIC_LIBRARIES=${MAKE_STATIC_LIBRARIES} SPLIT_SHARED=${SPLIT_SHARED_LIBRARIES} CCACHE=${CCACHE_FOUND} ${CCACHE_VERSION}") @@ -528,7 +526,7 @@ macro (add_executable target) # - _je_zone_register due to JEMALLOC_PRIVATE_NAMESPACE=je_ under OS X. # - but jemalloc-cmake does not run private_namespace.sh # so symbol name should be _zone_register - if (ENABLE_JEMALLOC AND MAKE_STATIC_LIBRARIES AND OS_DARWIN) + if (ENABLE_JEMALLOC AND USE_STATIC_LIBRARIES AND OS_DARWIN) set_property(TARGET ${target} APPEND PROPERTY LINK_OPTIONS -u_zone_register) endif() endif() diff --git a/base/base/CMakeLists.txt b/base/base/CMakeLists.txt index 9201a852373..3e6f174c6dc 100644 --- a/base/base/CMakeLists.txt +++ b/base/base/CMakeLists.txt @@ -42,7 +42,7 @@ endif () target_include_directories(common PUBLIC .. "${CMAKE_CURRENT_BINARY_DIR}/..") -if (OS_DARWIN AND NOT MAKE_STATIC_LIBRARIES) +if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES) target_link_libraries(common PUBLIC -Wl,-U,_inside_main) endif() diff --git a/base/daemon/CMakeLists.txt b/base/daemon/CMakeLists.txt index 2a4d3d33ff9..ae8f51cabd3 100644 --- a/base/daemon/CMakeLists.txt +++ b/base/daemon/CMakeLists.txt @@ -6,7 +6,7 @@ add_library (daemon target_include_directories (daemon PUBLIC ..) -if (OS_DARWIN AND NOT MAKE_STATIC_LIBRARIES) +if (OS_DARWIN AND NOT USE_STATIC_LIBRARIES) target_link_libraries (daemon PUBLIC -Wl,-undefined,dynamic_lookup) endif() diff --git a/base/glibc-compatibility/CMakeLists.txt b/base/glibc-compatibility/CMakeLists.txt index 4fc2a002cd8..ddec09121e1 100644 --- a/base/glibc-compatibility/CMakeLists.txt +++ b/base/glibc-compatibility/CMakeLists.txt @@ -37,7 +37,7 @@ if (GLIBC_COMPATIBILITY) target_include_directories(glibc-compatibility PRIVATE libcxxabi ${musl_arch_include_dir}) - if (NOT USE_STATIC_LIBRARIES AND NOT MAKE_STATIC_LIBRARIES) + if (NOT USE_STATIC_LIBRARIES AND NOT USE_STATIC_LIBRARIES) target_compile_options(glibc-compatibility PRIVATE -fPIC) endif () diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index f052948e731..73610545009 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -23,7 +23,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${ASAN_FLAGS}") endif() - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan") endif () if (COMPILER_GCC) @@ -48,7 +48,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=memory") endif() - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libmsan") endif () @@ -69,7 +69,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=thread") endif() - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libtsan") endif () if (COMPILER_GCC) @@ -101,7 +101,7 @@ if (SANITIZE) if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=undefined") endif() - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + if (USE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan") endif () if (COMPILER_GCC) diff --git a/contrib/arrow-cmake/CMakeLists.txt b/contrib/arrow-cmake/CMakeLists.txt index 5e500877f3c..54bfead6da7 100644 --- a/contrib/arrow-cmake/CMakeLists.txt +++ b/contrib/arrow-cmake/CMakeLists.txt @@ -29,7 +29,7 @@ if (OS_FREEBSD) message (FATAL_ERROR "Using internal parquet library on FreeBSD is not supported") endif() -if(MAKE_STATIC_LIBRARIES) +if(USE_STATIC_LIBRARIES) set(FLATBUFFERS_LIBRARY flatbuffers) else() set(FLATBUFFERS_LIBRARY flatbuffers_shared) @@ -84,7 +84,7 @@ set(FLATBUFFERS_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/flatbuffers") set(FLATBUFFERS_INCLUDE_DIR "${FLATBUFFERS_SRC_DIR}/include") # set flatbuffers CMake options -if (MAKE_STATIC_LIBRARIES) +if (USE_STATIC_LIBRARIES) set(FLATBUFFERS_BUILD_FLATLIB ON CACHE BOOL "Enable the build of the flatbuffers library") set(FLATBUFFERS_BUILD_SHAREDLIB OFF CACHE BOOL "Disable the build of the flatbuffers shared library") else () diff --git a/contrib/grpc-cmake/CMakeLists.txt b/contrib/grpc-cmake/CMakeLists.txt index 15b7550e810..520e04d198e 100644 --- a/contrib/grpc-cmake/CMakeLists.txt +++ b/contrib/grpc-cmake/CMakeLists.txt @@ -46,7 +46,7 @@ set(_gRPC_SSL_LIBRARIES OpenSSL::Crypto OpenSSL::SSL) set(gRPC_ABSL_PROVIDER "clickhouse" CACHE STRING "" FORCE) # Choose to build static or shared library for c-ares. -if (MAKE_STATIC_LIBRARIES) +if (USE_STATIC_LIBRARIES) set(CARES_STATIC ON CACHE BOOL "" FORCE) set(CARES_SHARED OFF CACHE BOOL "" FORCE) else () diff --git a/contrib/libuv-cmake/CMakeLists.txt b/contrib/libuv-cmake/CMakeLists.txt index 3c06bdcf6d0..45f6d8e2083 100644 --- a/contrib/libuv-cmake/CMakeLists.txt +++ b/contrib/libuv-cmake/CMakeLists.txt @@ -133,7 +133,7 @@ add_library(ch_contrib::uv ALIAS _uv) target_compile_definitions(_uv PRIVATE ${uv_defines}) target_include_directories(_uv SYSTEM PUBLIC ${SOURCE_DIR}/include PRIVATE ${SOURCE_DIR}/src) target_link_libraries(_uv ${uv_libraries}) -if (NOT MAKE_STATIC_LIBRARIES) +if (NOT USE_STATIC_LIBRARIES) target_compile_definitions(_uv INTERFACE USING_UV_SHARED=1 PRIVATE BUILDING_UV_SHARED=1) diff --git a/programs/CMakeLists.txt b/programs/CMakeLists.txt index f897e9812a8..8906d186bfc 100644 --- a/programs/CMakeLists.txt +++ b/programs/CMakeLists.txt @@ -160,7 +160,7 @@ else() message(STATUS "ClickHouse keeper-converter mode: OFF") endif() -if(NOT (MAKE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES)) +if(NOT (USE_STATIC_LIBRARIES OR SPLIT_SHARED_LIBRARIES)) set(CLICKHOUSE_ONE_SHARED ON) endif() diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index c18a9a3aa97..7b4373be95f 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -11,7 +11,7 @@ if(COMPILER_PIPE) else() set(MAX_COMPILER_MEMORY 1500) endif() -if(MAKE_STATIC_LIBRARIES) +if(USE_STATIC_LIBRARIES) set(MAX_LINKER_MEMORY 3500) else() set(MAX_LINKER_MEMORY 2500) @@ -193,7 +193,7 @@ add_subdirectory(Common/Config) set (all_modules) macro(add_object_library name common_path) - if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) + if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) add_headers_and_sources(dbms ${common_path}) else () list (APPEND all_modules ${name}) @@ -254,7 +254,7 @@ endif() set (DBMS_COMMON_LIBRARIES) -if (MAKE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) +if (USE_STATIC_LIBRARIES OR NOT SPLIT_SHARED_LIBRARIES) add_library (dbms STATIC ${dbms_headers} ${dbms_sources}) target_link_libraries (dbms PRIVATE ch_contrib::libdivide ${DBMS_COMMON_LIBRARIES}) if (TARGET ch_contrib::jemalloc) diff --git a/tests/integration/CMakeLists.txt b/tests/integration/CMakeLists.txt index f57ade79471..68c695f57a0 100644 --- a/tests/integration/CMakeLists.txt +++ b/tests/integration/CMakeLists.txt @@ -10,7 +10,7 @@ find_program(PYTEST_CMD pytest) find_program(SUDO_CMD sudo) # will mount only one binary to docker container - build with .so cant work -if(MAKE_STATIC_LIBRARIES AND DOCKER_CMD) +if(USE_STATIC_LIBRARIES AND DOCKER_CMD) if(INTEGRATION_USE_RUNNER AND SUDO_CMD) add_test(NAME integration-runner WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND ${SUDO_CMD} ${CMAKE_CURRENT_SOURCE_DIR}/runner --binary ${ClickHouse_BINARY_DIR}/programs/clickhouse --configs-dir ${ClickHouse_SOURCE_DIR}/programs/server/) message(STATUS "Using tests in docker with runner SUDO=${SUDO_CMD}; DOCKER=${DOCKER_CMD};") diff --git a/utils/CMakeLists.txt b/utils/CMakeLists.txt index 706532e2ac9..7822f47ff88 100644 --- a/utils/CMakeLists.txt +++ b/utils/CMakeLists.txt @@ -2,7 +2,7 @@ if (USE_CLANG_TIDY) set (CMAKE_CXX_CLANG_TIDY "${CLANG_TIDY_PATH}") endif () -if(MAKE_STATIC_LIBRARIES) +if(USE_STATIC_LIBRARIES) set(MAX_LINKER_MEMORY 3500) else() set(MAX_LINKER_MEMORY 2500) From 8e8ee8f8d15a02dcb3a6b5c92160cb91a30de49c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 24 Jan 2022 15:50:34 +0100 Subject: [PATCH 35/37] Add style-checker-aarch64 to metrics lambda --- tests/ci/metrics_lambda/app.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/metrics_lambda/app.py b/tests/ci/metrics_lambda/app.py index 1b63af30b59..0d84e7690a8 100644 --- a/tests/ci/metrics_lambda/app.py +++ b/tests/ci/metrics_lambda/app.py @@ -174,6 +174,7 @@ def group_runners_by_tag(listed_runners): "fuzzer-unit-tester", "stress-tester", "style-checker", + "style-checker-aarch64", ] for runner in listed_runners: for tag in runner.tags: From fc1c80f1be4546808724818eae1a84b47a16bb0b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 24 Jan 2022 15:55:18 +0100 Subject: [PATCH 36/37] Launch docker images building on on-demand hosts --- .github/actionlint.yml | 5 +++-- .github/workflows/backport_branches.yml | 2 +- .github/workflows/docs_check.yml | 2 +- .github/workflows/docs_release.yml | 2 +- .github/workflows/master.yml | 2 +- .github/workflows/pull_request.yml | 2 +- .github/workflows/release_branches.yml | 2 +- 7 files changed, 9 insertions(+), 8 deletions(-) diff --git a/.github/actionlint.yml b/.github/actionlint.yml index 8083186117f..0f88f30d42c 100644 --- a/.github/actionlint.yml +++ b/.github/actionlint.yml @@ -1,8 +1,9 @@ self-hosted-runner: labels: - builder + - func-tester + - func-tester-aarch64 - fuzzer-unit-tester - stress-tester - style-checker - - func-tester-aarch64 - - func-tester + - style-checker-aarch64 diff --git a/.github/workflows/backport_branches.yml b/.github/workflows/backport_branches.yml index 749c248af82..75f8a63368d 100644 --- a/.github/workflows/backport_branches.yml +++ b/.github/workflows/backport_branches.yml @@ -10,7 +10,7 @@ on: # yamllint disable-line rule:truthy - 'backport/**' jobs: DockerHubPushAarch64: - runs-on: [self-hosted, func-tester-aarch64] + runs-on: [self-hosted, style-checker-aarch64] steps: - name: Clear repository run: | diff --git a/.github/workflows/docs_check.yml b/.github/workflows/docs_check.yml index 633e654d656..d5b56bfef32 100644 --- a/.github/workflows/docs_check.yml +++ b/.github/workflows/docs_check.yml @@ -30,7 +30,7 @@ jobs: python3 run_check.py DockerHubPushAarch64: needs: CheckLabels - runs-on: [self-hosted, func-tester-aarch64] + runs-on: [self-hosted, style-checker-aarch64] steps: - name: Clear repository run: | diff --git a/.github/workflows/docs_release.yml b/.github/workflows/docs_release.yml index 54e1f27ab7c..66838a05552 100644 --- a/.github/workflows/docs_release.yml +++ b/.github/workflows/docs_release.yml @@ -20,7 +20,7 @@ on: # yamllint disable-line rule:truthy workflow_dispatch: jobs: DockerHubPushAarch64: - runs-on: [self-hosted, func-tester-aarch64] + runs-on: [self-hosted, style-checker-aarch64] steps: - name: Clear repository run: | diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index f1e1fd51d21..91b9ea5bf3d 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -22,7 +22,7 @@ jobs: cd "$GITHUB_WORKSPACE/tests/ci" python3 -m unittest discover -s . -p '*_test.py' DockerHubPushAarch64: - runs-on: [self-hosted, func-tester-aarch64] + runs-on: [self-hosted, style-checker-aarch64] steps: - name: Clear repository run: | diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index b131e3e8a12..cd8517de8fe 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -46,7 +46,7 @@ jobs: python3 -m unittest discover -s . -p '*_test.py' DockerHubPushAarch64: needs: CheckLabels - runs-on: [self-hosted, func-tester-aarch64] + runs-on: [self-hosted, style-checker-aarch64] steps: - name: Clear repository run: | diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index 4ab2638069c..d916699acc2 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -13,7 +13,7 @@ on: # yamllint disable-line rule:truthy jobs: DockerHubPushAarch64: - runs-on: [self-hosted, func-tester-aarch64] + runs-on: [self-hosted, style-checker-aarch64] steps: - name: Clear repository run: | From 57b4655fa276ef7efe82e5f285f7ed91f082008d Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 24 Jan 2022 16:34:01 +0100 Subject: [PATCH 37/37] Fix style for pr_info --- tests/ci/pr_info.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 2fe7254f610..f638961ff79 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -12,7 +12,7 @@ DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xm def get_pr_for_commit(sha, ref): if not ref: - return + return None try_get_pr_url = f"https://api.github.com/repos/{GITHUB_REPOSITORY}/commits/{sha}/pulls" try: response = requests.get(try_get_pr_url)